diff --git a/akka-actor/src/main/resources/logback.xml b/akka-actor/src/main/resources/logback.xml deleted file mode 100644 index 4635396601..0000000000 --- a/akka-actor/src/main/resources/logback.xml +++ /dev/null @@ -1,31 +0,0 @@ - - - - - - - - - - - - - - [%4p] [%d{ISO8601}] [%t] %c{1}: %m%n - - - - ./logs/akka.log - - [%4p] [%d{ISO8601}] [%t] %c{1}: %m%n - - - ./logs/akka.log.%d{yyyy-MM-dd-HH} - - - - - - - - diff --git a/akka-actor/src/main/scala/actor/Actor.scala b/akka-actor/src/main/scala/actor/Actor.scala index d3e7699403..d232ca2a77 100644 --- a/akka-actor/src/main/scala/actor/Actor.scala +++ b/akka-actor/src/main/scala/actor/Actor.scala @@ -60,8 +60,8 @@ case object ReceiveTimeout extends LifeCycleMessage case class MaximumNumberOfRestartsWithinTimeRangeReached( @BeanProperty val victim: ActorRef, - @BeanProperty val maxNrOfRetries: Int, - @BeanProperty val withinTimeRange: Int, + @BeanProperty val maxNrOfRetries: Option[Int], + @BeanProperty val withinTimeRange: Option[Int], @BeanProperty val lastExceptionCausingRestart: Throwable) extends LifeCycleMessage // Exceptions for Actors @@ -410,14 +410,14 @@ trait Actor extends Logging { *

* Is called when an Actor is started by invoking 'actor.start'. */ - def init {} + def preStart {} /** * User overridable callback. *

* Is called when 'actor.stop' is invoked. */ - def shutdown {} + def postStop {} /** * User overridable callback. @@ -433,13 +433,6 @@ trait Actor extends Logging { */ def postRestart(reason: Throwable) {} - /** - * User overridable callback. - *

- * Is called during initialization. Can be used to initialize transactional state. Will be invoked within a transaction. - */ - def initTransactionalState {} - /** * Is the actor able to handle the message passed in as arguments? */ diff --git a/akka-actor/src/main/scala/actor/ActorRef.scala b/akka-actor/src/main/scala/actor/ActorRef.scala index a6b42db579..4905e62670 100644 --- a/akka-actor/src/main/scala/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/actor/ActorRef.scala @@ -73,7 +73,6 @@ trait ActorRef extends @volatile protected[akka] var _isBeingRestarted = false @volatile protected[akka] var _homeAddress = new InetSocketAddress(RemoteServerModule.HOSTNAME, RemoteServerModule.PORT) @volatile protected[akka] var _futureTimeout: Option[ScheduledFuture[AnyRef]] = None - @volatile protected[akka] var startOnCreation = false @volatile protected[akka] var registeredInRemoteNodeDuringSerialization = false protected[akka] val guard = new ReentrantGuard @@ -197,19 +196,10 @@ trait ActorRef extends */ @volatile private[akka] var _transactionFactory: Option[TransactionFactory] = None - /** - * This lock ensures thread safety in the dispatching: only one message can - * be dispatched at once on the actor. - */ - protected[akka] val dispatcherLock = new ReentrantLock - /** * This is a reference to the message currently being processed by the actor */ - protected[akka] var _currentMessage: Option[MessageInvocation] = None - - protected[akka] def currentMessage_=(msg: Option[MessageInvocation]) = guard.withGuard { _currentMessage = msg } - protected[akka] def currentMessage = guard.withGuard { _currentMessage } + @volatile protected[akka] var currentMessage: MessageInvocation = null /** * Comparison only takes uuid into account. @@ -611,9 +601,9 @@ trait ActorRef extends protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit - protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit + protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit - protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit + protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit protected[akka] def registerSupervisorAsRemoteActor: Option[String] @@ -827,12 +817,11 @@ class LocalActorRef private[akka]( _transactionFactory = None _isRunning = false _isShutDown = true - actor.shutdown + actor.postStop ActorRegistry.unregister(this) if (isRemotingEnabled) { - remoteAddress.foreach { address => - RemoteClientModule.unregister(address, uuid) - } + if(remoteAddress.isDefined) + RemoteClientModule.unregister(remoteAddress.get, uuid) RemoteServerModule.unregister(this) } nullOutActorRefReferencesFor(actorInstance.get) @@ -875,11 +864,11 @@ class LocalActorRef private[akka]( *

* To be invoked from within the actor itself. */ - def startLink(actorRef: ActorRef) = guard.withGuard { + def startLink(actorRef: ActorRef):Unit = guard.withGuard { try { - actorRef.start - } finally { link(actorRef) + } finally { + actorRef.start } } @@ -888,13 +877,13 @@ class LocalActorRef private[akka]( *

* To be invoked from within the actor itself. */ - def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int) = guard.withGuard { + def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int): Unit = guard.withGuard { ensureRemotingEnabled try { actorRef.makeRemote(hostname, port) - actorRef.start - } finally { link(actorRef) + } finally { + actorRef.start } } @@ -904,9 +893,7 @@ class LocalActorRef private[akka]( * To be invoked from within the actor itself. */ def spawn(clazz: Class[_ <: Actor]): ActorRef = guard.withGuard { - val actorRef = spawnButDoNotStart(clazz) - actorRef.start - actorRef + spawnButDoNotStart(clazz).start } /** @@ -930,9 +917,9 @@ class LocalActorRef private[akka]( def spawnLink(clazz: Class[_ <: Actor]): ActorRef = guard.withGuard { val actor = spawnButDoNotStart(clazz) try { - actor.start - } finally { link(actor) + } finally { + actor.start } actor } @@ -947,10 +934,11 @@ class LocalActorRef private[akka]( val actor = spawnButDoNotStart(clazz) try { actor.makeRemote(hostname, port) - actor.start - } finally { link(actor) + } finally { + actor.start } + actor } /** @@ -980,12 +968,12 @@ class LocalActorRef private[akka]( protected[akka] def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit = { joinTransaction(message) - if (isRemotingEnabled && remoteAddress.isDefined) { + if (remoteAddress.isDefined && isRemotingEnabled) { RemoteClientModule.send[Any]( message, senderOption, None, remoteAddress.get, timeout, true, this, None, ActorType.ScalaActor) } else { val invocation = new MessageInvocation(this, message, senderOption, None, transactionSet.get) - invocation.send + dispatcher dispatch invocation } } @@ -996,7 +984,7 @@ class LocalActorRef private[akka]( senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T] = { joinTransaction(message) - if (isRemotingEnabled && remoteAddress.isDefined) { + if (remoteAddress.isDefined && isRemotingEnabled) { val future = RemoteClientModule.send[T]( message, senderOption, senderFuture, remoteAddress.get, timeout, false, this, None, ActorType.ScalaActor) if (future.isDefined) future.get @@ -1006,7 +994,7 @@ class LocalActorRef private[akka]( else new DefaultCompletableFuture[T](timeout) val invocation = new MessageInvocation( this, message, senderOption, Some(future.asInstanceOf[CompletableFuture[Any]]), transactionSet.get) - invocation.send + dispatcher dispatch invocation future } } @@ -1018,7 +1006,7 @@ class LocalActorRef private[akka]( if (isShutdown) Actor.log.warning("Actor [%s] is shut down,\n\tignoring message [%s]", toString, messageHandle) else { - currentMessage = Option(messageHandle) + currentMessage = messageHandle try { dispatch(messageHandle) } catch { @@ -1026,7 +1014,7 @@ class LocalActorRef private[akka]( Actor.log.error(e, "Could not invoke actor [%s]", this) throw e } finally { - currentMessage = None //TODO: Don't reset this, we might want to resend the message + currentMessage = null //TODO: Don't reset this, we might want to resend the message } } } @@ -1049,12 +1037,18 @@ class LocalActorRef private[akka]( } } - protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = { + protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = { if (maxNrOfRetriesCount == 0) restartsWithinTimeRangeTimestamp = System.currentTimeMillis // first time around - maxNrOfRetriesCount += 1 + + val tooManyRestarts = if (maxNrOfRetries.isDefined) { + maxNrOfRetriesCount += 1 + maxNrOfRetriesCount > maxNrOfRetries.get + } else false + + val restartingHasExpired = if (withinTimeRange.isDefined) + (System.currentTimeMillis - restartsWithinTimeRangeTimestamp) > withinTimeRange.get + else false - val tooManyRestarts = maxNrOfRetriesCount > maxNrOfRetries - val restartingHasExpired = (System.currentTimeMillis - restartsWithinTimeRangeTimestamp) > withinTimeRange if (tooManyRestarts || restartingHasExpired) { val notification = MaximumNumberOfRestartsWithinTimeRangeReached(this, maxNrOfRetries, withinTimeRange, reason) Actor.log.warning( @@ -1092,7 +1086,7 @@ class LocalActorRef private[akka]( } } - protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int) = { + protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) = { linkedActorsAsList.foreach { actorRef => actorRef.lifeCycle match { // either permanent or none where default is permanent @@ -1134,8 +1128,7 @@ class LocalActorRef private[akka]( failedActor.preRestart(reason) nullOutActorRefReferencesFor(failedActor) val freshActor = newActor - freshActor.init - freshActor.initTransactionalState + freshActor.preStart actorInstance.set(freshActor) if (failedActor.isInstanceOf[Proxyable]) failedActor.asInstanceOf[Proxyable].swapProxiedActor(freshActor) @@ -1143,26 +1136,20 @@ class LocalActorRef private[akka]( freshActor.postRestart(reason) } - private def spawnButDoNotStart(clazz: Class[_ <: Actor]): ActorRef = guard.withGuard { - val actorRef = Actor.actorOf(clazz.newInstance) - if (!dispatcher.isInstanceOf[ThreadBasedDispatcher]) actorRef.dispatcher = dispatcher - actorRef - } + private def spawnButDoNotStart(clazz: Class[_ <: Actor]): ActorRef = Actor.actorOf(clazz.newInstance) private[this] def newActor: Actor = { Actor.actorRefInCreation.withValue(Some(this)){ isInInitialization = true val actor = actorFactory match { case Left(Some(clazz)) => - try { - clazz.newInstance - } catch { - case e: InstantiationException => throw new ActorInitializationException( - "Could not instantiate Actor due to:\n" + e + + import ReflectiveAccess.{createInstance,noParams,noArgs} + createInstance(clazz.asInstanceOf[Class[_]],noParams,noArgs). + getOrElse(throw new ActorInitializationException( + "Could not instantiate Actor" + "\nMake sure Actor is NOT defined inside a class/trait," + "\nif so put it outside the class/trait, f.e. in a companion object," + - "\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'.") - } + "\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'.")) case Right(Some(factory)) => factory() case _ => @@ -1192,7 +1179,7 @@ class LocalActorRef private[akka]( } private def dispatch[T](messageHandle: MessageInvocation) = { - Actor.log.trace("Invoking actor with message:\n" + messageHandle) + Actor.log.trace("Invoking actor with message: %s\n",messageHandle) val message = messageHandle.message //serializeMessage(messageHandle.message) var topLevelTransaction = false val txSet: Option[CountDownCommitBarrier] = @@ -1305,8 +1292,7 @@ class LocalActorRef private[akka]( } private def initializeActorInstance = { - actor.init // run actor init and initTransactionalState callbacks - actor.initTransactionalState + actor.preStart // run actor preStart Actor.log.trace("[%s] has started", toString) ActorRegistry.register(this) if (id == "N/A") id = actorClass.getName // if no name set, then use default name (class name) @@ -1357,17 +1343,18 @@ object RemoteActorSystemMessage { * @author Jonas Bonér */ private[akka] case class RemoteActorRef private[akka] ( - uuuid: String, + classOrServiceName: String, val className: String, val hostname: String, val port: Int, _timeout: Long, - loader: Option[ClassLoader]) + loader: Option[ClassLoader], + val actorType: ActorType = ActorType.ScalaActor) extends ActorRef with ScalaActorRef { ensureRemotingEnabled - _uuid = uuuid + id = classOrServiceName timeout = _timeout start @@ -1375,7 +1362,7 @@ private[akka] case class RemoteActorRef private[akka] ( def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit = RemoteClientModule.send[Any]( - message, senderOption, None, remoteAddress.get, timeout, true, this, None, ActorType.ScalaActor) + message, senderOption, None, remoteAddress.get, timeout, true, this, None, actorType) def postMessageToMailboxAndCreateFutureResultWithTimeout[T]( message: Any, @@ -1383,7 +1370,7 @@ private[akka] case class RemoteActorRef private[akka] ( senderOption: Option[ActorRef], senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T] = { val future = RemoteClientModule.send[T]( - message, senderOption, senderFuture, remoteAddress.get, timeout, false, this, None, ActorType.ScalaActor) + message, senderOption, senderFuture, remoteAddress.get, timeout, false, this, None, actorType) if (future.isDefined) future.get else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString) } @@ -1431,8 +1418,8 @@ private[akka] case class RemoteActorRef private[akka] ( protected[akka] def mailbox: AnyRef = unsupported protected[akka] def mailbox_=(value: AnyRef):AnyRef = unsupported protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported - protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported - protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): 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 linkedActorsAsList: List[ActorRef] = unsupported protected[akka] def invoke(messageHandle: MessageInvocation): Unit = unsupported @@ -1539,10 +1526,9 @@ trait ScalaActorRef extends ActorRefShared { ref: ActorRef => * Is defined if the message was sent from another Actor, else None. */ def sender: Option[ActorRef] = { - // Five lines of map-performance-avoidance, could be just: currentMessage map { _.sender } val msg = currentMessage - if (msg.isEmpty) None - else msg.get.sender + if (msg eq null) None + else msg.sender } /** @@ -1550,10 +1536,9 @@ trait ScalaActorRef extends ActorRefShared { ref: ActorRef => * Is defined if the message was sent with sent with '!!' or '!!!', else None. */ def senderFuture(): Option[CompletableFuture[Any]] = { - // Five lines of map-performance-avoidance, could be just: currentMessage map { _.senderFuture } val msg = currentMessage - if (msg.isEmpty) None - else msg.get.senderFuture + if (msg eq null) None + else msg.senderFuture } diff --git a/akka-actor/src/main/scala/actor/ActorRegistry.scala b/akka-actor/src/main/scala/actor/ActorRegistry.scala index e8c38f2b76..51bbfd3477 100644 --- a/akka-actor/src/main/scala/actor/ActorRegistry.scala +++ b/akka-actor/src/main/scala/actor/ActorRegistry.scala @@ -11,6 +11,7 @@ import java.util.concurrent.{ConcurrentSkipListSet, ConcurrentHashMap} import java.util.{Set => JSet} import se.scalablesolutions.akka.util.ListenerManagement +import annotation.tailrec /** * Base trait for ActorRegistry events, allows listen to when an actor is added and removed from the ActorRegistry. @@ -35,10 +36,8 @@ case class ActorUnregistered(actor: ActorRef) extends ActorRegistryEvent */ object ActorRegistry extends ListenerManagement { private val actorsByUUID = new ConcurrentHashMap[String, ActorRef] - private val actorsById = new ConcurrentHashMap[String, JSet[ActorRef]] - - private val Naught = Array[ActorRef]() //Nil for Arrays - + private val actorsById = new Index[String,ActorRef] + /** * Returns all actors in the system. */ @@ -108,43 +107,25 @@ object ActorRegistry extends ListenerManagement { /** * Finds all actors that has a specific id. */ - def actorsFor(id: String): Array[ActorRef] = { - if (actorsById.containsKey(id)) { - actorsById.get(id).toArray(Naught) - } else Naught - } + def actorsFor(id: String): Array[ActorRef] = actorsById values id /** * Finds the actor that has a specific UUID. */ - def actorFor(uuid: String): Option[ActorRef] = { - if (actorsByUUID.containsKey(uuid)) Some(actorsByUUID.get(uuid)) - else None - } + def actorFor(uuid: String): Option[ActorRef] = Option(actorsByUUID get uuid) /** * Registers an actor in the ActorRegistry. */ def register(actor: ActorRef) = { // ID - val id = actor.id - if (id eq null) throw new IllegalActorStateException("Actor.id is null " + actor) - - val set = actorsById get id - if (set ne null) set add actor - else { - val newSet = new ConcurrentSkipListSet[ActorRef] - newSet add actor - val oldSet = actorsById.putIfAbsent(id,newSet) - // Parry for two simultaneous putIfAbsent(id,newSet) - if (oldSet ne null) oldSet add actor - } + actorsById.put(actor.id, actor) // UUID actorsByUUID.put(actor.uuid, actor) // notify listeners - foreachListener(_ ! ActorRegistered(actor)) + notifyListeners(ActorRegistered(actor)) } /** @@ -153,13 +134,10 @@ object ActorRegistry extends ListenerManagement { def unregister(actor: ActorRef) = { actorsByUUID remove actor.uuid - val set = actorsById get actor.id - if (set ne null) set remove actor - - //FIXME: safely remove set if empty, leaks memory + actorsById.remove(actor.id,actor) // notify listeners - foreachListener(_ ! ActorUnregistered(actor)) + notifyListeners(ActorUnregistered(actor)) } /** @@ -173,3 +151,85 @@ object ActorRegistry extends ListenerManagement { log.info("All actors have been shut down and unregistered from ActorRegistry") } } + +class Index[K <: AnyRef,V <: AnyRef : Manifest] { + import scala.collection.JavaConversions._ + + private val Naught = Array[V]() //Nil for Arrays + private val container = new ConcurrentHashMap[K, JSet[V]] + private val emptySet = new ConcurrentSkipListSet[V] + + def put(key: K, value: V) { + + //Returns whether it needs to be retried or not + def tryPut(set: JSet[V], v: V): Boolean = { + set.synchronized { + if (set.isEmpty) true //IF the set is empty then it has been removed, so signal retry + else { //Else add the value to the set and signal that retry is not needed + set add v + false + } + } + } + + @tailrec def syncPut(k: K, v: V): Boolean = { + var retry = false + val set = container get k + if (set ne null) retry = tryPut(set,v) + else { + val newSet = new ConcurrentSkipListSet[V] + newSet add v + + // Parry for two simultaneous putIfAbsent(id,newSet) + val oldSet = container.putIfAbsent(k,newSet) + if (oldSet ne null) + retry = tryPut(oldSet,v) + } + + if (retry) syncPut(k,v) + else true + } + + syncPut(key,value) + } + + def values(key: K) = { + val set: JSet[V] = container get key + if (set ne null) set toArray Naught + else Naught + } + + def foreach(key: K)(fun: (V) => Unit) { + val set = container get key + if (set ne null) + set foreach fun + } + + def findValue(key: K)(f: (V) => Boolean): Option[V] = { + val set = container get key + if (set ne null) + set.iterator.find(f) + else + None + } + + def foreach(fun: (K,V) => Unit) { + container.entrySet foreach { + (e) => e.getValue.foreach(fun(e.getKey,_)) + } + } + + def remove(key: K, value: V) { + val set = container get key + if (set ne null) { + set.synchronized { + if (set.remove(value)) { //If we can remove the value + if (set.isEmpty) //and the set becomes empty + container.remove(key,emptySet) //We try to remove the key if it's mapped to an empty set + } + } + } + } + + def clear = { foreach(remove _) } +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/actor/Agent.scala b/akka-actor/src/main/scala/actor/Agent.scala index df358cdfc4..e5b00d4f5e 100644 --- a/akka-actor/src/main/scala/actor/Agent.scala +++ b/akka-actor/src/main/scala/actor/Agent.scala @@ -6,7 +6,7 @@ package se.scalablesolutions.akka.actor import se.scalablesolutions.akka.stm.Ref import se.scalablesolutions.akka.AkkaException - +import se.scalablesolutions.akka.util.{ Function => JFunc, Procedure => JProc } import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.CountDownLatch @@ -136,6 +136,13 @@ sealed class Agent[T] private (initialValue: T) { */ final def apply(message: (T => T)): Unit = dispatcher ! Function(message) + /** + * Submits the provided function for execution against the internal agent's state. + * Java API + */ + final def apply(message: JFunc[T,T]): Unit = dispatcher ! Function((t: T) => message(t)) + + /** * Submits a new value to be set as the new agent's internal state. */ @@ -146,6 +153,12 @@ sealed class Agent[T] private (initialValue: T) { */ final def send(message: (T) => T): Unit = dispatcher ! Function(message) + /** + * Submits the provided function of type 'T => T' for execution against the internal agent's state. + * Java API + */ + final def send(message: JFunc[T,T]): Unit = dispatcher ! Function((t: T) => message(t)) + /** * Submits a new value to be set as the new agent's internal state. */ @@ -159,6 +172,15 @@ sealed class Agent[T] private (initialValue: T) { */ final def sendProc(f: (T) => Unit): Unit = dispatcher ! Procedure(f) + /** + * Asynchronously submits a procedure of type 'T => Unit' to read the internal state. + * The supplied procedure will be executed on the returned internal state value. A copy + * of the internal state will be used, depending on the underlying effective copyStrategy. + * Does not change the value of the agent (this). + * Java API + */ + final def sendProc(f: JProc[T]): Unit = dispatcher ! Procedure((t:T) => f(t)) + /** * Applies function with type 'T => B' to the agent's internal state and then returns a new agent with the result. * Does not change the value of the agent (this). @@ -177,6 +199,27 @@ sealed class Agent[T] private (initialValue: T) { */ final def foreach(f: (T) => Unit): Unit = f(get) + /** + * Applies function with type 'T => B' to the agent's internal state and then returns a new agent with the result. + * Does not change the value of the agent (this). + * Java API + */ + final def map[B](f: JFunc[T,B]): Agent[B] = Agent(f(get)) + + /** + * Applies function with type 'T => B' to the agent's internal state and then returns a new agent with the result. + * Does not change the value of the agent (this). + * Java API + */ + final def flatMap[B](f: JFunc[T,Agent[B]]): Agent[B] = Agent(f(get)()) + + /** + * Applies procedure with type T to the agent's internal state. + * Does not change the value of the agent (this). + * Java API + */ + final def foreach(f: JProc[T]): Unit = f(get) + /** * Closes the agents and makes it eligable for garbage collection. * diff --git a/akka-actor/src/main/scala/actor/BootableActorLoaderService.scala b/akka-actor/src/main/scala/actor/BootableActorLoaderService.scala index dfb8541396..278a75418e 100644 --- a/akka-actor/src/main/scala/actor/BootableActorLoaderService.scala +++ b/akka-actor/src/main/scala/actor/BootableActorLoaderService.scala @@ -55,8 +55,8 @@ trait BootableActorLoaderService extends Bootable with Logging { protected def createApplicationClassLoader : Option[ClassLoader] = { Some( if (HOME.isDefined) { - val CONFIG = HOME.get + "/config" - val DEPLOY = HOME.get + "/deploy" + val CONFIG = HOME.getOrElse(throwNoAkkaHomeException) + "/config" + val DEPLOY = HOME.getOrElse(throwNoAkkaHomeException) + "/deploy" val DEPLOY_DIR = new File(DEPLOY) if (!DEPLOY_DIR.exists) { log.error("Could not find a deploy directory at [%s]", DEPLOY) diff --git a/akka-actor/src/main/scala/actor/Supervisor.scala b/akka-actor/src/main/scala/actor/Supervisor.scala index 1af351a33d..f575cda299 100644 --- a/akka-actor/src/main/scala/actor/Supervisor.scala +++ b/akka-actor/src/main/scala/actor/Supervisor.scala @@ -187,7 +187,7 @@ final class SupervisorActor private[akka] ( trapExit = trapExceptions faultHandler = Some(handler) - override def shutdown(): Unit = shutdownLinkedActors + override def postStop(): Unit = shutdownLinkedActors def receive = { // FIXME add a way to respond to MaximumNumberOfRestartsWithinTimeRangeReached in declaratively configured Supervisor diff --git a/akka-actor/src/main/scala/config/Config.scala b/akka-actor/src/main/scala/config/Config.scala index 3b50d613c1..c9d9a4968b 100644 --- a/akka-actor/src/main/scala/config/Config.scala +++ b/akka-actor/src/main/scala/config/Config.scala @@ -64,9 +64,12 @@ object Config { Configgy.config } else if (HOME.isDefined) { try { - val configFile = HOME.get + "/config/akka.conf" + val configFile = HOME.getOrElse(throwNoAkkaHomeException) + "/config/akka.conf" Configgy.configure(configFile) - ConfigLogger.log.info("AKKA_HOME is defined as [%s], config loaded from [%s].", HOME.get, configFile) + ConfigLogger.log.info( + "AKKA_HOME is defined as [%s], config loaded from [%s].", + HOME.getOrElse(throwNoAkkaHomeException), + configFile) } catch { case e: ParseException => throw new ConfigurationException( "AKKA_HOME is defined as [" + HOME.get + "] " + @@ -86,7 +89,7 @@ object Config { CConfig.fromString("") // default empty config } } - + val CONFIG_VERSION = config.getString("akka.version", VERSION) if (VERSION != CONFIG_VERSION) throw new ConfigurationException( "Akka JAR version [" + VERSION + "] is different than the provided config ('akka.conf') version [" + CONFIG_VERSION + "]") @@ -95,4 +98,9 @@ object Config { val startTime = System.currentTimeMillis def uptime = (System.currentTimeMillis - startTime) / 1000 + + def throwNoAkkaHomeException = throw new ConfigurationException( + "Akka home is not defined. Either:" + + "\n\t1. Define 'AKKA_HOME' environment variable pointing to the root of the Akka distribution." + + "\n\t2. Add the '-Dakka.home=...' option pointing to the root of the Akka distribution.") } diff --git a/akka-actor/src/main/scala/config/SupervisionConfig.scala b/akka-actor/src/main/scala/config/SupervisionConfig.scala index 2f25f4ed33..d85001b5ca 100644 --- a/akka-actor/src/main/scala/config/SupervisionConfig.scala +++ b/akka-actor/src/main/scala/config/SupervisionConfig.scala @@ -8,8 +8,19 @@ import se.scalablesolutions.akka.actor.{ActorRef} import se.scalablesolutions.akka.dispatch.MessageDispatcher sealed abstract class FaultHandlingStrategy -case class AllForOneStrategy(maxNrOfRetries: Int, withinTimeRange: Int) extends FaultHandlingStrategy -case class OneForOneStrategy(maxNrOfRetries: Int, withinTimeRange: Int) extends FaultHandlingStrategy +object AllForOneStrategy { + def apply(maxNrOfRetries: Int, withinTimeRange: Int): AllForOneStrategy = + AllForOneStrategy(if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), + if (withinTimeRange < 0) None else Some(withinTimeRange)) +} +case class AllForOneStrategy(maxNrOfRetries: Option[Int] = None, withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy + +object OneForOneStrategy { + def apply(maxNrOfRetries: Int, withinTimeRange: Int): OneForOneStrategy = + this(if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), + if (withinTimeRange < 0) None else Some(withinTimeRange)) +} +case class OneForOneStrategy(maxNrOfRetries: Option[Int] = None, withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy /** * Configuration classes - not to be used as messages. diff --git a/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala b/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala index 787793dc5f..6608f6075b 100644 --- a/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala +++ b/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala @@ -11,6 +11,7 @@ import se.scalablesolutions.akka.actor.{Actor, ActorRef} import se.scalablesolutions.akka.actor.Actor._ import se.scalablesolutions.akka.dispatch.CompletableFuture import se.scalablesolutions.akka.AkkaException +import se.scalablesolutions.akka.util.{ Function, SideEffect } /** * Implements Oz-style dataflow (single assignment) variables. @@ -27,9 +28,22 @@ object DataFlow { */ def thread(body: => Unit): Unit = spawn(body) + /** Executes the supplied SideEffect in another thread + * JavaAPI + */ + def thread(body: SideEffect): Unit = spawn(body.apply) + + /** Executes the supplied function in another thread + */ def thread[A <: AnyRef, R <: AnyRef](body: A => R) = actorOf(new ReactiveEventBasedThread(body)).start + /** Executes the supplied Function in another thread + * JavaAPI + */ + def thread[A <: AnyRef, R <: AnyRef](body: Function[A,R]) = + actorOf(new ReactiveEventBasedThread(body.apply)).start + private class ReactiveEventBasedThread[A <: AnyRef, T <: AnyRef](body: A => T) extends Actor { def receive = { @@ -91,6 +105,11 @@ object DataFlow { "Attempt to change data flow variable (from [" + this.value.get + "] to [" + ref() + "])") } + /** Sets the value of this variable (if unset) with the value of the supplied variable + * JavaAPI + */ + def set(ref: DataFlowVariable[T]) { this << ref } + /** Sets the value of this variable (if unset) */ def <<(value: T) { @@ -99,6 +118,16 @@ object DataFlow { "Attempt to change data flow variable (from [" + this.value.get + "] to [" + value + "])") } + /** Sets the value of this variable (if unset) with the value of the supplied variable + * JavaAPI + */ + def set(value: T) { this << value } + + /** Retrieves the value of variable + * throws a DataFlowVariableException if it times out + */ + def get(): T = this() + /** Retrieves the value of variable * throws a DataFlowVariableException if it times out */ diff --git a/akka-actor/src/main/scala/dispatch/AbstractReactorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/AbstractReactorBasedEventDrivenDispatcher.scala deleted file mode 100644 index 6bacec73be..0000000000 --- a/akka-actor/src/main/scala/dispatch/AbstractReactorBasedEventDrivenDispatcher.scala +++ /dev/null @@ -1,63 +0,0 @@ -/** - * Copyright (C) 2009-2010 Scalable Solutions AB - */ - -package se.scalablesolutions.akka.dispatch - -import java.util.{LinkedList, Queue, List} -import java.util.HashMap - -import se.scalablesolutions.akka.actor.{Actor, ActorRef} - -abstract class AbstractReactorBasedEventDrivenDispatcher(val name: String) extends MessageDispatcher { - @volatile protected var active: Boolean = false - protected val queue = new ReactiveMessageQueue(name) - protected val messageInvokers = new HashMap[ActorRef, ActorRef] - protected var selectorThread: Thread = _ - protected val guard = new Object - - def dispatch(invocation: MessageInvocation) = queue.append(invocation) - - override def register(actorRef: ActorRef) = synchronized { - messageInvokers.put(actorRef, actorRef) - super.register(actorRef) - } - - override def unregister(actorRef: ActorRef) = synchronized { - messageInvokers.remove(actorRef) - super.unregister(actorRef) - } - - def shutdown = if (active) { - log.debug("Shutting down %s", toString) - active = false - selectorThread.interrupt - doShutdown - } - - /** - * Subclass callback. Override if additional shutdown behavior is needed. - */ - protected def doShutdown = {} -} - -class ReactiveMessageQueue(name: String) extends MessageQueue { - private[akka] val queue: Queue[MessageInvocation] = new LinkedList[MessageInvocation] - @volatile private var interrupted = false - - def append(handle: MessageInvocation) = queue.synchronized { - queue.offer(handle) - queue.notifyAll - } - - def read(destination: List[MessageInvocation]) = queue.synchronized { - while (queue.isEmpty && !interrupted) queue.wait - if (!interrupted) while (!queue.isEmpty) destination.add(queue.remove) - else interrupted = false - } - - def interrupt = queue.synchronized { - interrupted = true - queue.notifyAll - } -} diff --git a/akka-actor/src/main/scala/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/dispatch/Dispatchers.scala index 803fd700cc..7e7904ec29 100644 --- a/akka-actor/src/main/scala/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/dispatch/Dispatchers.scala @@ -5,12 +5,11 @@ package se.scalablesolutions.akka.dispatch import se.scalablesolutions.akka.actor.{Actor, ActorRef} -import se.scalablesolutions.akka.util.Logging import se.scalablesolutions.akka.config.Config.config import net.lag.configgy.ConfigMap -import se.scalablesolutions.akka.util.UUID import java.util.concurrent.ThreadPoolExecutor.{AbortPolicy, CallerRunsPolicy, DiscardOldestPolicy, DiscardPolicy} import java.util.concurrent.TimeUnit +import se.scalablesolutions.akka.util.{Duration, Logging, UUID} /** * Scala API. Dispatcher factory. @@ -45,8 +44,14 @@ import java.util.concurrent.TimeUnit * @author Jonas Bonér */ object Dispatchers extends Logging { - val THROUGHPUT = config.getInt("akka.actor.throughput", 5) - val MAILBOX_CAPACITY = config.getInt("akka.actor.default-dispatcher.mailbox-capacity", 1000) + val THROUGHPUT = config.getInt("akka.actor.throughput", 5) + val THROUGHPUT_DEADLINE_MS = config.getInt("akka.actor.throughput-deadline-ms",-1) + val MAILBOX_CAPACITY = config.getInt("akka.actor.default-dispatcher.mailbox-capacity", -1) + val MAILBOX_CONFIG = MailboxConfig( + capacity = Dispatchers.MAILBOX_CAPACITY, + pushTimeOut = config.getInt("akka.actor.default-dispatcher.mailbox-push-timeout-ms").map(Duration(_,TimeUnit.MILLISECONDS)), + blockingDequeue = false + ) lazy val defaultGlobalDispatcher = { config.getConfigMap("akka.actor.default-dispatcher").flatMap(from).getOrElse(globalExecutorBasedEventDrivenDispatcher) @@ -54,17 +59,13 @@ object Dispatchers extends Logging { object globalHawtDispatcher extends HawtDispatcher - object globalExecutorBasedEventDrivenDispatcher extends ExecutorBasedEventDrivenDispatcher("global") { + object globalExecutorBasedEventDrivenDispatcher extends ExecutorBasedEventDrivenDispatcher("global",THROUGHPUT,THROUGHPUT_DEADLINE_MS,MAILBOX_CONFIG) { override def register(actor: ActorRef) = { if (isShutdown) init super.register(actor) } } - object globalReactorBasedSingleThreadEventDrivenDispatcher extends ReactorBasedSingleThreadEventDrivenDispatcher("global") - - object globalReactorBasedThreadPoolEventDrivenDispatcher extends ReactorBasedThreadPoolEventDrivenDispatcher("global") - /** * Creates an event-driven dispatcher based on the excellent HawtDispatch library. *

@@ -95,7 +96,7 @@ object Dispatchers extends Logging { *

* E.g. each actor consumes its own thread. */ - def newThreadBasedDispatcher(actor: ActorRef, mailboxCapacity: Int, pushTimeout: Long, pushTimeUnit: TimeUnit) = new ThreadBasedDispatcher(actor, mailboxCapacity, pushTimeout, pushTimeUnit) + def newThreadBasedDispatcher(actor: ActorRef, mailboxCapacity: Int, pushTimeOut: Duration) = new ThreadBasedDispatcher(actor, MailboxConfig(mailboxCapacity,Option(pushTimeOut),true)) /** * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool. @@ -116,7 +117,15 @@ object Dispatchers extends Logging { *

* Has a fluent builder interface for configuring its semantics. */ - def newExecutorBasedEventDrivenDispatcher(name: String, throughput: Int, mailboxCapacity: Int) = new ExecutorBasedEventDrivenDispatcher(name, throughput, mailboxCapacity) + def newExecutorBasedEventDrivenDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxCapacity: Int) = new ExecutorBasedEventDrivenDispatcher(name, throughput, throughputDeadlineMs, mailboxCapacity) + + /** + * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool. + *

+ * Has a fluent builder interface for configuring its semantics. + */ + def newExecutorBasedEventDrivenDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxCapacity: Int, pushTimeOut: Duration) = new ExecutorBasedEventDrivenDispatcher(name, throughput, throughputDeadlineMs, MailboxConfig(mailboxCapacity,Some(pushTimeOut),false)) + /** * Creates a executor-based event-driven dispatcher with work stealing (TODO: better doc) serving multiple (millions) of actors through a thread pool. @@ -132,18 +141,6 @@ object Dispatchers extends Logging { */ def newExecutorBasedEventDrivenWorkStealingDispatcher(name: String, mailboxCapacity: Int) = new ExecutorBasedEventDrivenWorkStealingDispatcher(name, mailboxCapacity) - /** - * Creates a reactor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool. - *

- * Has a fluent builder interface for configuring its semantics. - */ - def newReactorBasedThreadPoolEventDrivenDispatcher(name: String) = new ReactorBasedThreadPoolEventDrivenDispatcher(name) - - /** - * Creates a reactor-based event-driven dispatcher serving multiple (millions) of actors through a single thread. - */ - def newReactorBasedSingleThreadEventDrivenDispatcher(name: String) = new ReactorBasedSingleThreadEventDrivenDispatcher(name) - /** * Utility function that tries to load the specified dispatcher config from the akka.conf * or else use the supplied default dispatcher @@ -156,9 +153,8 @@ object Dispatchers extends Logging { * * default-dispatcher { * type = "GlobalExecutorBasedEventDriven" # Must be one of the following, all "Global*" are non-configurable - * # ReactorBasedSingleThreadEventDriven, (ExecutorBasedEventDrivenWorkStealing), ExecutorBasedEventDriven, - * # ReactorBasedThreadPoolEventDriven, Hawt, GlobalReactorBasedSingleThreadEventDriven, - * # GlobalReactorBasedThreadPoolEventDriven, GlobalExecutorBasedEventDriven, GlobalHawt + * # (ExecutorBasedEventDrivenWorkStealing), ExecutorBasedEventDriven, + * # Hawt, GlobalExecutorBasedEventDriven, GlobalHawt * keep-alive-ms = 60000 # Keep alive time for threads * core-pool-size-factor = 1.0 # No of core threads ... ceil(available processors * factor) * max-pool-size-factor = 4.0 # Max no of threads ... ceil(available processors * factor) @@ -177,23 +173,8 @@ object Dispatchers extends Logging { def from(cfg: ConfigMap): Option[MessageDispatcher] = { lazy val name = cfg.getString("name", UUID.newUuid.toString) - val dispatcher: Option[MessageDispatcher] = cfg.getString("type") map { - case "ReactorBasedSingleThreadEventDriven" => newReactorBasedSingleThreadEventDrivenDispatcher(name) - case "ExecutorBasedEventDrivenWorkStealing" => newExecutorBasedEventDrivenWorkStealingDispatcher(name) - case "ExecutorBasedEventDriven" => newExecutorBasedEventDrivenDispatcher(name,cfg.getInt("throughput",THROUGHPUT)) - case "ReactorBasedThreadPoolEventDriven" => newReactorBasedThreadPoolEventDrivenDispatcher(name) - case "Hawt" => newHawtDispatcher(cfg.getBool("aggregate").getOrElse(true)) - case "GlobalReactorBasedSingleThreadEventDriven" => globalReactorBasedSingleThreadEventDrivenDispatcher - case "GlobalReactorBasedThreadPoolEventDriven" => globalReactorBasedThreadPoolEventDrivenDispatcher - case "GlobalExecutorBasedEventDriven" => globalExecutorBasedEventDrivenDispatcher - case "GlobalHawt" => globalHawtDispatcher - - case unknown => throw new IllegalArgumentException("Unknown dispatcher type [%s]" format unknown) - } - - dispatcher foreach { - case d: ThreadPoolBuilder => d.configureIfPossible( builder => { - + def threadPoolConfig(b: ThreadPoolBuilder) { + b.configureIfPossible( builder => { cfg.getInt("keep-alive-ms").foreach(builder.setKeepAliveTimeInMillis(_)) cfg.getDouble("core-pool-size-factor").foreach(builder.setCorePoolSizeFromFactor(_)) cfg.getDouble("max-pool-size-factor").foreach(builder.setMaxPoolSizeFromFactor(_)) @@ -209,7 +190,37 @@ object Dispatchers extends Logging { case x => throw new IllegalArgumentException("[%s] is not a valid rejectionPolicy!" format x) }).foreach(builder.setRejectionPolicy(_)) }) - case _ => + } + + lazy val mailboxBounds: MailboxConfig = { + val capacity = cfg.getInt("mailbox-capacity",Dispatchers.MAILBOX_CAPACITY) + val timeout = cfg.getInt("mailbox-push-timeout-ms").map(Duration(_,TimeUnit.MILLISECONDS)) + MailboxConfig(capacity,timeout,false) + } + + val dispatcher: Option[MessageDispatcher] = cfg.getString("type") map { + case "ExecutorBasedEventDrivenWorkStealing" => + new ExecutorBasedEventDrivenWorkStealingDispatcher(name,MAILBOX_CAPACITY,threadPoolConfig) + + case "ExecutorBasedEventDriven" => + new ExecutorBasedEventDrivenDispatcher( + name, + cfg.getInt("throughput",THROUGHPUT), + cfg.getInt("throughput-deadline-ms",THROUGHPUT_DEADLINE_MS), + mailboxBounds, + threadPoolConfig) + + case "Hawt" => + new HawtDispatcher(cfg.getBool("aggregate").getOrElse(true)) + + case "GlobalExecutorBasedEventDriven" => + globalExecutorBasedEventDrivenDispatcher + + case "GlobalHawt" => + globalHawtDispatcher + + case unknown => + throw new IllegalArgumentException("Unknown dispatcher type [%s]" format unknown) } dispatcher diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala index 1f8a6bfe9c..63ce310848 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala @@ -7,7 +7,7 @@ package se.scalablesolutions.akka.dispatch import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException} import java.util.Queue -import java.util.concurrent.{ConcurrentLinkedQueue, LinkedBlockingQueue} +import java.util.concurrent.{RejectedExecutionException, ConcurrentLinkedQueue, LinkedBlockingQueue} /** * Default settings are: @@ -64,89 +64,104 @@ import java.util.concurrent.{ConcurrentLinkedQueue, LinkedBlockingQueue} */ class ExecutorBasedEventDrivenDispatcher( _name: String, - throughput: Int = Dispatchers.THROUGHPUT, - capacity: Int = Dispatchers.MAILBOX_CAPACITY) extends MessageDispatcher with ThreadPoolBuilder { + val throughput: Int = Dispatchers.THROUGHPUT, + val throughputDeadlineMs: Int = Dispatchers.THROUGHPUT_DEADLINE_MS, + mailboxConfig: MailboxConfig = Dispatchers.MAILBOX_CONFIG, + config: (ThreadPoolBuilder) => Unit = _ => ()) extends MessageDispatcher with ThreadPoolBuilder { - def this(_name: String, throughput: Int) = this(_name, throughput, Dispatchers.MAILBOX_CAPACITY) // Needed for Java API usage - def this(_name: String) = this(_name, Dispatchers.THROUGHPUT) // Needed for Java API usage + def this(_name: String, throughput: Int, throughputDeadlineMs: Int, capacity: Int) = this(_name,throughput,throughputDeadlineMs,MailboxConfig(capacity,None,false)) + def this(_name: String, throughput: Int) = this(_name, throughput, Dispatchers.THROUGHPUT_DEADLINE_MS, Dispatchers.MAILBOX_CAPACITY) // Needed for Java API usage + def this(_name: String) = this(_name,Dispatchers.THROUGHPUT, Dispatchers.THROUGHPUT_DEADLINE_MS,Dispatchers.MAILBOX_CAPACITY) // Needed for Java API usage - mailboxCapacity = capacity + //FIXME remove this from ThreadPoolBuilder + mailboxCapacity = mailboxConfig.capacity @volatile private var active: Boolean = false val name = "akka:event-driven:dispatcher:" + _name init + /** + * This is the behavior of an ExecutorBasedEventDrivenDispatchers mailbox + */ + trait ExecutableMailbox extends Runnable { self: MessageQueue => + final def run = { + + val reschedule = try { + processMailbox() + } finally { + dispatcherLock.unlock() + } + + if (reschedule || !self.isEmpty) + registerForExecution(self) + } + + /** + * Process the messages in the mailbox + * + * @return true if the processing finished before the mailbox was empty, due to the throughput constraint + */ + final def processMailbox(): Boolean = { + var nextMessage = self.dequeue + if (nextMessage ne null) { + val throttle = throughput > 0 + var processedMessages = 0 + val isDeadlineEnabled = throttle && throughputDeadlineMs > 0 + val started = if (isDeadlineEnabled) System.currentTimeMillis else 0 + + do { + nextMessage.invoke + + if(throttle) { //Will be elided when false + processedMessages += 1 + if ((processedMessages >= throughput) + || (isDeadlineEnabled && (System.currentTimeMillis - started) >= throughputDeadlineMs)) //If we're throttled, break out + return !self.isEmpty + } + nextMessage = self.dequeue + } + while (nextMessage ne null) + } + + false + } + } + def dispatch(invocation: MessageInvocation) = { - getMailbox(invocation.receiver).add(invocation) - dispatch(invocation.receiver) + val mbox = getMailbox(invocation.receiver) + mbox enqueue invocation + registerForExecution(mbox) + } + + protected def registerForExecution(mailbox: MessageQueue with ExecutableMailbox): Unit = if (active) { + if (mailbox.dispatcherLock.tryLock()) { + try { + executor execute mailbox + } catch { + case e: RejectedExecutionException => + mailbox.dispatcherLock.unlock() + throw e + } + } + } else { + log.warning("%s is shut down,\n\tignoring the rest of the messages in the mailbox of\n\t%s", toString, mailbox) } /** * @return the mailbox associated with the actor */ - private def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[Queue[MessageInvocation]] + private def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[MessageQueue with ExecutableMailbox] override def mailboxSize(actorRef: ActorRef) = getMailbox(actorRef).size - override def register(actorRef: ActorRef) = { - if (actorRef.mailbox eq null ) { - if (mailboxCapacity <= 0) actorRef.mailbox = new ConcurrentLinkedQueue[MessageInvocation] - else actorRef.mailbox = new LinkedBlockingQueue[MessageInvocation](mailboxCapacity) - } - super.register(actorRef) + override def createMailbox(actorRef: ActorRef): AnyRef = { + if (mailboxCapacity > 0) + new DefaultBoundedMessageQueue(mailboxCapacity,mailboxConfig.pushTimeOut,blockDequeue = false) with ExecutableMailbox + else + new DefaultUnboundedMessageQueue(blockDequeue = false) with ExecutableMailbox } - def dispatch(receiver: ActorRef): Unit = if (active) { - - executor.execute(new Runnable() { - def run = { - var lockAcquiredOnce = false - var finishedBeforeMailboxEmpty = false - val lock = receiver.dispatcherLock - val mailbox = getMailbox(receiver) - // this do-while loop is required to prevent missing new messages between the end of the inner while - // loop and releasing the lock - do { - if (lock.tryLock) { - // Only dispatch if we got the lock. Otherwise another thread is already dispatching. - lockAcquiredOnce = true - try { - finishedBeforeMailboxEmpty = processMailbox(receiver) - } finally { - lock.unlock - if (finishedBeforeMailboxEmpty) dispatch(receiver) - } - } - } while ((lockAcquiredOnce && !finishedBeforeMailboxEmpty && !mailbox.isEmpty)) - } - }) - } else { - log.warning("%s is shut down,\n\tignoring the rest of the messages in the mailbox of\n\t%s", toString, receiver) - } - - - /** - * Process the messages in the mailbox of the given actor. - * - * @return true if the processing finished before the mailbox was empty, due to the throughput constraint - */ - def processMailbox(receiver: ActorRef): Boolean = { - var processedMessages = 0 - val mailbox = getMailbox(receiver) - var messageInvocation = mailbox.poll - while (messageInvocation != null) { - messageInvocation.invoke - processedMessages += 1 - // check if we simply continue with other messages, or reached the throughput limit - if (throughput <= 0 || processedMessages < throughput) messageInvocation = mailbox.poll - else { - messageInvocation = null - return !mailbox.isEmpty - } - } - false - } def start = if (!active) { log.debug("Starting up %s\n\twith throughput [%d]", toString, throughput) @@ -160,11 +175,17 @@ class ExecutorBasedEventDrivenDispatcher( uuids.clear } - def ensureNotActive(): Unit = if (active) throw new IllegalActorStateException( + def ensureNotActive(): Unit = if (active) { + throw new IllegalActorStateException( "Can't build a new thread pool for a dispatcher that is already up and running") + } override def toString = "ExecutorBasedEventDrivenDispatcher[" + name + "]" // FIXME: should we have an unbounded queue and not bounded as default ???? - private[akka] def init = withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity.buildThreadPool + private[akka] def init = { + withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity + config(this) + buildThreadPool + } } diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala index 4e5d626aed..10afb1bfb6 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala @@ -31,7 +31,11 @@ import se.scalablesolutions.akka.actor.{Actor, ActorRef, IllegalActorStateExcept */ class ExecutorBasedEventDrivenWorkStealingDispatcher( _name: String, - capacity: Int = Dispatchers.MAILBOX_CAPACITY) extends MessageDispatcher with ThreadPoolBuilder { + capacity: Int = Dispatchers.MAILBOX_CAPACITY, + config: (ThreadPoolBuilder) => Unit = _ => ()) extends MessageDispatcher with ThreadPoolBuilder { + + def this(_name: String, capacity: Int) = this(_name,capacity, _ => ()) + mailboxCapacity = capacity @volatile private var active: Boolean = false @@ -52,21 +56,14 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( /** * @return the mailbox associated with the actor */ - private def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[Deque[MessageInvocation]] + private def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[Deque[MessageInvocation] with MessageQueue with Runnable] override def mailboxSize(actorRef: ActorRef) = getMailbox(actorRef).size def dispatch(invocation: MessageInvocation) = if (active) { - getMailbox(invocation.receiver).add(invocation) - executor.execute(new Runnable() { - def run = { - if (!tryProcessMailbox(invocation.receiver)) { - // we are not able to process our mailbox (another thread is busy with it), so lets donate some of our mailbox - // to another actor and then process his mailbox in stead. - findThief(invocation.receiver).foreach( tryDonateAndProcessMessages(invocation.receiver,_) ) - } - } - }) + val mbox = getMailbox(invocation.receiver) + mbox enqueue invocation + executor execute mbox } else throw new IllegalActorStateException("Can't submit invocations to dispatcher since it's not started") /** @@ -75,22 +72,21 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( * * @return true if the mailbox was processed, false otherwise */ - private def tryProcessMailbox(receiver: ActorRef): Boolean = { + private def tryProcessMailbox(mailbox: MessageQueue): Boolean = { var lockAcquiredOnce = false - val lock = receiver.dispatcherLock // this do-wile loop is required to prevent missing new messages between the end of processing // the mailbox and releasing the lock do { - if (lock.tryLock) { + if (mailbox.dispatcherLock.tryLock) { lockAcquiredOnce = true try { - processMailbox(receiver) + processMailbox(mailbox) } finally { - lock.unlock + mailbox.dispatcherLock.unlock } } - } while ((lockAcquiredOnce && !getMailbox(receiver).isEmpty)) + } while ((lockAcquiredOnce && !mailbox.isEmpty)) lockAcquiredOnce } @@ -98,12 +94,11 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( /** * Process the messages in the mailbox of the given actor. */ - private def processMailbox(receiver: ActorRef) = { - val mailbox = getMailbox(receiver) - var messageInvocation = mailbox.poll - while (messageInvocation != null) { + private def processMailbox(mailbox: MessageQueue) = { + var messageInvocation = mailbox.dequeue + while (messageInvocation ne null) { messageInvocation.invoke - messageInvocation = mailbox.poll + messageInvocation = mailbox.dequeue } } @@ -141,11 +136,12 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( * the thiefs dispatching lock, because in that case another thread is already processing the thiefs mailbox. */ private def tryDonateAndProcessMessages(receiver: ActorRef, thief: ActorRef) = { - if (thief.dispatcherLock.tryLock) { + val mailbox = getMailbox(thief) + if (mailbox.dispatcherLock.tryLock) { try { - while(donateMessage(receiver, thief)) processMailbox(thief) + while(donateMessage(receiver, thief)) processMailbox(mailbox) } finally { - thief.dispatcherLock.unlock + mailbox.dispatcherLock.unlock } } } @@ -180,21 +176,51 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( override def toString = "ExecutorBasedEventDrivenWorkStealingDispatcher[" + name + "]" - private[akka] def init = withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity.buildThreadPool + private[akka] def init = { + withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity + config(this) + buildThreadPool + } + + protected override def createMailbox(actorRef: ActorRef): AnyRef = { + if (mailboxCapacity <= 0) { + new ConcurrentLinkedDeque[MessageInvocation] with MessageQueue with Runnable { + def enqueue(handle: MessageInvocation): Unit = this.add(handle) + def dequeue: MessageInvocation = this.poll() + + def run = { + if (!tryProcessMailbox(this)) { + // we are not able to process our mailbox (another thread is busy with it), so lets donate some of our mailbox + // to another actor and then process his mailbox in stead. + findThief(actorRef).foreach( tryDonateAndProcessMessages(actorRef,_) ) + } + } + } + } + else { + new LinkedBlockingDeque[MessageInvocation](mailboxCapacity) with MessageQueue with Runnable { + def enqueue(handle: MessageInvocation): Unit = this.add(handle) + def dequeue: MessageInvocation = this.poll() + + def run = { + if (!tryProcessMailbox(this)) { + // we are not able to process our mailbox (another thread is busy with it), so lets donate some of our mailbox + // to another actor and then process his mailbox in stead. + findThief(actorRef).foreach( tryDonateAndProcessMessages(actorRef,_) ) + } + } + } + } + } override def register(actorRef: ActorRef) = { verifyActorsAreOfSameType(actorRef) - // The actor will need a ConcurrentLinkedDeque based mailbox - if (actorRef.mailbox == null) { - if (mailboxCapacity <= 0) actorRef.mailbox = new ConcurrentLinkedDeque[MessageInvocation] - else actorRef.mailbox = new LinkedBlockingDeque[MessageInvocation](mailboxCapacity) - } - pooledActors.add(actorRef) + pooledActors add actorRef super.register(actorRef) } override def unregister(actorRef: ActorRef) = { - pooledActors.remove(actorRef) + pooledActors remove actorRef super.unregister(actorRef) } diff --git a/akka-actor/src/main/scala/dispatch/Future.scala b/akka-actor/src/main/scala/dispatch/Future.scala index 17c63bcd57..0a3cd48aa5 100644 --- a/akka-actor/src/main/scala/dispatch/Future.scala +++ b/akka-actor/src/main/scala/dispatch/Future.scala @@ -160,6 +160,7 @@ class DefaultCompletableFuture[T](timeout: Long) extends CompletableFuture[T] { if (!_completed) { _completed = true _result = Some(result) + onComplete(result) } } finally { _signal.signalAll @@ -171,6 +172,7 @@ class DefaultCompletableFuture[T](timeout: Long) extends CompletableFuture[T] { if (!_completed) { _completed = true _exception = Some(exception) + onCompleteException(exception) } } finally { _signal.signalAll @@ -178,4 +180,6 @@ class DefaultCompletableFuture[T](timeout: Long) extends CompletableFuture[T] { } private def currentTimeInNanos: Long = TIME_UNIT.toNanos(System.currentTimeMillis) + protected def onComplete(result: T) {} + protected def onCompleteException(exception: Throwable) {} } diff --git a/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala b/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala index e0ddf05d26..cf3f71295c 100644 --- a/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala @@ -185,16 +185,10 @@ class HawtDispatcher(val aggregate:Boolean=true, val parent:DispatchQueue=global // TODO: figure out if this can be optional in akka override def mailboxSize(actorRef: ActorRef) = 0 - override def register(actorRef: ActorRef) = { - if( actorRef.mailbox == null ) { - val queue = parent.createSerialQueue(actorRef.toString) - if( aggregate ) { - actorRef.mailbox = new AggregatingHawtDispatcherMailbox(queue) - } else { - actorRef.mailbox = new HawtDispatcherMailbox(queue) - } - } - super.register(actorRef) + override def createMailbox(actorRef: ActorRef): AnyRef = { + val queue = parent.createSerialQueue(actorRef.toString) + if (aggregate) new AggregatingHawtDispatcherMailbox(queue) + else new HawtDispatcherMailbox(queue) } override def toString = "HawtDispatchEventDrivenDispatcher" diff --git a/akka-actor/src/main/scala/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/dispatch/MessageHandling.scala index 395c572f0e..25a02f2603 100644 --- a/akka-actor/src/main/scala/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/dispatch/MessageHandling.scala @@ -4,14 +4,14 @@ package se.scalablesolutions.akka.dispatch -import java.util.List - -import se.scalablesolutions.akka.util.{HashCode, Logging} import se.scalablesolutions.akka.actor.{Actor, ActorRef, ActorInitializationException} import org.multiverse.commitbarriers.CountDownCommitBarrier import se.scalablesolutions.akka.AkkaException -import java.util.concurrent.{ConcurrentSkipListSet} +import java.util.{Queue, List} +import java.util.concurrent._ +import concurrent.forkjoin.LinkedTransferQueue +import se.scalablesolutions.akka.util.{SimpleLock, Duration, HashCode, Logging} /** * @author Jonas Bonér @@ -30,8 +30,6 @@ final class MessageInvocation(val receiver: ActorRef, "Don't call 'self ! message' in the Actor's constructor (e.g. body of the class).") } - def send = receiver.dispatcher.dispatch(this) - override def hashCode(): Int = synchronized { var result = HashCode.SEED result = HashCode.hash(result, receiver.actor) @@ -63,33 +61,94 @@ class MessageQueueAppendFailedException(message: String) extends AkkaException(m * @author Jonas Bonér */ trait MessageQueue { - def append(handle: MessageInvocation) + val dispatcherLock = new SimpleLock + def enqueue(handle: MessageInvocation) + def dequeue(): MessageInvocation + def size: Int + def isEmpty: Boolean +} + +/* Tells the dispatcher that it should create a bounded mailbox with the specified push timeout + * (If capacity > 0) + */ +case class MailboxConfig(capacity: Int, pushTimeOut: Option[Duration], blockingDequeue: Boolean) { + + /** + * Creates a MessageQueue (Mailbox) with the specified properties + * bounds = whether the mailbox should be bounded (< 0 means unbounded) + * pushTime = only used if bounded, indicates if and how long an enqueue should block + * blockDequeue = whether dequeues should block or not + * + * The bounds + pushTime generates a MessageQueueAppendFailedException if enqueue times out + */ + def newMailbox(bounds: Int = capacity, + pushTime: Option[Duration] = pushTimeOut, + blockDequeue: Boolean = blockingDequeue) : MessageQueue = + if (capacity > 0) new DefaultBoundedMessageQueue(bounds,pushTime,blockDequeue) + else new DefaultUnboundedMessageQueue(blockDequeue) +} + +class DefaultUnboundedMessageQueue(blockDequeue: Boolean) extends LinkedBlockingQueue[MessageInvocation] with MessageQueue { + final def enqueue(handle: MessageInvocation) { + this add handle + } + + final def dequeue(): MessageInvocation = + if (blockDequeue) this.take() + else this.poll() +} + +class DefaultBoundedMessageQueue(capacity: Int, pushTimeOut: Option[Duration], blockDequeue: Boolean) extends LinkedBlockingQueue[MessageInvocation](capacity) with MessageQueue { + final def enqueue(handle: MessageInvocation) { + if (pushTimeOut.isDefined) { + if(!this.offer(handle,pushTimeOut.get.length,pushTimeOut.get.unit)) + throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + toString) + } + else { + this put handle + } + } + + final def dequeue(): MessageInvocation = + if (blockDequeue) this.take() + else this.poll() + } /** - * @author Jonas Bonér + * @author Jonas Bonér */ trait MessageDispatcher extends Logging { protected val uuids = new ConcurrentSkipListSet[String] + def dispatch(invocation: MessageInvocation) + def start + def shutdown - def register(actorRef: ActorRef) = uuids add actorRef.uuid + + def register(actorRef: ActorRef) { + if(actorRef.mailbox eq null) + actorRef.mailbox = createMailbox(actorRef) + uuids add actorRef.uuid + } def unregister(actorRef: ActorRef) = { uuids remove actorRef.uuid + actorRef.mailbox = null if (canBeShutDown) shutdown // shut down in the dispatcher's references is zero } + def canBeShutDown: Boolean = uuids.isEmpty - def isShutdown: Boolean - def mailboxSize(actorRef: ActorRef):Int = 0 -} -/** - * @author Jonas Bonér - */ -trait MessageDemultiplexer { - def select - def wakeUp - def acquireSelectedInvocations: List[MessageInvocation] - def releaseSelectedInvocations -} + def isShutdown: Boolean + + /** + * Returns the size of the mailbox for the specified actor + */ + def mailboxSize(actorRef: ActorRef):Int + + /** + * Creates and returns a mailbox for the given actor + */ + protected def createMailbox(actorRef: ActorRef): AnyRef = null +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/dispatch/Queues.scala b/akka-actor/src/main/scala/dispatch/Queues.scala deleted file mode 100644 index 5b5aa6683e..0000000000 --- a/akka-actor/src/main/scala/dispatch/Queues.scala +++ /dev/null @@ -1,148 +0,0 @@ -/** - * Copyright (C) 2009-2010 Scalable Solutions AB - */ - -package se.scalablesolutions.akka.dispatch - -import concurrent.forkjoin.LinkedTransferQueue -import java.util.concurrent.{TimeUnit, Semaphore} -import java.util.Iterator -import se.scalablesolutions.akka.util.Logger - -class BoundedTransferQueue[E <: AnyRef]( - val capacity: Int, - val pushTimeout: Long, - val pushTimeUnit: TimeUnit) - extends LinkedTransferQueue[E] { - require(capacity > 0) - require(pushTimeout > 0) - require(pushTimeUnit ne null) - - protected val guard = new Semaphore(capacity) - - override def take(): E = { - val e = super.take - if (e ne null) guard.release - e - } - - override def poll(): E = { - val e = super.poll - if (e ne null) guard.release - e - } - - override def poll(timeout: Long, unit: TimeUnit): E = { - val e = super.poll(timeout,unit) - if (e ne null) guard.release - e - } - - override def remainingCapacity = guard.availablePermits - - override def remove(o: AnyRef): Boolean = { - if (super.remove(o)) { - guard.release - true - } else { - false - } - } - - override def offer(e: E): Boolean = { - if (guard.tryAcquire(pushTimeout,pushTimeUnit)) { - val result = try { - super.offer(e) - } catch { - case e => guard.release; throw e - } - if (!result) guard.release - result - } else - false - } - - override def offer(e: E, timeout: Long, unit: TimeUnit): Boolean = { - if (guard.tryAcquire(pushTimeout,pushTimeUnit)) { - val result = try { - super.offer(e,timeout,unit) - } catch { - case e => guard.release; throw e - } - if (!result) guard.release - result - } else - false - } - - override def add(e: E): Boolean = { - if (guard.tryAcquire(pushTimeout,pushTimeUnit)) { - val result = try { - super.add(e) - } catch { - case e => guard.release; throw e - } - if (!result) guard.release - result - } else - false - } - - override def put(e :E): Unit = { - if (guard.tryAcquire(pushTimeout,pushTimeUnit)) { - try { - super.put(e) - } catch { - case e => guard.release; throw e - } - } - } - - override def tryTransfer(e: E): Boolean = { - if (guard.tryAcquire(pushTimeout,pushTimeUnit)) { - val result = try { - super.tryTransfer(e) - } catch { - case e => guard.release; throw e - } - if (!result) guard.release - result - } else - false - } - - override def tryTransfer(e: E, timeout: Long, unit: TimeUnit): Boolean = { - if (guard.tryAcquire(pushTimeout,pushTimeUnit)) { - val result = try { - super.tryTransfer(e,timeout,unit) - } catch { - case e => guard.release; throw e - } - if (!result) guard.release - result - } else - false - } - - override def transfer(e: E): Unit = { - if (guard.tryAcquire(pushTimeout,pushTimeUnit)) { - try { - super.transfer(e) - } catch { - case e => guard.release; throw e - } - } - } - - override def iterator: Iterator[E] = { - val it = super.iterator - new Iterator[E] { - def hasNext = it.hasNext - def next = it.next - def remove { - it.remove - guard.release //Assume remove worked if no exception was thrown - } - } - } -} \ No newline at end of file diff --git a/akka-actor/src/main/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcher.scala deleted file mode 100644 index d0850aa830..0000000000 --- a/akka-actor/src/main/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcher.scala +++ /dev/null @@ -1,59 +0,0 @@ -/** - * Copyright (C) 2009-2010 Scalable Solutions AB - */ - -/** - * Implements the Reactor pattern as defined in: [http://www.cs.wustl.edu/~schmidt/PDF/reactor-siemens.pdf]. - * See also this article: [http://today.java.net/cs/user/print/a/350]. - * - * Based on code from the actorom actor framework by Sergio Bossa [http://code.google.com/p/actorom/]. - */ -package se.scalablesolutions.akka.dispatch - -import java.util.{LinkedList, List} - -class ReactorBasedSingleThreadEventDrivenDispatcher(_name: String) - extends AbstractReactorBasedEventDrivenDispatcher("akka:event-driven:reactor:single-thread:dispatcher:" + _name) { - - def start = if (!active) { - log.debug("Starting up %s", toString) - active = true - val messageDemultiplexer = new Demultiplexer(queue) - selectorThread = new Thread(name) { - override def run = { - while (active) { - try { - messageDemultiplexer.select - } catch { case e: InterruptedException => active = false } - val selectedInvocations = messageDemultiplexer.acquireSelectedInvocations - val iter = selectedInvocations.iterator - while (iter.hasNext) { - val invocation = iter.next - val invoker = messageInvokers.get(invocation.receiver) - if (invoker ne null) invoker.invoke(invocation) - iter.remove - } - } - } - } - selectorThread.start - } - - def isShutdown = !active - - override def toString = "ReactorBasedSingleThreadEventDrivenDispatcher[" + name + "]" - - class Demultiplexer(private val messageQueue: ReactiveMessageQueue) extends MessageDemultiplexer { - - private val selectedQueue: List[MessageInvocation] = new LinkedList[MessageInvocation] - - def select = messageQueue.read(selectedQueue) - - def acquireSelectedInvocations: List[MessageInvocation] = selectedQueue - - def releaseSelectedInvocations = throw new UnsupportedOperationException("Demultiplexer can't release its queue") - - def wakeUp = throw new UnsupportedOperationException("Demultiplexer can't be woken up") - } -} - diff --git a/akka-actor/src/main/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcher.scala deleted file mode 100644 index 530184d4b2..0000000000 --- a/akka-actor/src/main/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcher.scala +++ /dev/null @@ -1,167 +0,0 @@ -/** - * Copyright (C) 2009-2010 Scalable Solutions AB - */ - -package se.scalablesolutions.akka.dispatch - -import java.util.concurrent.locks.ReentrantLock - -import java.util.{HashSet, HashMap, LinkedList, List} - -import se.scalablesolutions.akka.actor.IllegalActorStateException - -/** - * Implements the Reactor pattern as defined in: [http://www.cs.wustl.edu/~schmidt/PDF/reactor-siemens.pdf].
- * See also this article: [http://today.java.net/cs/user/print/a/350]. - *

- * - * Default settings are: - *

- *   - withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
- *   - NR_START_THREADS = 16
- *   - NR_MAX_THREADS = 128
- *   - KEEP_ALIVE_TIME = 60000L // one minute
- * 
- *

- * - * The dispatcher has a fluent builder interface to build up a thread pool to suite your use-case. - * There is a default thread pool defined but make use of the builder if you need it. Here are some examples. - *

- * - * Scala API. - *

- * Example usage: - *

- *   val dispatcher = new ReactorBasedThreadPoolEventDrivenDispatcher("name")
- *   dispatcher
- *     .withNewThreadPoolWithBoundedBlockingQueue(100)
- *     .setCorePoolSize(16)
- *     .setMaxPoolSize(128)
- *     .setKeepAliveTimeInMillis(60000)
- *     .setRejectionPolicy(new CallerRunsPolicy)
- *     .buildThreadPool
- * 
- *

- * - * Java API. - *

- * Example usage: - *

- *   ReactorBasedThreadPoolEventDrivenDispatcher dispatcher = new ReactorBasedThreadPoolEventDrivenDispatcher("name");
- *   dispatcher
- *     .withNewThreadPoolWithBoundedBlockingQueue(100)
- *     .setCorePoolSize(16)
- *     .setMaxPoolSize(128)
- *     .setKeepAliveTimeInMillis(60000)
- *     .setRejectionPolicy(new CallerRunsPolicy())
- *     .buildThreadPool();
- * 
- *

- * - * But the preferred way of creating dispatchers is to use - * the {@link se.scalablesolutions.akka.dispatch.Dispatchers} factory object. - * - * @author Jonas Bonér - */ -class ReactorBasedThreadPoolEventDrivenDispatcher(_name: String) - extends AbstractReactorBasedEventDrivenDispatcher("akka:event-driven:reactor:dispatcher:" + _name) - with ThreadPoolBuilder { - - private var fair = true - private val busyActors = new HashSet[AnyRef] - private val messageDemultiplexer = new Demultiplexer(queue) - - // build default thread pool - withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity.buildThreadPool - - def start = if (!active) { - log.debug("Starting up %s", toString) - active = true - - /** - * This dispatcher code is based on code from the actorom actor framework by Sergio Bossa - * [http://code.google.com/p/actorom/]. - */ - selectorThread = new Thread(name) { - override def run = { - while (active) { - try { - try { - messageDemultiplexer.select - } catch { case e: InterruptedException => active = false } - process(messageDemultiplexer.acquireSelectedInvocations) - } finally { - messageDemultiplexer.releaseSelectedInvocations - } - } - } - }; - selectorThread.start - } - - override protected def doShutdown = executor.shutdownNow - - private def process(selectedInvocations: List[MessageInvocation]) = synchronized { - var nrOfBusyMessages = 0 - val totalNrOfActors = messageInvokers.size - val totalNrOfBusyActors = busyActors.size - val invocations = selectedInvocations.iterator - while (invocations.hasNext && totalNrOfActors > totalNrOfBusyActors && passFairnessCheck(nrOfBusyMessages)) { - val invocation = invocations.next - if (invocation eq null) throw new IllegalActorStateException("Message invocation is null [" + invocation + "]") - if (!busyActors.contains(invocation.receiver)) { - val invoker = messageInvokers.get(invocation.receiver) - if (invoker eq null) throw new IllegalActorStateException( - "Message invoker for invocation [" + invocation + "] is null") - resume(invocation.receiver) - invocations.remove - executor.execute(new Runnable() { - def run = { - invoker.invoke(invocation) - suspend(invocation.receiver) - messageDemultiplexer.wakeUp - } - }) - } else nrOfBusyMessages += 1 - } - } - - private def resume(actor: AnyRef) = synchronized { - busyActors.add(actor) - } - - private def suspend(actor: AnyRef) = synchronized { - busyActors.remove(actor) - } - - private def passFairnessCheck(nrOfBusyMessages: Int) = { - if (fair) true - else nrOfBusyMessages < 100 - } - - def ensureNotActive(): Unit = if (active) throw new IllegalActorStateException( - "Can't build a new thread pool for a dispatcher that is already up and running") - - override def toString = "ReactorBasedThreadPoolEventDrivenDispatcher[" + name + "]" - - class Demultiplexer(private val messageQueue: ReactiveMessageQueue) extends MessageDemultiplexer { - private val selectedInvocations: List[MessageInvocation] = new LinkedList[MessageInvocation] - private val selectedInvocationsLock = new ReentrantLock - - def select = try { - selectedInvocationsLock.lock - messageQueue.read(selectedInvocations) - } finally { - selectedInvocationsLock.unlock - } - - def acquireSelectedInvocations: List[MessageInvocation] = { - selectedInvocationsLock.lock - selectedInvocations - } - - def releaseSelectedInvocations = selectedInvocationsLock.unlock - - def wakeUp = messageQueue.interrupt - } -} diff --git a/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala b/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala index 589500f413..eda5a86a9e 100644 --- a/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala @@ -8,8 +8,8 @@ import java.util.Queue import se.scalablesolutions.akka.actor.{Actor, ActorRef} import se.scalablesolutions.akka.config.Config.config -import java.util.concurrent.{TimeUnit, LinkedBlockingQueue} import concurrent.forkjoin.{TransferQueue, LinkedTransferQueue} +import java.util.concurrent.{ConcurrentLinkedQueue, BlockingQueue, TimeUnit, LinkedBlockingQueue} /** * Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue. @@ -17,10 +17,9 @@ import concurrent.forkjoin.{TransferQueue, LinkedTransferQueue} * @author Jonas Bonér */ class ThreadBasedDispatcher(private val actor: ActorRef, - val mailboxCapacity: Int = Dispatchers.MAILBOX_CAPACITY, - val pushTimeout: Long = 10000, - val pushTimeoutUnit: TimeUnit = TimeUnit.MILLISECONDS + val mailboxConfig: MailboxConfig ) extends MessageDispatcher { + def this(actor: ActorRef, capacity: Int) = this(actor,MailboxConfig(capacity,None,true)) def this(actor: ActorRef) = this(actor, Dispatchers.MAILBOX_CAPACITY)// For Java private val name = actor.getClass.getName + ":" + actor.uuid @@ -28,13 +27,8 @@ class ThreadBasedDispatcher(private val actor: ActorRef, private var selectorThread: Thread = _ @volatile private var active: Boolean = false - if (actor.mailbox eq null) { - actor.mailbox = if (mailboxCapacity > 0) - new BoundedTransferQueue[MessageInvocation](mailboxCapacity,pushTimeout,pushTimeoutUnit) with ThreadMessageQueue - else - new LinkedTransferQueue[MessageInvocation] with ThreadMessageQueue - } - + override def createMailbox(actorRef: ActorRef): AnyRef = mailboxConfig.newMailbox(blockDequeue = true) + override def register(actorRef: ActorRef) = { if(actorRef != actor) throw new IllegalArgumentException("Cannot register to anyone but " + actor) @@ -42,9 +36,11 @@ class ThreadBasedDispatcher(private val actor: ActorRef, super.register(actorRef) } - def mailbox = actor.mailbox.asInstanceOf[ThreadMessageQueue] + def mailbox = actor.mailbox.asInstanceOf[Queue[MessageInvocation] with MessageQueue] - def dispatch(invocation: MessageInvocation) = mailbox append invocation + def mailboxSize(a: ActorRef) = mailbox.size + + def dispatch(invocation: MessageInvocation) = mailbox enqueue invocation def start = if (!active) { log.debug("Starting up %s", toString) @@ -53,7 +49,7 @@ class ThreadBasedDispatcher(private val actor: ActorRef, override def run = { while (active) { try { - actor.invoke(mailbox.next) + actor.invoke(mailbox.dequeue) } catch { case e: InterruptedException => active = false } } } @@ -71,16 +67,4 @@ class ThreadBasedDispatcher(private val actor: ActorRef, } override def toString = "ThreadBasedDispatcher[" + threadName + "]" -} - -trait ThreadMessageQueue extends MessageQueue { self: TransferQueue[MessageInvocation] => - - final def append(invocation: MessageInvocation): Unit = { - if(!self.tryTransfer(invocation)) { //First, try to send the invocation to a waiting consumer - if(!self.offer(invocation)) //If no consumer found, append it to the queue, if that fails, we're aborting - throw new MessageQueueAppendFailedException("BlockingMessageTransferQueue transfer timed out") - } - } - - final def next: MessageInvocation = self.take -} +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala index 9fe47d5415..5ad1b89aca 100644 --- a/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala @@ -46,7 +46,7 @@ trait ThreadPoolBuilder extends Logging { if (boundedExecutorBound > 0) { val boundedExecutor = new BoundedExecutorDecorator(threadPoolBuilder, boundedExecutorBound) - boundedExecutorBound = -1 + boundedExecutorBound = -1 //Why is this here? executor = boundedExecutor } else { executor = threadPoolBuilder @@ -56,7 +56,6 @@ trait ThreadPoolBuilder extends Logging { def withNewThreadPoolWithCustomBlockingQueue(queue: BlockingQueue[Runnable]): ThreadPoolBuilder = synchronized { ensureNotActive verifyNotInConstructionPhase - inProcessOfBuilding = false blockingQueue = queue threadPoolBuilder = new ThreadPoolExecutor(NR_START_THREADS, NR_MAX_THREADS, KEEP_ALIVE_TIME, MILLISECONDS, queue) this @@ -209,7 +208,7 @@ trait ThreadPoolBuilder extends Logging { /** * @author Jonas Bonér */ - class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extends ExecutorService { + class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extends ExecutorService with Logging { protected val semaphore = new Semaphore(bound) def execute(command: Runnable) = { @@ -227,6 +226,9 @@ trait ThreadPoolBuilder extends Logging { } catch { case e: RejectedExecutionException => semaphore.release + case e => + log.error(e,"Unexpected exception") + throw e } } diff --git a/akka-actor/src/main/scala/routing/Iterators.scala b/akka-actor/src/main/scala/routing/Iterators.scala index 6e73af08e4..7f21589cfa 100644 --- a/akka-actor/src/main/scala/routing/Iterators.scala +++ b/akka-actor/src/main/scala/routing/Iterators.scala @@ -5,6 +5,7 @@ package se.scalablesolutions.akka.routing import se.scalablesolutions.akka.actor.ActorRef +import scala.collection.JavaConversions._ /** * An Iterator that is either always empty or yields an infinite number of Ts. @@ -15,6 +16,8 @@ trait InfiniteIterator[T] extends Iterator[T] * CyclicIterator is a round-robin style InfiniteIterator that cycles the supplied List. */ class CyclicIterator[T](items: List[T]) extends InfiniteIterator[T] { + def this(items: java.util.List[T]) = this(items.toList) + @volatile private[this] var current: List[T] = items def hasNext = items != Nil @@ -34,6 +37,7 @@ class CyclicIterator[T](items: List[T]) extends InfiniteIterator[T] { * useful for work-stealing. */ class SmallestMailboxFirstIterator(items : List[ActorRef]) extends InfiniteIterator[ActorRef] { + def this(items: java.util.List[ActorRef]) = this(items.toList) def hasNext = items != Nil def next = items.reduceLeft((a1, a2) => if (a1.mailboxSize < a2.mailboxSize) a1 else a2) diff --git a/akka-actor/src/main/scala/routing/Routers.scala b/akka-actor/src/main/scala/routing/Routers.scala index 7f2effee29..d916e6c584 100644 --- a/akka-actor/src/main/scala/routing/Routers.scala +++ b/akka-actor/src/main/scala/routing/Routers.scala @@ -4,7 +4,7 @@ package se.scalablesolutions.akka.routing -import se.scalablesolutions.akka.actor.{Actor, ActorRef} +import se.scalablesolutions.akka.actor.{UntypedActor, Actor, ActorRef} /** * A Dispatcher is a trait whose purpose is to route incoming messages to actors. @@ -26,6 +26,26 @@ trait Dispatcher { this: Actor => private def isSenderDefined = self.senderFuture.isDefined || self.sender.isDefined } +/** + * An UntypedDispatcher is an abstract class whose purpose is to route incoming messages to actors. + */ +abstract class UntypedDispatcher extends UntypedActor { + protected def transform(msg: Any): Any = msg + + protected def route(msg: Any): ActorRef + + private def isSenderDefined = self.senderFuture.isDefined || self.sender.isDefined + + @throws(classOf[Exception]) + def onReceive(msg: Any): Unit = { + val r = route(msg) + if(r eq null) + throw new IllegalStateException("No route for " + msg + " defined!") + if (isSenderDefined) r.forward(transform(msg))(someSelf) + else r.!(transform(msg))(None) + } +} + /** * A LoadBalancer is a specialized kind of Dispatcher, that is supplied an InfiniteIterator of targets * to dispatch incoming messages to. @@ -37,3 +57,17 @@ trait LoadBalancer extends Dispatcher { self: Actor => override def isDefinedAt(msg: Any) = seq.exists( _.isDefinedAt(msg) ) } + +/** + * A UntypedLoadBalancer is a specialized kind of UntypedDispatcher, that is supplied an InfiniteIterator of targets + * to dispatch incoming messages to. + */ +abstract class UntypedLoadBalancer extends UntypedDispatcher { + protected def seq: InfiniteIterator[ActorRef] + + protected def route(msg: Any) = + if (seq.hasNext) seq.next + else null + + override def isDefinedAt(msg: Any) = seq.exists( _.isDefinedAt(msg) ) +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/util/JavaAPI.scala b/akka-actor/src/main/scala/util/JavaAPI.scala new file mode 100644 index 0000000000..099082595d --- /dev/null +++ b/akka-actor/src/main/scala/util/JavaAPI.scala @@ -0,0 +1,23 @@ +package se.scalablesolutions.akka.util + +/** A Function interface + * Used to create first-class-functions is Java (sort of) + * Java API + */ +trait Function[T,R] { + def apply(param: T): R +} + +/** A Procedure is like a Function, but it doesn't produce a return value + * Java API + */ +trait Procedure[T] { + def apply(param: T): Unit +} + +/** + * An executable piece of code that takes no parameters and doesn't return any value + */ +trait SideEffect { + def apply: Unit +} diff --git a/akka-actor/src/main/scala/util/ListenerManagement.scala b/akka-actor/src/main/scala/util/ListenerManagement.scala index 0e17058380..7ad0f451f1 100644 --- a/akka-actor/src/main/scala/util/ListenerManagement.scala +++ b/akka-actor/src/main/scala/util/ListenerManagement.scala @@ -40,6 +40,23 @@ trait ListenerManagement extends Logging { if (manageLifeCycleOfListeners) listener.stop } + /* + * Returns whether there are any listeners currently + */ + def hasListeners: Boolean = !listeners.isEmpty + + protected def notifyListeners(message: => Any) { + if (hasListeners) { + val msg = message + val iterator = listeners.iterator + while (iterator.hasNext) { + val listener = iterator.next + if (listener.isRunning) listener ! msg + else log.warning("Can't notify [%s] since it is not running.", listener) + } + } + } + /** * Execute f with each listener as argument. */ diff --git a/akka-actor/src/main/scala/util/LockUtil.scala b/akka-actor/src/main/scala/util/LockUtil.scala index 885e11def7..3d1261e468 100644 --- a/akka-actor/src/main/scala/util/LockUtil.scala +++ b/akka-actor/src/main/scala/util/LockUtil.scala @@ -5,6 +5,7 @@ package se.scalablesolutions.akka.util import java.util.concurrent.locks.{ReentrantReadWriteLock, ReentrantLock} +import java.util.concurrent.atomic.AtomicBoolean /** * @author Jonas Bonér @@ -58,3 +59,56 @@ class ReadWriteGuard { } } +/** + * A very simple lock that uses CCAS (Compare Compare-And-Swap) + * Does not keep track of the owner and isn't Reentrant, so don't nest and try to stick to the if*-methods + */ +class SimpleLock { + val acquired = new AtomicBoolean(false) + + def ifPossible(perform: () => Unit): Boolean = { + if (tryLock()) { + try { + perform + } finally { + unlock() + } + true + } else false + } + + def ifPossibleYield[T](perform: () => T): Option[T] = { + if (tryLock()) { + try { + Some(perform()) + } finally { + unlock() + } + } else None + } + + def ifPossibleApply[T,R](value: T)(function: (T) => R): Option[R] = { + if (tryLock()) { + try { + Some(function(value)) + } finally { + unlock() + } + } else None + } + + def tryLock() = { + if (acquired.get) false + else acquired.compareAndSet(false,true) + } + + def tryUnlock() = { + acquired.compareAndSet(true,false) + } + + def locked = acquired.get + + def unlock() { + acquired.set(false) + } +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/util/Logging.scala b/akka-actor/src/main/scala/util/Logging.scala index f13c229653..b6ddaaa16a 100644 --- a/akka-actor/src/main/scala/util/Logging.scala +++ b/akka-actor/src/main/scala/util/Logging.scala @@ -99,18 +99,26 @@ class Logger(val logger: SLFLogger) { warning(t,message(fmt,arg,argN:_*)) } + def warn(t: Throwable, fmt: => String, arg: Any, argN: Any*) = warning(t, fmt, arg, argN) + def warning(t: Throwable, msg: => String) { if (warning_?) logger.warn(msg,t) } + def warn(t: Throwable, msg: => String) = warning(t, msg) + def warning(fmt: => String, arg: Any, argN: Any*) { warning(message(fmt,arg,argN:_*)) } + def warn(fmt: => String, arg: Any, argN: Any*) = warning(fmt, arg, argN) + def warning(msg: => String) { if (warning_?) logger warn msg } + def warn(msg: => String) = warning(msg) + //Error def error(t: Throwable, fmt: => String, arg: Any, argN: Any*) { error(t,message(fmt,arg,argN:_*)) diff --git a/akka-actor/src/main/scala/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/util/ReflectiveAccess.scala index 4582304188..abccd5d9b0 100644 --- a/akka-actor/src/main/scala/util/ReflectiveAccess.scala +++ b/akka-actor/src/main/scala/util/ReflectiveAccess.scala @@ -62,14 +62,8 @@ object ReflectiveAccess { def ensureRemotingEnabled = if (!isRemotingEnabled) throw new ModuleNotAvailableException( "Can't load the remoting module, make sure that akka-remote.jar is on the classpath") - val remoteClientObjectInstance: Option[RemoteClientObject] = { - try { - val clazz = loader.loadClass("se.scalablesolutions.akka.remote.RemoteClient$") - val ctor = clazz.getDeclaredConstructor(Array[Class[_]](): _*) - ctor.setAccessible(true) - Some(ctor.newInstance(Array[AnyRef](): _*).asInstanceOf[RemoteClientObject]) - } catch { case e: Exception => None } - } + val remoteClientObjectInstance: Option[RemoteClientObject] = + getObject("se.scalablesolutions.akka.remote.RemoteClient$") def register(address: InetSocketAddress, uuid: String) = { ensureRemotingEnabled @@ -126,23 +120,11 @@ object ReflectiveAccess { def unregister(actorRef: ActorRef): Unit } - val remoteServerObjectInstance: Option[RemoteServerObject] = { - try { - val clazz = loader.loadClass("se.scalablesolutions.akka.remote.RemoteServer$") - val ctor = clazz.getDeclaredConstructor(Array[Class[_]](): _*) - ctor.setAccessible(true) - Some(ctor.newInstance(Array[AnyRef](): _*).asInstanceOf[RemoteServerObject]) - } catch { case e: Exception => None } - } + val remoteServerObjectInstance: Option[RemoteServerObject] = + getObject("se.scalablesolutions.akka.remote.RemoteServer$") - val remoteNodeObjectInstance: Option[RemoteNodeObject] = { - try { - val clazz = loader.loadClass("se.scalablesolutions.akka.remote.RemoteNode$") - val ctor = clazz.getDeclaredConstructor(Array[Class[_]](): _*) - ctor.setAccessible(true) - Some(ctor.newInstance(Array[AnyRef](): _*).asInstanceOf[RemoteNodeObject]) - } catch { case e: Exception => None } - } + val remoteNodeObjectInstance: Option[RemoteNodeObject] = + getObject("se.scalablesolutions.akka.remote.RemoteNode$") def registerActor(address: InetSocketAddress, uuid: String, actorRef: ActorRef) = { ensureRemotingEnabled @@ -177,14 +159,8 @@ object ReflectiveAccess { def ensureTypedActorEnabled = if (!isTypedActorEnabled) throw new ModuleNotAvailableException( "Can't load the typed actor module, make sure that akka-typed-actor.jar is on the classpath") - val typedActorObjectInstance: Option[TypedActorObject] = { - try { - val clazz = loader.loadClass("se.scalablesolutions.akka.actor.TypedActor$") - val ctor = clazz.getDeclaredConstructor(Array[Class[_]](): _*) - ctor.setAccessible(true) - Some(ctor.newInstance(Array[AnyRef](): _*).asInstanceOf[TypedActorObject]) - } catch { case e: Exception => None } - } + val typedActorObjectInstance: Option[TypedActorObject] = + getObject("se.scalablesolutions.akka.actor.TypedActor$") def resolveFutureIfMessageIsJoinPoint(message: Any, future: Future[_]): Boolean = { ensureTypedActorEnabled @@ -212,18 +188,46 @@ object ReflectiveAccess { def ensureJtaEnabled = if (!isJtaEnabled) throw new ModuleNotAvailableException( "Can't load the typed actor module, make sure that akka-jta.jar is on the classpath") - val transactionContainerObjectInstance: Option[TransactionContainerObject] = { - try { - val clazz = loader.loadClass("se.scalablesolutions.akka.actor.TransactionContainer$") - val ctor = clazz.getDeclaredConstructor(Array[Class[_]](): _*) - ctor.setAccessible(true) - Some(ctor.newInstance(Array[AnyRef](): _*).asInstanceOf[TransactionContainerObject]) - } catch { case e: Exception => None } - } + val transactionContainerObjectInstance: Option[TransactionContainerObject] = + getObject("se.scalablesolutions.akka.actor.TransactionContainer$") def createTransactionContainer: TransactionContainer = { ensureJtaEnabled transactionContainerObjectInstance.get.apply.asInstanceOf[TransactionContainer] } } + + val noParams = Array[Class[_]]() + val noArgs = Array[AnyRef]() + + def createInstance[T](clazz: Class[_], + params: Array[Class[_]], + args: Array[AnyRef]): Option[T] = try { + val ctor = clazz.getDeclaredConstructor(params: _*) + ctor.setAccessible(true) + Some(ctor.newInstance(args: _*).asInstanceOf[T]) + } catch { + case e: Exception => None + } + + def createInstance[T](fqn: String, + params: Array[Class[_]], + args: Array[AnyRef], + classloader: ClassLoader = loader): Option[T] = try { + val clazz = classloader.loadClass(fqn) + val ctor = clazz.getDeclaredConstructor(params: _*) + ctor.setAccessible(true) + Some(ctor.newInstance(args: _*).asInstanceOf[T]) + } catch { + case e: Exception => None + } + + def getObject[T](fqn: String, classloader: ClassLoader = loader): Option[T] = try {//Obtains a reference to $MODULE$ + val clazz = classloader.loadClass(fqn) + val instance = clazz.getDeclaredField("MODULE$") + instance.setAccessible(true) + Option(instance.get(null).asInstanceOf[T]) + } catch { + case e: Exception => None + } } diff --git a/akka-actor/src/test/resources/logback-test.xml b/akka-actor/src/test/resources/logback-test.xml deleted file mode 100644 index 78eae40ec4..0000000000 --- a/akka-actor/src/test/resources/logback-test.xml +++ /dev/null @@ -1,21 +0,0 @@ - - - - - - - - - - - - - - [%4p] [%d{ISO8601}] [%t] %c{1}: %m%n - - - - - - - diff --git a/akka-actor/src/test/scala/actor/actor/Bench.scala b/akka-actor/src/test/scala/actor/actor/Bench.scala index 8e3a44f3a0..ded90edad5 100644 --- a/akka-actor/src/test/scala/actor/actor/Bench.scala +++ b/akka-actor/src/test/scala/actor/actor/Bench.scala @@ -78,7 +78,7 @@ object Chameneos { var sumMeetings = 0 var numFaded = 0 - override def init = { + override def preStart = { for (i <- 0 until numChameneos) actorOf(new Chameneo(self, colours(i % 3), i)) } diff --git a/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala b/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala index 5023c756e1..b9fa238963 100644 --- a/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala +++ b/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala @@ -42,7 +42,7 @@ class RestartStrategySpec extends JUnitSuite { restartLatch.open } - override def shutdown = { + override def postStop = { if (restartLatch.isOpen) { secondRestartLatch.open } @@ -70,5 +70,32 @@ class RestartStrategySpec extends JUnitSuite { } assert(exceptionLatch.tryAwait(1, TimeUnit.SECONDS)) } + + @Test + def slaveShouldBeImmortalWithoutMaxRestarts = { + + val boss = actorOf(new Actor{ + self.trapExit = List(classOf[Throwable]) + self.faultHandler = Some(OneForOneStrategy(None, None)) + protected def receive = { case _ => () } + }).start + + val countDownLatch = new CountDownLatch(100) + + val slave = actorOf(new Actor{ + + protected def receive = { + case Crash => throw new Exception("Crashing...") + } + + override def postRestart(reason: Throwable) = { + countDownLatch.countDown + } + }) + + boss.startLink(slave) + (1 to 100) foreach { _ => slave ! Crash } + assert(countDownLatch.await(120, TimeUnit.SECONDS)) + } } diff --git a/akka-actor/src/test/scala/dispatch/DispatchersSpec.scala b/akka-actor/src/test/scala/dispatch/DispatchersSpec.scala index bb548b9251..81fd933cda 100644 --- a/akka-actor/src/test/scala/dispatch/DispatchersSpec.scala +++ b/akka-actor/src/test/scala/dispatch/DispatchersSpec.scala @@ -28,13 +28,9 @@ object DispatchersSpec { def ofType[T <: MessageDispatcher : Manifest]: (MessageDispatcher) => Boolean = _.getClass == manifest[T].erasure def typesAndValidators: Map[String,(MessageDispatcher) => Boolean] = Map( - "ReactorBasedSingleThreadEventDriven" -> ofType[ReactorBasedSingleThreadEventDrivenDispatcher], "ExecutorBasedEventDrivenWorkStealing" -> ofType[ExecutorBasedEventDrivenWorkStealingDispatcher], "ExecutorBasedEventDriven" -> ofType[ExecutorBasedEventDrivenDispatcher], - "ReactorBasedThreadPoolEventDriven" -> ofType[ReactorBasedThreadPoolEventDrivenDispatcher], "Hawt" -> ofType[HawtDispatcher], - "GlobalReactorBasedSingleThreadEventDriven" -> instance(globalReactorBasedSingleThreadEventDrivenDispatcher), - "GlobalReactorBasedThreadPoolEventDriven" -> instance(globalReactorBasedThreadPoolEventDrivenDispatcher), "GlobalExecutorBasedEventDriven" -> instance(globalExecutorBasedEventDrivenDispatcher), "GlobalHawt" -> instance(globalHawtDispatcher) ) diff --git a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala index 9cdf43682e..3935bc9b0b 100644 --- a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala @@ -3,9 +3,10 @@ package se.scalablesolutions.akka.actor.dispatch import java.util.concurrent.{CountDownLatch, TimeUnit} import org.scalatest.junit.JUnitSuite import org.junit.Test -import se.scalablesolutions.akka.dispatch.Dispatchers +import se.scalablesolutions.akka.dispatch.{Dispatchers,ExecutorBasedEventDrivenDispatcher} import se.scalablesolutions.akka.actor.Actor import Actor._ +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} object ExecutorBasedEventDrivenDispatcherActorSpec { class TestActor extends Actor { @@ -65,4 +66,73 @@ class ExecutorBasedEventDrivenDispatcherActorSpec extends JUnitSuite { } actor.stop } + + @Test def shouldRespectThroughput { + val throughputDispatcher = new ExecutorBasedEventDrivenDispatcher("THROUGHPUT",101,0,Dispatchers.MAILBOX_CONFIG, (e) => { + e.setCorePoolSize(1) + }) + + val works = new AtomicBoolean(true) + val latch = new CountDownLatch(100) + val start = new CountDownLatch(1) + val fastOne = actorOf( + new Actor { + self.dispatcher = throughputDispatcher + def receive = { case "sabotage" => works.set(false) } + }).start + + val slowOne = actorOf( + new Actor { + self.dispatcher = throughputDispatcher + def receive = { + case "hogexecutor" => start.await + case "ping" => if (works.get) latch.countDown + } + }).start + + slowOne ! "hogexecutor" + (1 to 100) foreach { _ => slowOne ! "ping"} + fastOne ! "sabotage" + start.countDown + val result = latch.await(3,TimeUnit.SECONDS) + fastOne.stop + slowOne.stop + throughputDispatcher.shutdown + assert(result === true) + } + + @Test def shouldRespectThroughputDeadline { + val deadlineMs = 100 + val throughputDispatcher = new ExecutorBasedEventDrivenDispatcher("THROUGHPUT",2,deadlineMs,Dispatchers.MAILBOX_CONFIG, (e) => { + e.setCorePoolSize(1) + }) + + val works = new AtomicBoolean(true) + val latch = new CountDownLatch(1) + val start = new CountDownLatch(1) + val ready = new CountDownLatch(1) + + val fastOne = actorOf( + new Actor { + self.dispatcher = throughputDispatcher + def receive = { case "ping" => if(works.get) latch.countDown; self.stop } + }).start + + val slowOne = actorOf( + new Actor { + self.dispatcher = throughputDispatcher + def receive = { + case "hogexecutor" => ready.countDown; start.await + case "ping" => works.set(false); self.stop + } + }).start + + slowOne ! "hogexecutor" + slowOne ! "ping" + fastOne ! "ping" + assert(ready.await(5,TimeUnit.SECONDS) === true) + Thread.sleep(deadlineMs) + start.countDown + assert(latch.await(10,TimeUnit.SECONDS) === true) + } } diff --git a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala index cde57a0544..3285e450c6 100644 --- a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala @@ -5,11 +5,10 @@ import org.scalatest.junit.JUnitSuite import org.junit.Test -import se.scalablesolutions.akka.dispatch.Dispatchers - import java.util.concurrent.{TimeUnit, CountDownLatch} import se.scalablesolutions.akka.actor.{IllegalActorStateException, Actor} import Actor._ +import se.scalablesolutions.akka.dispatch.{MessageQueue, Dispatchers} object ExecutorBasedEventDrivenWorkStealingDispatcherSpec { val delayableActorDispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher") @@ -18,7 +17,7 @@ object ExecutorBasedEventDrivenWorkStealingDispatcherSpec { class DelayableActor(name: String, delay: Int, finishedCounter: CountDownLatch) extends Actor { self.dispatcher = delayableActorDispatcher - var invocationCount = 0 + @volatile var invocationCount = 0 self.id = name def receive = { @@ -61,10 +60,14 @@ class ExecutorBasedEventDrivenWorkStealingDispatcherSpec extends JUnitSuite with val slow = actorOf(new DelayableActor("slow", 50, finishedCounter)).start val fast = actorOf(new DelayableActor("fast", 10, finishedCounter)).start + var sentToFast = 0 + for (i <- 1 to 100) { // send most work to slow actor - if (i % 20 == 0) + if (i % 20 == 0) { fast ! i + sentToFast += 1 + } else slow ! i } @@ -72,13 +75,18 @@ class ExecutorBasedEventDrivenWorkStealingDispatcherSpec extends JUnitSuite with // now send some messages to actors to keep the dispatcher dispatching messages for (i <- 1 to 10) { Thread.sleep(150) - if (i % 2 == 0) + if (i % 2 == 0) { fast ! i + sentToFast += 1 + } else slow ! i } finishedCounter.await(5, TimeUnit.SECONDS) + fast.mailbox.asInstanceOf[MessageQueue].isEmpty must be(true) + slow.mailbox.asInstanceOf[MessageQueue].isEmpty must be(true) + fast.actor.asInstanceOf[DelayableActor].invocationCount must be > sentToFast fast.actor.asInstanceOf[DelayableActor].invocationCount must be > (slow.actor.asInstanceOf[DelayableActor].invocationCount) slow.stop diff --git a/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala b/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala index 97f2e0df9d..8d4c8dedc1 100644 --- a/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala +++ b/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala @@ -53,7 +53,7 @@ object HawtDispatcherEchoServer { var accept_source:DispatchSource = _ var sessions = ListBuffer[ActorRef]() - override def init = { + override def preStart = { channel = ServerSocketChannel.open(); channel.socket().bind(new InetSocketAddress(port)); channel.configureBlocking(false); @@ -122,7 +122,7 @@ object HawtDispatcherEchoServer { var writeCounter = 0L var closed = false - override def init = { + override def preStart = { if(useReactorPattern) { // Then we will be using the reactor pattern for handling IO: @@ -154,7 +154,7 @@ object HawtDispatcherEchoServer { println("Accepted connection from: "+remote_address); } - override def shutdown = { + override def postStop = { closed = true read_source.release write_source.release diff --git a/akka-actor/src/test/scala/dispatch/MailboxConfigSpec.scala b/akka-actor/src/test/scala/dispatch/MailboxConfigSpec.scala new file mode 100644 index 0000000000..27afdbbce6 --- /dev/null +++ b/akka-actor/src/test/scala/dispatch/MailboxConfigSpec.scala @@ -0,0 +1,53 @@ +package se.scalablesolutions.akka.actor.dispatch + +import org.scalatest.junit.JUnitSuite +import org.junit.Test +import se.scalablesolutions.akka.actor.Actor +import Actor._ +import java.util.concurrent.{BlockingQueue, CountDownLatch, TimeUnit} +import se.scalablesolutions.akka.util.Duration +import se.scalablesolutions.akka.dispatch.{MessageQueueAppendFailedException, MessageInvocation, MailboxConfig, Dispatchers} +import java.util.concurrent.atomic.{AtomicReference} + +object MailboxConfigSpec { + +} + +class MailboxConfigSpec extends JUnitSuite { + import MailboxConfigSpec._ + + private val unit = TimeUnit.MILLISECONDS + + @Test def shouldCreateUnboundedQueue = { + val m = MailboxConfig(-1,None,false) + assert(m.newMailbox().asInstanceOf[BlockingQueue[MessageInvocation]].remainingCapacity === Integer.MAX_VALUE) + } + + @Test def shouldCreateBoundedQueue = { + val m = MailboxConfig(1,None,false) + assert(m.newMailbox().asInstanceOf[BlockingQueue[MessageInvocation]].remainingCapacity === 1) + } + + @Test(expected = classOf[MessageQueueAppendFailedException]) def shouldThrowMessageQueueAppendFailedExceptionWhenTimeOutEnqueue = { + val m = MailboxConfig(1,Some(Duration(1,unit)),false) + val testActor = actorOf( new Actor { def receive = { case _ => }} ) + val mbox = m.newMailbox() + (1 to 10000) foreach { i => mbox.enqueue(new MessageInvocation(testActor,i,None,None,None)) } + } + + + @Test def shouldBeAbleToDequeueUnblocking = { + val m = MailboxConfig(1,Some(Duration(1,unit)),false) + val mbox = m.newMailbox() + val latch = new CountDownLatch(1) + val t = new Thread { override def run = { + mbox.dequeue + latch.countDown + }} + t.start + val result = latch.await(5000,unit) + if (!result) + t.interrupt + assert(result === true) + } +} diff --git a/akka-actor/src/test/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcherActorSpec.scala deleted file mode 100644 index de9b912bf5..0000000000 --- a/akka-actor/src/test/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcherActorSpec.scala +++ /dev/null @@ -1,71 +0,0 @@ -package se.scalablesolutions.akka.actor.dispatch - -import java.util.concurrent.{CountDownLatch, TimeUnit} -import org.scalatest.junit.JUnitSuite -import org.junit.Test - -import se.scalablesolutions.akka.dispatch.Dispatchers -import se.scalablesolutions.akka.actor.Actor -import Actor._ - -object ReactorBasedSingleThreadEventDrivenDispatcherActorSpec { - class TestActor extends Actor { - self.dispatcher = Dispatchers.newReactorBasedSingleThreadEventDrivenDispatcher(self.uuid) - - def receive = { - case "Hello" => - self.reply("World") - case "Failure" => - throw new RuntimeException("Expected exception; to test fault-tolerance") - } - } - - object OneWayTestActor { - val oneWay = new CountDownLatch(1) - } - class OneWayTestActor extends Actor { - self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(self.uuid) - def receive = { - case "OneWay" => OneWayTestActor.oneWay.countDown - } - } -} - -class ReactorBasedSingleThreadEventDrivenDispatcherActorSpec extends JUnitSuite { - import ReactorBasedSingleThreadEventDrivenDispatcherActorSpec._ - - private val unit = TimeUnit.MILLISECONDS - - @Test def shouldSendOneWay = { - val actor = actorOf[OneWayTestActor].start - val result = actor ! "OneWay" - assert(OneWayTestActor.oneWay.await(1, TimeUnit.SECONDS)) - actor.stop - } - - @Test def shouldSendReplySync = { - val actor = actorOf[TestActor].start - val result = (actor !! ("Hello", 10000)).as[String].get - assert("World" === result) - actor.stop - } - - @Test def shouldSendReplyAsync = { - val actor = actorOf[TestActor].start - val result = actor !! "Hello" - assert("World" === result.get.asInstanceOf[String]) - actor.stop - } - - @Test def shouldSendReceiveException = { - val actor = actorOf[TestActor].start - try { - actor !! "Failure" - fail("Should have thrown an exception") - } catch { - case e => - assert("Expected exception; to test fault-tolerance" === e.getMessage()) - } - actor.stop - } -} diff --git a/akka-actor/src/test/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcherActorSpec.scala deleted file mode 100644 index 4001df8f56..0000000000 --- a/akka-actor/src/test/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcherActorSpec.scala +++ /dev/null @@ -1,66 +0,0 @@ -package se.scalablesolutions.akka.actor.dispatch - -import java.util.concurrent.{CountDownLatch, TimeUnit} -import org.scalatest.junit.JUnitSuite -import org.junit.Test - -import se.scalablesolutions.akka.dispatch.Dispatchers -import se.scalablesolutions.akka.actor.Actor -import Actor._ - -object ReactorBasedThreadPoolEventDrivenDispatcherActorSpec { - class TestActor extends Actor { - self.dispatcher = Dispatchers.newReactorBasedThreadPoolEventDrivenDispatcher(self.uuid) - def receive = { - case "Hello" => - self.reply("World") - case "Failure" => - throw new RuntimeException("Expected exception; to test fault-tolerance") - } - } -} - -class ReactorBasedThreadPoolEventDrivenDispatcherActorSpec extends JUnitSuite { - import ReactorBasedThreadPoolEventDrivenDispatcherActorSpec._ - - private val unit = TimeUnit.MILLISECONDS - - @Test def shouldSendOneWay { - val oneWay = new CountDownLatch(1) - val actor = actorOf(new Actor { - self.dispatcher = Dispatchers.newReactorBasedThreadPoolEventDrivenDispatcher(self.uuid) - def receive = { - case "OneWay" => oneWay.countDown - } - }).start - val result = actor ! "OneWay" - assert(oneWay.await(1, TimeUnit.SECONDS)) - actor.stop - } - - @Test def shouldSendReplySync = { - val actor = actorOf[TestActor].start - val result = (actor !! ("Hello", 10000)).as[String].get - assert("World" === result) - actor.stop - } - - @Test def shouldSendReplyAsync = { - val actor = actorOf[TestActor].start - val result = actor !! "Hello" - assert("World" === result.get.asInstanceOf[String]) - actor.stop - } - - @Test def shouldSendReceiveException = { - val actor = actorOf[TestActor].start - try { - actor !! "Failure" - fail("Should have thrown an exception") - } catch { - case e => - assert("Expected exception; to test fault-tolerance" === e.getMessage()) - } - actor.stop - } -} diff --git a/akka-actor/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala b/akka-actor/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala index 44cd9aade3..7ecef80e39 100644 --- a/akka-actor/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala @@ -85,7 +85,7 @@ class ThreadBasedDispatcherSpec extends JUnitSuite { } assert(handleLatch.await(5, TimeUnit.SECONDS)) assert(!threadingIssueDetected.get) - dispatcher.shutdown + dispatcher.postStop } } */ diff --git a/akka-actor/src/test/scala/misc/ActorRegistrySpec.scala b/akka-actor/src/test/scala/misc/ActorRegistrySpec.scala index 8c9e0778ca..1fe72d6c68 100644 --- a/akka-actor/src/test/scala/misc/ActorRegistrySpec.scala +++ b/akka-actor/src/test/scala/misc/ActorRegistrySpec.scala @@ -219,37 +219,35 @@ class ActorRegistrySpec extends JUnitSuite { @Test def shouldBeAbleToRegisterActorsConcurrently { ActorRegistry.shutdownAll - val latch = new CountDownLatch(3) - val barrier = new CyclicBarrier(3) - - def mkTestActor(i:Int) = actorOf( new Actor { + def mkTestActors = for(i <- (1 to 10).toList;j <- 1 to 3000) yield actorOf( new Actor { self.id = i.toString def receive = { case _ => } }) - def mkTestActors = for(i <- 1 to 10;j <- 1 to 1000) yield mkTestActor(i) + val latch = new CountDownLatch(3) + val barrier = new CyclicBarrier(3) def mkThread(actors: Iterable[ActorRef]) = new Thread { - start + this.start override def run { barrier.await actors foreach { _.start } latch.countDown } } + val a1,a2,a3 = mkTestActors + val t1 = mkThread(a1) + val t2 = mkThread(a2) + val t3 = mkThread(a3) - val testActors1 = mkTestActors - val testActors2 = mkTestActors - val testActors3 = mkTestActors - - mkThread(testActors1) - mkThread(testActors2) - mkThread(testActors3) assert(latch.await(30,TimeUnit.SECONDS) === true) for(i <- 1 to 10) { - assert(ActorRegistry.actorsFor(i.toString).length === 3000) + val theId = i.toString + val actors = ActorRegistry.actorsFor(theId).toSet + for(a <- actors if a.id == theId) assert(actors contains a) + assert(actors.size === 9000) } } } diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala index d3f0acd1cf..0ca9046093 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala +++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala @@ -108,10 +108,10 @@ private[amqp] class ConsumerActor(consumerParameters: ConsumerParameters) super.preRestart(reason) } - override def shutdown = { + override def postStop = { listenerTag.foreach(tag => channel.foreach(_.basicCancel(tag))) self.shutdownLinkedActors - super.shutdown + super.postStop } override def toString = diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala index f45553520d..ecb3029444 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala +++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala @@ -46,7 +46,7 @@ object ExampleSession { printTopic("Happy hAkking :-)") - // shutdown everything the amqp tree except the main AMQP supervisor + // postStop everything the amqp tree except the main AMQP supervisor // all connections/consumers/producers will be stopped AMQP.shutdownAll diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala index 5ecae4c6d3..4d642df554 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala +++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala @@ -103,5 +103,5 @@ abstract private[amqp] class FaultTolerantChannelActor( closeChannel } - override def shutdown = closeChannel + override def postStop = closeChannel } diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala index 1e50a985be..0fd3f715b5 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala +++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala @@ -104,9 +104,9 @@ private[amqp] class FaultTolerantConnectionActor(connectionParameters: Connectio connectionCallback.foreach(cb => if (cb.isRunning) cb ! message) } - override def shutdown = { + override def postStop = { reconnectionTimer.cancel - // make sure shutdown is called on all linked actors so they can do channel cleanup before connection is killed + // make sure postStop is called on all linked actors so they can do channel cleanup before connection is killed self.shutdownLinkedActors disconnect } diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcClientActor.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcClientActor.scala index 5c717cb8bb..10596e393f 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcClientActor.scala +++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcClientActor.scala @@ -40,9 +40,9 @@ class RpcClientActor[I,O]( } - override def shutdown = { + override def postStop = { rpcClient.foreach(rpc => rpc.close) - super.shutdown + super.postStop } override def toString = "AMQP.RpcClient[exchange=" +exchangeName + ", routingKey=" + routingKey+ "]" diff --git a/akka-camel/src/main/scala/Producer.scala b/akka-camel/src/main/scala/Producer.scala index 8764b91b4c..0be07e9737 100644 --- a/akka-camel/src/main/scala/Producer.scala +++ b/akka-camel/src/main/scala/Producer.scala @@ -54,10 +54,10 @@ trait ProducerSupport { this: Actor => def headersToCopy: Set[String] = headersToCopyDefault /** - * Default implementation of Actor.shutdown for freeing resources needed + * Default implementation of Actor.postStop for freeing resources needed * to actually send messages to endpointUri. */ - override def shutdown { + override def postStop { processor.stop } diff --git a/akka-camel/src/main/scala/component/ActorComponent.scala b/akka-camel/src/main/scala/component/ActorComponent.scala index a5d56dd9dc..6c1c5902fa 100644 --- a/akka-camel/src/main/scala/component/ActorComponent.scala +++ b/akka-camel/src/main/scala/component/ActorComponent.scala @@ -247,8 +247,8 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout[T](message: Any, timeout: Long, senderOption: Option[ActorRef], senderFuture: Option[CompletableFuture[T]]) = unsupported protected[akka] def mailbox: AnyRef = unsupported protected[akka] def mailbox_=(msg: AnyRef):AnyRef = unsupported - protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported - protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): 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 handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported protected[akka] def linkedActors: JavaMap[String, ActorRef] = unsupported protected[akka] def linkedActorsAsList: List[ActorRef] = unsupported diff --git a/akka-core/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala b/akka-core/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala index 3d048684cd..26fdb6e1ef 100644 --- a/akka-core/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala +++ b/akka-core/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala @@ -26,7 +26,7 @@ class SupervisorMiscSpec extends WordSpec with MustMatchers { }).start val actor2 = Actor.actorOf(new Actor { - self.dispatcher = Dispatchers.newReactorBasedSingleThreadEventDrivenDispatcher("test") + self.dispatcher = Dispatchers.newThreadBasedDispatcher(self) override def postRestart(cause: Throwable) {countDownLatch.countDown} protected def receive = { diff --git a/akka-http/src/main/scala/AkkaBroadcaster.scala b/akka-http/src/main/scala/AkkaBroadcaster.scala index 8f724ff445..ca5abc6f1d 100644 --- a/akka-http/src/main/scala/AkkaBroadcaster.scala +++ b/akka-http/src/main/scala/AkkaBroadcaster.scala @@ -9,10 +9,13 @@ import se.scalablesolutions.akka.actor.Actor._ import se.scalablesolutions.akka.actor.Actor import se.scalablesolutions.akka.dispatch.Dispatchers -class AkkaBroadcaster extends org.atmosphere.jersey.JerseyBroadcaster { - name = classOf[AkkaBroadcaster].getName - +object AkkaBroadcaster { val broadcasterDispatcher = Dispatchers.fromConfig("akka.rest.comet-dispatcher") +} + +class AkkaBroadcaster extends org.atmosphere.jersey.JerseyBroadcaster { + import AkkaBroadcaster._ + name = classOf[AkkaBroadcaster].getName //FIXME should be supervised val caster = actorOf(new Actor { diff --git a/akka-http/src/main/scala/AkkaClusterBroadcastFilter.scala b/akka-http/src/main/scala/AkkaClusterBroadcastFilter.scala index 775c8b554d..7ea963872f 100644 --- a/akka-http/src/main/scala/AkkaClusterBroadcastFilter.scala +++ b/akka-http/src/main/scala/AkkaClusterBroadcastFilter.scala @@ -24,6 +24,11 @@ class AkkaClusterBroadcastFilter extends Actor with ClusterBroadcastFilter { @BeanProperty var clusterName = "" @BeanProperty var broadcaster : Broadcaster = null + def init() { + //Since this class is instantiated by Atmosphere, we need to make sure it's started + self.start + } + /** * Stops the actor */ @@ -48,7 +53,4 @@ class AkkaClusterBroadcastFilter extends Actor with ClusterBroadcastFilter { case b @ ClusterCometBroadcast(c, _) if (c == clusterName) && (broadcaster ne null) => broadcaster broadcast b case _ => } - - //Since this class is instantiated by Atmosphere, we need to make sure it's started - self.start } diff --git a/akka-http/src/main/scala/Initializer.scala b/akka-http/src/main/scala/Initializer.scala index da95a39b77..c1cd8bfc87 100644 --- a/akka-http/src/main/scala/Initializer.scala +++ b/akka-http/src/main/scala/Initializer.scala @@ -13,7 +13,7 @@ import se.scalablesolutions.akka.util.{Logging, Bootable} import javax.servlet.{ServletContextListener, ServletContextEvent} /** - * This class can be added to web.xml mappings as a listener to start and shutdown Akka. + * This class can be added to web.xml mappings as a listener to start and postStop Akka. * * * ... diff --git a/akka-jta/src/main/scala/AtomikosTransactionService.scala b/akka-jta/src/main/scala/AtomikosTransactionService.scala index 305ddb6ace..4acbb1a013 100644 --- a/akka-jta/src/main/scala/AtomikosTransactionService.scala +++ b/akka-jta/src/main/scala/AtomikosTransactionService.scala @@ -36,6 +36,6 @@ class AtomikosTransactionService extends TransactionService with TransactionProt "Could not create a new Atomikos J2EE Transaction Manager, due to: " + e.toString) } ))) - // TODO: gracefully shutdown of the TM - //txService.shutdown(false) + // TODO: gracefully postStop of the TM + //txService.postStop(false) } diff --git a/akka-kernel/src/main/scala/EmbeddedAppServer.scala b/akka-kernel/src/main/scala/EmbeddedAppServer.scala index b77a215157..9afcfbe572 100644 --- a/akka-kernel/src/main/scala/EmbeddedAppServer.scala +++ b/akka-kernel/src/main/scala/EmbeddedAppServer.scala @@ -6,12 +6,13 @@ package se.scalablesolutions.akka.kernel import javax.ws.rs.core.UriBuilder import javax.servlet.ServletConfig +import java.io.File import se.scalablesolutions.akka.actor.BootableActorLoaderService import se.scalablesolutions.akka.util.{Bootable, Logging} -import se.scalablesolutions.akka.comet.{ AkkaServlet } +import se.scalablesolutions.akka.comet.AkkaServlet + import org.eclipse.jetty.xml.XmlConfiguration -import java.io.File import org.eclipse.jetty.server.{Handler, Server} import org.eclipse.jetty.server.handler.{HandlerList, HandlerCollection, ContextHandler} @@ -28,16 +29,17 @@ trait EmbeddedAppServer extends Bootable with Logging { protected var server: Option[Server] = None - abstract override def onLoad = { + abstract override def onLoad = { super.onLoad if (config.getBool("akka.rest.service", true)) { log.info("Attempting to start Akka REST service (Jersey)") System.setProperty("jetty.port",REST_PORT.toString) System.setProperty("jetty.host",REST_HOSTNAME) - System.setProperty("jetty.home",HOME.get + "/deploy/root") + System.setProperty("jetty.home",HOME.getOrElse(throwNoAkkaHomeException) + "/deploy/root") - val configuration = new XmlConfiguration(new File(HOME.get + "/config/microkernel-server.xml").toURI.toURL) + val configuration = new XmlConfiguration( + new File(HOME.getOrElse(throwNoAkkaHomeException) + "/config/microkernel-server.xml").toURI.toURL) server = Option(configuration.configure.asInstanceOf[Server]) map { s => //Set the correct classloader to our contexts applicationLoader foreach { loader => diff --git a/akka-kernel/src/main/scala/Kernel.scala b/akka-kernel/src/main/scala/Kernel.scala index 6489a60680..646ca34bcc 100644 --- a/akka-kernel/src/main/scala/Kernel.scala +++ b/akka-kernel/src/main/scala/Kernel.scala @@ -15,7 +15,7 @@ object Main { } /** - * The Akka Kernel, is used to start And shutdown Akka in standalone/kernel mode. + * The Akka Kernel, is used to start And postStop Akka in standalone/kernel mode. * * @author Jonas Bonér */ diff --git a/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorage.scala b/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorage.scala index be5fc4f4c7..0c6f239ef7 100644 --- a/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorage.scala +++ b/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorage.scala @@ -29,7 +29,7 @@ object CassandraStorage extends Storage { * * @author Jonas Bonér */ -class CassandraPersistentMap(id: String) extends PersistentMap[Array[Byte], Array[Byte]] { +class CassandraPersistentMap(id: String) extends PersistentMapBinary { val uuid = id val storage = CassandraStorageBackend } diff --git a/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala b/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala index ccaf7518f1..4d9ff48a60 100644 --- a/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala +++ b/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala @@ -7,9 +7,11 @@ package se.scalablesolutions.akka.persistence.common import se.scalablesolutions.akka.stm._ import se.scalablesolutions.akka.stm.TransactionManagement.transaction import se.scalablesolutions.akka.util.Logging -import se.scalablesolutions.akka.AkkaException -class StorageException(message: String) extends AkkaException(message) +// FIXME move to 'stm' package + add message with more info +class NoTransactionInScopeException extends RuntimeException + +class StorageException(message: String) extends RuntimeException(message) /** * Example Scala usage. @@ -80,24 +82,90 @@ trait Storage { */ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V] with Transactional with Committable with Abortable with Logging { - protected val newAndUpdatedEntries = TransactionalMap[K, V]() - protected val removedEntries = TransactionalVector[K]() protected val shouldClearOnCommit = Ref[Boolean]() + // operations on the Map + trait Op + case object GET extends Op + case object PUT extends Op + case object REM extends Op + case object UPD extends Op + + // append only log: records all mutating operations + protected val appendOnlyTxLog = TransactionalVector[LogEntry]() + + case class LogEntry(key: K, value: Option[V], op: Op) + + // need to override in subclasses e.g. "sameElements" for Array[Byte] + def equal(k1: K, k2: K): Boolean = k1 == k2 + + // Seqable type that's required for maintaining the log of distinct keys affected in current transaction + type T <: Equals + + // converts key K to the Seqable type Equals + def toEquals(k: K): T + + // keys affected in the current transaction + protected val keysInCurrentTx = TransactionalMap[T, K]() + + protected def addToListOfKeysInTx(key: K): Unit = + keysInCurrentTx += (toEquals(key), key) + + protected def clearDistinctKeys = keysInCurrentTx.clear + + protected def filterTxLogByKey(key: K): IndexedSeq[LogEntry] = + appendOnlyTxLog filter(e => equal(e.key, key)) + + // need to get current value considering the underlying storage as well as the transaction log + protected def getCurrentValue(key: K): Option[V] = { + + // get all mutating entries for this key for this tx + val txEntries = filterTxLogByKey(key) + + // get the snapshot from the underlying store for this key + val underlying = try { + storage.getMapStorageEntryFor(uuid, key) + } catch { case e: Exception => None } + + if (txEntries.isEmpty) underlying + else replay(txEntries, key, underlying) + } + + // replay all tx entries for key k with seed = initial + private def replay(txEntries: IndexedSeq[LogEntry], key: K, initial: Option[V]): Option[V] = { + import scala.collection.mutable._ + + val m = initial match { + case None => Map.empty[K, V] + case Some(v) => Map((key, v)) + } + txEntries.foreach {case LogEntry(k, v, o) => o match { + case PUT => m.put(k, v.get) + case REM => m -= k + case UPD => m.update(k, v.get) + }} + m get key + } + // to be concretized in subclasses val storage: MapStorageBackend[K, V] def commit = { - if (shouldClearOnCommit.isDefined && shouldClearOnCommit.get) storage.removeMapStorageFor(uuid) - removedEntries.toList.foreach(key => storage.removeMapStorageFor(uuid, key)) - storage.insertMapStorageEntriesFor(uuid, newAndUpdatedEntries.toList) - newAndUpdatedEntries.clear - removedEntries.clear + // if (shouldClearOnCommit.isDefined && shouldClearOnCommit.get) storage.removeMapStorageFor(uuid) + + appendOnlyTxLog.foreach { case LogEntry(k, v, o) => o match { + case PUT => storage.insertMapStorageEntryFor(uuid, k, v.get) + case UPD => storage.insertMapStorageEntryFor(uuid, k, v.get) + case REM => storage.removeMapStorageFor(uuid, k) + }} + + appendOnlyTxLog.clear + clearDistinctKeys } def abort = { - newAndUpdatedEntries.clear - removedEntries.clear + appendOnlyTxLog.clear + clearDistinctKeys shouldClearOnCommit.swap(false) } @@ -118,68 +186,84 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V] override def put(key: K, value: V): Option[V] = { register - newAndUpdatedEntries.put(key, value) + val curr = getCurrentValue(key) + appendOnlyTxLog add LogEntry(key, Some(value), PUT) + addToListOfKeysInTx(key) + curr } override def update(key: K, value: V) = { register - newAndUpdatedEntries.update(key, value) + val curr = getCurrentValue(key) + appendOnlyTxLog add LogEntry(key, Some(value), UPD) + addToListOfKeysInTx(key) + curr } override def remove(key: K) = { register - removedEntries.add(key) - newAndUpdatedEntries.get(key) + val curr = getCurrentValue(key) + appendOnlyTxLog add LogEntry(key, None, REM) + addToListOfKeysInTx(key) + curr } - def slice(start: Option[K], count: Int): List[Tuple2[K, V]] = + def slice(start: Option[K], count: Int): List[(K, V)] = slice(start, None, count) - def slice(start: Option[K], finish: Option[K], count: Int): List[Tuple2[K, V]] = try { - storage.getMapStorageRangeFor(uuid, start, finish, count) - } catch { case e: Exception => Nil } + def slice(start: Option[K], finish: Option[K], count: Int): List[(K, V)] override def clear = { register + appendOnlyTxLog.clear + clearDistinctKeys shouldClearOnCommit.swap(true) } override def contains(key: K): Boolean = try { - newAndUpdatedEntries.contains(key) || - storage.getMapStorageEntryFor(uuid, key).isDefined + filterTxLogByKey(key) match { + case Seq() => // current tx doesn't use this + storage.getMapStorageEntryFor(uuid, key).isDefined // check storage + case txs => // present in log + txs.last.op != REM // last entry cannot be a REM + } } catch { case e: Exception => false } + protected def existsInStorage(key: K): Option[V] = try { + storage.getMapStorageEntryFor(uuid, key) + } catch { + case e: Exception => None + } + override def size: Int = try { - storage.getMapStorageSizeFor(uuid) - } catch { case e: Exception => 0 } + // partition key set affected in current tx into those which r added & which r deleted + val (keysAdded, keysRemoved) = keysInCurrentTx.map { + case (kseq, k) => ((kseq, k), getCurrentValue(k)) + }.partition(_._2.isDefined) - override def get(key: K): Option[V] = { - if (newAndUpdatedEntries.contains(key)) { - newAndUpdatedEntries.get(key) - } - else try { - storage.getMapStorageEntryFor(uuid, key) - } catch { case e: Exception => None } + // keys which existed in storage but removed in current tx + val inStorageRemovedInTx = + keysRemoved.keySet + .map(_._2) + .filter(k => existsInStorage(k).isDefined) + .size + + // all keys in storage + val keysInStorage = + storage.getMapStorageFor(uuid) + .map { case (k, v) => toEquals(k) } + .toSet + + // (keys that existed UNION keys added ) - (keys removed) + (keysInStorage union keysAdded.keySet.map(_._1)).size - inStorageRemovedInTx + } catch { + case e: Exception => 0 } - def iterator = elements + // get must consider underlying storage & current uncommitted tx log + override def get(key: K): Option[V] = getCurrentValue(key) - override def elements: Iterator[Tuple2[K, V]] = { - new Iterator[Tuple2[K, V]] { - private val originalList: List[Tuple2[K, V]] = try { - storage.getMapStorageFor(uuid) - } catch { - case e: Throwable => Nil - } - private var elements = newAndUpdatedEntries.toList union originalList.reverse - override def next: Tuple2[K, V]= synchronized { - val element = elements.head - elements = elements.tail - element - } - override def hasNext: Boolean = synchronized { !elements.isEmpty } - } - } + def iterator: Iterator[Tuple2[K, V]] private def register = { if (transaction.get.isEmpty) throw new NoTransactionInScopeException @@ -187,6 +271,95 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V] } } +trait PersistentMapBinary extends PersistentMap[Array[Byte], Array[Byte]] { + import scala.collection.mutable.ArraySeq + + type T = ArraySeq[Byte] + def toEquals(k: Array[Byte]) = ArraySeq(k: _*) + override def equal(k1: Array[Byte], k2: Array[Byte]): Boolean = k1 sameElements k2 + + object COrdering { + implicit object ArraySeqOrdering extends Ordering[ArraySeq[Byte]] { + def compare(o1: ArraySeq[Byte], o2: ArraySeq[Byte]) = + new String(o1.toArray) compare new String(o2.toArray) + } + } + + import scala.collection.immutable.{TreeMap, SortedMap} + private def replayAllKeys: SortedMap[ArraySeq[Byte], Array[Byte]] = { + import COrdering._ + + // need ArraySeq for ordering + val fromStorage = + TreeMap(storage.getMapStorageFor(uuid).map { case (k, v) => (ArraySeq(k: _*), v) }: _*) + + val (keysAdded, keysRemoved) = keysInCurrentTx.map { + case (_, k) => (k, getCurrentValue(k)) + }.partition(_._2.isDefined) + + val inStorageRemovedInTx = + keysRemoved.keySet + .filter(k => existsInStorage(k).isDefined) + .map(k => ArraySeq(k: _*)) + + (fromStorage -- inStorageRemovedInTx) ++ keysAdded.map { case (k, Some(v)) => (ArraySeq(k: _*), v) } + } + + override def slice(start: Option[Array[Byte]], finish: Option[Array[Byte]], count: Int): List[(Array[Byte], Array[Byte])] = try { + val newMap = replayAllKeys + + if (newMap isEmpty) List[(Array[Byte], Array[Byte])]() + + val startKey = + start match { + case Some(bytes) => Some(ArraySeq(bytes: _*)) + case None => None + } + + val endKey = + finish match { + case Some(bytes) => Some(ArraySeq(bytes: _*)) + case None => None + } + + ((startKey, endKey, count): @unchecked) match { + case ((Some(s), Some(e), _)) => + newMap.range(s, e) + .toList + .map(e => (e._1.toArray, e._2)) + .toList + case ((Some(s), None, c)) if c > 0 => + newMap.from(s) + .iterator + .take(count) + .map(e => (e._1.toArray, e._2)) + .toList + case ((Some(s), None, _)) => + newMap.from(s) + .toList + .map(e => (e._1.toArray, e._2)) + .toList + case ((None, Some(e), _)) => + newMap.until(e) + .toList + .map(e => (e._1.toArray, e._2)) + .toList + } + } catch { case e: Exception => Nil } + + override def iterator: Iterator[(Array[Byte], Array[Byte])] = { + new Iterator[(Array[Byte], Array[Byte])] { + private var elements = replayAllKeys + override def next: (Array[Byte], Array[Byte]) = synchronized { + val (k, v) = elements.head + elements = elements.tail + (k.toArray, v) + } + override def hasNext: Boolean = synchronized { !elements.isEmpty } + } + } +} + /** * Implements a template for a concrete persistent transactional vector based storage. * @@ -198,42 +371,83 @@ trait PersistentVector[T] extends IndexedSeq[T] with Transactional with Committa protected val removedElems = TransactionalVector[T]() protected val shouldClearOnCommit = Ref[Boolean]() + // operations on the Vector + trait Op + case object ADD extends Op + case object UPD extends Op + case object POP extends Op + + // append only log: records all mutating operations + protected val appendOnlyTxLog = TransactionalVector[LogEntry]() + + case class LogEntry(index: Option[Int], value: Option[T], op: Op) + + // need to override in subclasses e.g. "sameElements" for Array[Byte] + def equal(v1: T, v2: T): Boolean = v1 == v2 + val storage: VectorStorageBackend[T] def commit = { - for (element <- newElems) storage.insertVectorStorageEntryFor(uuid, element) - for (entry <- updatedElems) storage.updateVectorStorageEntryFor(uuid, entry._1, entry._2) - newElems.clear - updatedElems.clear + for(entry <- appendOnlyTxLog) { + entry match { + case LogEntry(_, Some(v), ADD) => storage.insertVectorStorageEntryFor(uuid, v) + case LogEntry(Some(i), Some(v), UPD) => storage.updateVectorStorageEntryFor(uuid, i, v) + case LogEntry(_, _, POP) => //.. + } + } + appendOnlyTxLog.clear } def abort = { - newElems.clear - updatedElems.clear - removedElems.clear + appendOnlyTxLog.clear shouldClearOnCommit.swap(false) } + private def replay: List[T] = { + import scala.collection.mutable.ArrayBuffer + var elemsStorage = ArrayBuffer(storage.getVectorStorageRangeFor(uuid, None, None, storage.getVectorStorageSizeFor(uuid)).reverse: _*) + + for(entry <- appendOnlyTxLog) { + entry match { + case LogEntry(_, Some(v), ADD) => elemsStorage += v + case LogEntry(Some(i), Some(v), UPD) => elemsStorage.update(i, v) + case LogEntry(_, _, POP) => elemsStorage = elemsStorage.drop(1) + } + } + elemsStorage.toList.reverse + } + def +(elem: T) = add(elem) def add(elem: T) = { register - newElems + elem + appendOnlyTxLog + LogEntry(None, Some(elem), ADD) } def apply(index: Int): T = get(index) def get(index: Int): T = { - if (newElems.size > index) newElems(index) - else storage.getVectorStorageEntryFor(uuid, index) + if (appendOnlyTxLog.isEmpty) { + storage.getVectorStorageEntryFor(uuid, index) + } else { + val curr = replay + curr(index) + } } override def slice(start: Int, finish: Int): IndexedSeq[T] = slice(Some(start), Some(finish)) def slice(start: Option[Int], finish: Option[Int], count: Int = 0): IndexedSeq[T] = { - val buffer = new scala.collection.mutable.ArrayBuffer[T] - storage.getVectorStorageRangeFor(uuid, start, finish, count).foreach(buffer.append(_)) - buffer + val curr = replay + val s = if (start.isDefined) start.get else 0 + val cnt = + if (finish.isDefined) { + val f = finish.get + if (f >= s) (f - s) else count + } + else count + if (s == 0 && cnt == 0) List().toIndexedSeq + else curr.slice(s, s + cnt).toIndexedSeq } /** @@ -241,12 +455,13 @@ trait PersistentVector[T] extends IndexedSeq[T] with Transactional with Committa */ def pop: T = { register + appendOnlyTxLog + LogEntry(None, None, POP) throw new UnsupportedOperationException("PersistentVector::pop is not implemented") } def update(index: Int, newElem: T) = { register - storage.updateVectorStorageEntryFor(uuid, index, newElem) + appendOnlyTxLog + LogEntry(Some(index), Some(newElem), UPD) } override def first: T = get(0) @@ -260,7 +475,7 @@ trait PersistentVector[T] extends IndexedSeq[T] with Transactional with Committa } } - def length: Int = storage.getVectorStorageSizeFor(uuid) + newElems.length + def length: Int = replay.length private def register = { if (transaction.get.isEmpty) throw new NoTransactionInScopeException diff --git a/akka-persistence/akka-persistence-common/src/main/scala/StorageBackend.scala b/akka-persistence/akka-persistence-common/src/main/scala/StorageBackend.scala index df74040b68..7e6a95f9a1 100644 --- a/akka-persistence/akka-persistence-common/src/main/scala/StorageBackend.scala +++ b/akka-persistence/akka-persistence-common/src/main/scala/StorageBackend.scala @@ -38,7 +38,7 @@ trait RefStorageBackend[T] extends StorageBackend { // for Queue trait QueueStorageBackend[T] extends StorageBackend { // add to the end of the queue - def enqueue(name: String, item: T): Boolean + def enqueue(name: String, item: T): Option[Int] // pop from the front of the queue def dequeue(name: String): Option[T] diff --git a/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorage.scala b/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorage.scala index 98776253a5..83e47e3ba5 100644 --- a/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorage.scala +++ b/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorage.scala @@ -9,7 +9,7 @@ import se.scalablesolutions.akka.persistence.common._ import se.scalablesolutions.akka.util.UUID object MongoStorage extends Storage { - type ElementType = AnyRef + type ElementType = Array[Byte] def newMap: PersistentMap[ElementType, ElementType] = newMap(UUID.newUuid.toString) def newVector: PersistentVector[ElementType] = newVector(UUID.newUuid.toString) @@ -29,7 +29,7 @@ object MongoStorage extends Storage { * * @author Debasish Ghosh */ -class MongoPersistentMap(id: String) extends PersistentMap[AnyRef, AnyRef] { +class MongoPersistentMap(id: String) extends PersistentMapBinary { val uuid = id val storage = MongoStorageBackend } @@ -40,12 +40,12 @@ class MongoPersistentMap(id: String) extends PersistentMap[AnyRef, AnyRef] { * * @author Debaissh Ghosh */ -class MongoPersistentVector(id: String) extends PersistentVector[AnyRef] { +class MongoPersistentVector(id: String) extends PersistentVector[Array[Byte]] { val uuid = id val storage = MongoStorageBackend } -class MongoPersistentRef(id: String) extends PersistentRef[AnyRef] { +class MongoPersistentRef(id: String) extends PersistentRef[Array[Byte]] { val uuid = id val storage = MongoStorageBackend } diff --git a/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorageBackend.scala b/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorageBackend.scala index 950165567d..01d8ababce 100644 --- a/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorageBackend.scala +++ b/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorageBackend.scala @@ -9,13 +9,8 @@ import se.scalablesolutions.akka.persistence.common._ import se.scalablesolutions.akka.util.Logging import se.scalablesolutions.akka.config.Config.config -import sjson.json.Serializer._ - import java.util.NoSuchElementException - -import com.mongodb._ - -import java.util.{Map=>JMap, List=>JList, ArrayList=>JArrayList} +import com.novus.casbah.mongodb.Imports._ /** * A module for supporting MongoDB based persistence. @@ -28,294 +23,208 @@ import java.util.{Map=>JMap, List=>JList, ArrayList=>JArrayList} * @author Debasish Ghosh */ private[akka] object MongoStorageBackend extends - MapStorageBackend[AnyRef, AnyRef] with - VectorStorageBackend[AnyRef] with - RefStorageBackend[AnyRef] with + MapStorageBackend[Array[Byte], Array[Byte]] with + VectorStorageBackend[Array[Byte]] with + RefStorageBackend[Array[Byte]] with Logging { - // enrich with null safe findOne - class RichDBCollection(value: DBCollection) { - def findOneNS(o: DBObject): Option[DBObject] = { - value.findOne(o) match { - case null => None - case x => Some(x) - } - } - } - - implicit def enrichDBCollection(c: DBCollection) = new RichDBCollection(c) - - val KEY = "key" - val VALUE = "value" + val KEY = "__key" + val REF = "__ref" val COLLECTION = "akka_coll" - val MONGODB_SERVER_HOSTNAME = config.getString("akka.storage.mongodb.hostname", "127.0.0.1") - val MONGODB_SERVER_DBNAME = config.getString("akka.storage.mongodb.dbname", "testdb") - val MONGODB_SERVER_PORT = config.getInt("akka.storage.mongodb.port", 27017) + val HOSTNAME = config.getString("akka.storage.mongodb.hostname", "127.0.0.1") + val DBNAME = config.getString("akka.storage.mongodb.dbname", "testdb") + val PORT = config.getInt("akka.storage.mongodb.port", 27017) - val db = new Mongo(MONGODB_SERVER_HOSTNAME, MONGODB_SERVER_PORT) - val coll = db.getDB(MONGODB_SERVER_DBNAME).getCollection(COLLECTION) + val db: MongoDB = MongoConnection(HOSTNAME, PORT)(DBNAME) + val coll: MongoCollection = db(COLLECTION) - private[this] val serializer = SJSON + def drop() { db.dropDatabase() } - def insertMapStorageEntryFor(name: String, key: AnyRef, value: AnyRef) { + def insertMapStorageEntryFor(name: String, key: Array[Byte], value: Array[Byte]) { insertMapStorageEntriesFor(name, List((key, value))) } - def insertMapStorageEntriesFor(name: String, entries: List[Tuple2[AnyRef, AnyRef]]) { - import java.util.{Map, HashMap} - - val m: Map[AnyRef, AnyRef] = new HashMap - for ((k, v) <- entries) { - m.put(k, serializer.out(v)) - } - - nullSafeFindOne(name) match { - case None => - coll.insert(new BasicDBObject().append(KEY, name).append(VALUE, m)) - case Some(dbo) => { - // collate the maps - val o = dbo.get(VALUE).asInstanceOf[Map[AnyRef, AnyRef]] - o.putAll(m) - - val newdbo = new BasicDBObject().append(KEY, name).append(VALUE, o) - coll.update(new BasicDBObject().append(KEY, name), newdbo, true, false) + def insertMapStorageEntriesFor(name: String, entries: List[(Array[Byte], Array[Byte])]) { + db.safely { db => + val q: DBObject = MongoDBObject(KEY -> name) + coll.findOne(q) match { + case Some(dbo) => + entries.foreach { case (k, v) => dbo += new String(k) -> v } + db.safely { db => coll.update(q, dbo, true, false) } + case None => + val builder = MongoDBObject.newBuilder + builder += KEY -> name + entries.foreach { case (k, v) => builder += new String(k) -> v } + coll += builder.result.asDBObject } } } def removeMapStorageFor(name: String): Unit = { - val q = new BasicDBObject - q.put(KEY, name) - coll.remove(q) + val q: DBObject = MongoDBObject(KEY -> name) + db.safely { db => coll.remove(q) } } - def removeMapStorageFor(name: String, key: AnyRef): Unit = { - nullSafeFindOne(name) match { - case None => - case Some(dbo) => { - val orig = dbo.get(VALUE).asInstanceOf[DBObject].toMap - if (key.isInstanceOf[List[_]]) { - val keys = key.asInstanceOf[List[_]] - keys.foreach(k => orig.remove(k.asInstanceOf[String])) - } else { - orig.remove(key.asInstanceOf[String]) - } - // remove existing reference - removeMapStorageFor(name) - // and insert - coll.insert(new BasicDBObject().append(KEY, name).append(VALUE, orig)) - } + private def queryFor[T](name: String)(body: (MongoDBObject, Option[DBObject]) => T): T = { + val q = MongoDBObject(KEY -> name) + body(q, coll.findOne(q)) + } + + def removeMapStorageFor(name: String, key: Array[Byte]): Unit = queryFor(name) { (q, dbo) => + dbo.foreach { d => + d -= new String(key) + db.safely { db => coll.update(q, d, true, false) } } } - def getMapStorageEntryFor(name: String, key: AnyRef): Option[AnyRef] = - getValueForKey(name, key.asInstanceOf[String]) - - def getMapStorageSizeFor(name: String): Int = { - nullSafeFindOne(name) match { - case None => 0 - case Some(dbo) => - dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]].keySet.size - } + def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = queryFor(name) { (q, dbo) => + dbo.map { d => + d.getAs[Array[Byte]](new String(key)) + }.getOrElse(None) } - def getMapStorageFor(name: String): List[Tuple2[AnyRef, AnyRef]] = { - val m = - nullSafeFindOne(name) match { - case None => - throw new NoSuchElementException(name + " not present") - case Some(dbo) => - dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]] - } - val n = - List(m.keySet.toArray: _*).asInstanceOf[List[String]] - val vals = - for(s <- n) - yield (s, serializer.in[AnyRef](m.get(s).asInstanceOf[Array[Byte]])) - vals.asInstanceOf[List[Tuple2[String, AnyRef]]] + def getMapStorageSizeFor(name: String): Int = queryFor(name) { (q, dbo) => + dbo.map { d => + d.size - 2 // need to exclude object id and our KEY + }.getOrElse(0) } - def getMapStorageRangeFor(name: String, start: Option[AnyRef], - finish: Option[AnyRef], - count: Int): List[Tuple2[AnyRef, AnyRef]] = { - val m = - nullSafeFindOne(name) match { - case None => - throw new NoSuchElementException(name + " not present") - case Some(dbo) => - dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]] - } - - /** - * count is the max number of results to return. Start with - * start or 0 (if start is not defined) and go until - * you hit finish or count. - */ - val s = if (start.isDefined) start.get.asInstanceOf[Int] else 0 - val cnt = - if (finish.isDefined) { - val f = finish.get.asInstanceOf[Int] - if (f >= s) math.min(count, (f - s)) else count - } - else count - - val n = - List(m.keySet.toArray: _*).asInstanceOf[List[String]].sortWith((e1, e2) => (e1 compareTo e2) < 0).slice(s, s + cnt) - val vals = - for(s <- n) - yield (s, serializer.in[AnyRef](m.get(s).asInstanceOf[Array[Byte]])) - vals.asInstanceOf[List[Tuple2[String, AnyRef]]] + def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = queryFor(name) { (q, dbo) => + dbo.map { d => + for { + (k, v) <- d.toList + if k != "_id" && k != KEY + } yield (k.getBytes, v.asInstanceOf[Array[Byte]]) + }.getOrElse(List.empty[(Array[Byte], Array[Byte])]) } - private def getValueForKey(name: String, key: String): Option[AnyRef] = { - try { - nullSafeFindOne(name) match { - case None => None - case Some(dbo) => - Some(serializer.in[AnyRef]( - dbo.get(VALUE) - .asInstanceOf[JMap[String, AnyRef]] - .get(key).asInstanceOf[Array[Byte]])) - } - } catch { - case e => - throw new NoSuchElementException(e.getMessage) - } + def getMapStorageRangeFor(name: String, start: Option[Array[Byte]], + finish: Option[Array[Byte]], + count: Int): List[(Array[Byte], Array[Byte])] = queryFor(name) { (q, dbo) => + dbo.map { d => + // get all keys except the special ones + val keys = d.keys + .filter(k => k != "_id" && k != KEY) + .toList + .sortWith(_ < _) + + // if the supplied start is not defined, get the head of keys + val s = start.map(new String(_)).getOrElse(keys.head) + + // if the supplied finish is not defined, get the last element of keys + val f = finish.map(new String(_)).getOrElse(keys.last) + + // slice from keys: both ends inclusive + val ks = keys.slice(keys.indexOf(s), scala.math.min(count, keys.indexOf(f) + 1)) + ks.map(k => (k.getBytes, d.getAs[Array[Byte]](k).get)) + }.getOrElse(List.empty[(Array[Byte], Array[Byte])]) } - def insertVectorStorageEntriesFor(name: String, elements: List[AnyRef]) = { - val q = new BasicDBObject - q.put(KEY, name) - - val currentList = - coll.findOneNS(q) match { - case None => - new JArrayList[AnyRef] - case Some(dbo) => - dbo.get(VALUE).asInstanceOf[JArrayList[AnyRef]] - } - if (!currentList.isEmpty) { - // record exists - // remove before adding - coll.remove(q) - } - - // add to the current list - elements.map(serializer.out(_)).foreach(currentList.add(_)) - - coll.insert( - new BasicDBObject() - .append(KEY, name) - .append(VALUE, currentList) - ) - } - - def insertVectorStorageEntryFor(name: String, element: AnyRef) = { + def insertVectorStorageEntryFor(name: String, element: Array[Byte]) = { insertVectorStorageEntriesFor(name, List(element)) } - def getVectorStorageEntryFor(name: String, index: Int): AnyRef = { - try { - val o = - nullSafeFindOne(name) match { - case None => - throw new NoSuchElementException(name + " not present") + def insertVectorStorageEntriesFor(name: String, elements: List[Array[Byte]]) = { + // lookup with name + val q: DBObject = MongoDBObject(KEY -> name) - case Some(dbo) => - dbo.get(VALUE).asInstanceOf[JList[AnyRef]] + db.safely { db => + coll.findOne(q) match { + // exists : need to update + case Some(dbo) => + dbo -= KEY + dbo -= "_id" + val listBuilder = MongoDBList.newBuilder + + // expensive! + listBuilder ++= (elements ++ dbo.toSeq.sortWith((e1, e2) => (e1._1.toInt < e2._1.toInt)).map(_._2)) + + val builder = MongoDBObject.newBuilder + builder += KEY -> name + builder ++= listBuilder.result + coll.update(q, builder.result.asDBObject, true, false) + + // new : just add + case None => + val listBuilder = MongoDBList.newBuilder + listBuilder ++= elements + + val builder = MongoDBObject.newBuilder + builder += KEY -> name + builder ++= listBuilder.result + coll += builder.result.asDBObject } - serializer.in[AnyRef]( - o.get(index).asInstanceOf[Array[Byte]]) - } catch { - case e => - throw new NoSuchElementException(e.getMessage) } } - def getVectorStorageRangeFor(name: String, - start: Option[Int], finish: Option[Int], count: Int): List[AnyRef] = { - try { - val o = - nullSafeFindOne(name) match { - case None => - throw new NoSuchElementException(name + " not present") + def updateVectorStorageEntryFor(name: String, index: Int, elem: Array[Byte]) = queryFor(name) { (q, dbo) => + dbo.foreach { d => + d += ((index.toString, elem)) + db.safely { db => coll.update(q, d, true, false) } + } + } - case Some(dbo) => - dbo.get(VALUE).asInstanceOf[JList[AnyRef]] - } + def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = queryFor(name) { (q, dbo) => + dbo.map { d => + d(index.toString).asInstanceOf[Array[Byte]] + }.getOrElse(Array.empty[Byte]) + } - val s = if (start.isDefined) start.get else 0 + /** + * if start and finish both are defined, ignore count and + * report the range [start, finish) + * if start is not defined, assume start = 0 + * if start == 0 and finish == 0, return an empty collection + */ + def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): List[Array[Byte]] = queryFor(name) { (q, dbo) => + dbo.map { d => + val ls = d.filter { case (k, v) => k != KEY && k != "_id" } + .toSeq + .sortWith((e1, e2) => (e1._1.toInt < e2._1.toInt)) + .map(_._2) + + val st = start.getOrElse(0) val cnt = if (finish.isDefined) { val f = finish.get - if (f >= s) (f - s) else count + if (f >= st) (f - st) else count } else count - - // pick the subrange and make a Scala list - val l = - List(o.subList(s, s + cnt).toArray: _*) - - for(e <- l) - yield serializer.in[AnyRef](e.asInstanceOf[Array[Byte]]) - } catch { - case e => - throw new NoSuchElementException(e.getMessage) - } + if (st == 0 && cnt == 0) List() + ls.slice(st, st + cnt).asInstanceOf[List[Array[Byte]]] + }.getOrElse(List.empty[Array[Byte]]) } - def updateVectorStorageEntryFor(name: String, index: Int, elem: AnyRef) = { - val q = new BasicDBObject - q.put(KEY, name) - - val dbobj = - coll.findOneNS(q) match { - case None => - throw new NoSuchElementException(name + " not present") - case Some(dbo) => dbo - } - val currentList = dbobj.get(VALUE).asInstanceOf[JArrayList[AnyRef]] - currentList.set(index, serializer.out(elem)) - coll.update(q, - new BasicDBObject().append(KEY, name).append(VALUE, currentList)) + def getVectorStorageSizeFor(name: String): Int = queryFor(name) { (q, dbo) => + dbo.map { d => d.size - 2 }.getOrElse(0) } - def getVectorStorageSizeFor(name: String): Int = { - nullSafeFindOne(name) match { - case None => 0 - case Some(dbo) => - dbo.get(VALUE).asInstanceOf[JList[AnyRef]].size - } - } + def insertRefStorageFor(name: String, element: Array[Byte]) = { + // lookup with name + val q: DBObject = MongoDBObject(KEY -> name) - private def nullSafeFindOne(name: String): Option[DBObject] = { - val o = new BasicDBObject - o.put(KEY, name) - coll.findOneNS(o) - } + db.safely { db => + coll.findOne(q) match { + // exists : need to update + case Some(dbo) => + dbo += ((REF, element)) + coll.update(q, dbo, true, false) - def insertRefStorageFor(name: String, element: AnyRef) = { - nullSafeFindOne(name) match { - case None => - case Some(dbo) => { - val q = new BasicDBObject - q.put(KEY, name) - coll.remove(q) + // not found : make one + case None => + val builder = MongoDBObject.newBuilder + builder += KEY -> name + builder += REF -> element + coll += builder.result.asDBObject } } - coll.insert( - new BasicDBObject() - .append(KEY, name) - .append(VALUE, serializer.out(element))) } - def getRefStorageFor(name: String): Option[AnyRef] = { - nullSafeFindOne(name) match { - case None => None - case Some(dbo) => - Some(serializer.in[AnyRef](dbo.get(VALUE).asInstanceOf[Array[Byte]])) - } + def getRefStorageFor(name: String): Option[Array[Byte]] = queryFor(name) { (q, dbo) => + dbo.map { d => + d.getAs[Array[Byte]](REF) + }.getOrElse(None) } } diff --git a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoPersistentActorSpec.scala b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoPersistentActorSpec.scala index 1acc9ee97d..01f735b254 100644 --- a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoPersistentActorSpec.scala +++ b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoPersistentActorSpec.scala @@ -1,32 +1,19 @@ package se.scalablesolutions.akka.persistence.mongo -import org.junit.{Test, Before} -import org.junit.Assert._ -import org.scalatest.junit.JUnitSuite - -import _root_.dispatch.json.{JsNumber, JsValue} -import _root_.dispatch.json.Js._ +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._ -/** - * A persistent actor based on MongoDB storage. - *

- * Demonstrates a bank account operation consisting of messages that: - *

  • checks balance Balance
  • - *
  • debits amountDebit
  • - *
  • debits multiple amountsMultiDebit
  • - *
  • credits amountCredit
  • - *

    - * Needs a running Mongo server. - * @author Debasish Ghosh - */ case class Balance(accountNo: String) -case class Debit(accountNo: String, amount: BigInt, failer: ActorRef) -case class MultiDebit(accountNo: String, amounts: List[BigInt], failer: ActorRef) -case class Credit(accountNo: String, amount: BigInt) +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 @@ -35,63 +22,65 @@ class BankAccountActor extends Transactor { private lazy val accountState = MongoStorage.newMap private lazy val txnLog = MongoStorage.newVector + import sjson.json.DefaultProtocol._ + import sjson.json.JsonSerialization._ + def receive: Receive = { // check balance case Balance(accountNo) => - txnLog.add("Balance:" + accountNo) - self.reply(accountState.get(accountNo).get) + 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) + 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" - val m: BigInt = - accountState.get(accountNo) match { - case Some(JsNumber(n)) => - BigInt(n.asInstanceOf[BigDecimal].intValue) - case None => 0 - } - accountState.put(accountNo, (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) => - txnLog.add("MultiDebit:" + accountNo + " " + amounts.map(_.intValue).foldLeft(0)(_ + _)) + val sum = amounts.foldRight(0)(_ + _) + txnLog.add(("MultiDebit:" + accountNo + " " + sum).getBytes) - val m: BigInt = - accountState.get(accountNo) match { - case Some(JsNumber(n)) => BigInt(n.toString) - case None => 0 + 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" } - var bal: BigInt = 0 - amounts.foreach {amount => - bal = bal + amount - accountState.put(accountNo, (m - bal)) - } - if (bal > m) failer !! "Failure" - self.reply(m - bal) + + self.reply(m - sum) // credit amount case Credit(accountNo, amount) => - txnLog.add("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)) - val m: BigInt = - accountState.get(accountNo) match { - case Some(JsNumber(n)) => - BigInt(n.asInstanceOf[BigDecimal].intValue) - case None => 0 - } - accountState.put(accountNo, (m + amount)) self.reply(m + amount) case LogSize => - self.reply(txnLog.length.asInstanceOf[AnyRef]) + self.reply(txnLog.length) case Log(start, finish) => - self.reply(txnLog.slice(start, finish)) + self.reply(txnLog.slice(start, finish).map(new String(_))) } } @@ -102,82 +91,71 @@ class BankAccountActor extends Transactor { } } -class MongoPersistentActorSpec extends JUnitSuite { - @Test - def testSuccessfulDebit = { - val bactor = actorOf[BankAccountActor] - bactor.start - val failer = actorOf[PersistentFailerActor] - failer.start - bactor !! Credit("a-123", 5000) - bactor !! Debit("a-123", 3000, failer) +@RunWith(classOf[JUnitRunner]) +class MongoPersistentActorSpec extends + Spec with + ShouldMatchers with + BeforeAndAfterEach { - val JsNumber(b) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue] - assertEquals(BigInt(2000), BigInt(b.intValue)) - - bactor !! Credit("a-123", 7000) - - val JsNumber(b1) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue] - assertEquals(BigInt(9000), BigInt(b1.intValue)) - - bactor !! Debit("a-123", 8000, failer) - - val JsNumber(b2) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue] - assertEquals(BigInt(1000), BigInt(b2.intValue)) - - assert(7 == (bactor !! LogSize).get.asInstanceOf[Int]) - - import scala.collection.mutable.ArrayBuffer - assert((bactor !! Log(0, 7)).get.asInstanceOf[ArrayBuffer[String]].size == 7) - assert((bactor !! Log(0, 0)).get.asInstanceOf[ArrayBuffer[String]].size == 0) - assert((bactor !! Log(1, 2)).get.asInstanceOf[ArrayBuffer[String]].size == 1) - assert((bactor !! Log(6, 7)).get.asInstanceOf[ArrayBuffer[String]].size == 1) - assert((bactor !! Log(0, 1)).get.asInstanceOf[ArrayBuffer[String]].size == 1) + override def beforeEach { + MongoStorageBackend.drop } - @Test - def testUnsuccessfulDebit = { - val bactor = actorOf[BankAccountActor] - bactor.start - bactor !! Credit("a-123", 5000) - - val JsNumber(b) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue] - assertEquals(BigInt(5000), BigInt(b.intValue)) - - val failer = actorOf[PersistentFailerActor] - failer.start - try { - bactor !! Debit("a-123", 7000, failer) - fail("should throw exception") - } catch { case e: RuntimeException => {}} - - val JsNumber(b1) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue] - assertEquals(BigInt(5000), BigInt(b1.intValue)) - - // should not count the failed one - assert(3 == (bactor !! LogSize).get.asInstanceOf[Int]) + override def afterEach { + MongoStorageBackend.drop } - @Test - def testUnsuccessfulMultiDebit = { - val bactor = actorOf[BankAccountActor] - bactor.start - bactor !! Credit("a-123", 5000) + describe("successful debit") { + it("should debit successfully") { + val bactor = actorOf[BankAccountActor] + bactor.start + val failer = actorOf[PersistentFailerActor] + failer.start + bactor !! Credit("a-123", 5000) + bactor !! Debit("a-123", 3000, failer) - val JsNumber(b) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue] - assertEquals(BigInt(5000), BigInt(b.intValue)) + (bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(2000) - val failer = actorOf[PersistentFailerActor] - failer.start - try { - bactor !! MultiDebit("a-123", List(500, 2000, 1000, 3000), failer) - fail("should throw exception") - } catch { case e: RuntimeException => {}} + bactor !! Credit("a-123", 7000) + (bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(9000) - val JsNumber(b1) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue] - assertEquals(BigInt(5000), BigInt(b1.intValue)) + bactor !! Debit("a-123", 8000, failer) + (bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(1000) - // should not count the failed one - assert(3 == (bactor !! LogSize).get.asInstanceOf[Int]) + (bactor !! LogSize).get.asInstanceOf[Int] should equal(7) + (bactor !! Log(0, 7)).get.asInstanceOf[Iterable[String]].size should equal(7) + } + } + + describe("unsuccessful debit") { + it("debit should fail") { + val bactor = actorOf[BankAccountActor] + bactor.start + val failer = actorOf[PersistentFailerActor] + failer.start + bactor !! Credit("a-123", 5000) + (bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(5000) + evaluating { + bactor !! Debit("a-123", 7000, failer) + } should produce [Exception] + (bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(5000) + (bactor !! LogSize).get.asInstanceOf[Int] should equal(3) + } + } + + describe("unsuccessful multidebit") { + it("multidebit should fail") { + val bactor = actorOf[BankAccountActor] + bactor.start + val failer = actorOf[PersistentFailerActor] + failer.start + bactor !! Credit("a-123", 5000) + (bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(5000) + evaluating { + bactor !! MultiDebit("a-123", List(1000, 2000, 4000), failer) + } should produce [Exception] + (bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(5000) + (bactor !! LogSize).get.asInstanceOf[Int] should equal(3) + } } } diff --git a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoStorageSpec.scala b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoStorageSpec.scala index e518b28d66..e9576cc152 100644 --- a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoStorageSpec.scala +++ b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoStorageSpec.scala @@ -1,364 +1,158 @@ package se.scalablesolutions.akka.persistence.mongo -import org.junit.{Test, Before} -import org.junit.Assert._ -import org.scalatest.junit.JUnitSuite -import _root_.dispatch.json._ -import _root_.dispatch.json.Js._ +import org.scalatest.Spec +import org.scalatest.matchers.ShouldMatchers +import org.scalatest.BeforeAndAfterEach +import org.scalatest.junit.JUnitRunner +import org.junit.runner.RunWith import java.util.NoSuchElementException -@scala.reflect.BeanInfo case class Foo(no: Int, name: String) -class MongoStorageSpec extends JUnitSuite { +@RunWith(classOf[JUnitRunner]) +class MongoStorageSpec extends + Spec with + ShouldMatchers with + BeforeAndAfterEach { - val changeSetV = new scala.collection.mutable.ArrayBuffer[AnyRef] - val changeSetM = new scala.collection.mutable.HashMap[AnyRef, AnyRef] - - @Before def initialize() = { - MongoStorageBackend.coll.drop + override def beforeEach { + MongoStorageBackend.drop } - @Test - def testVectorInsertForTransactionId = { - changeSetV += "debasish" // string - changeSetV += List(1, 2, 3) // Scala List - changeSetV += List(100, 200) - MongoStorageBackend.insertVectorStorageEntriesFor("U-A1", changeSetV.toList) - assertEquals( - 3, - MongoStorageBackend.getVectorStorageSizeFor("U-A1")) - changeSetV.clear - - // changeSetV should be reinitialized - changeSetV += List(12, 23, 45) - changeSetV += "maulindu" - MongoStorageBackend.insertVectorStorageEntriesFor("U-A1", changeSetV.toList) - assertEquals( - 5, - MongoStorageBackend.getVectorStorageSizeFor("U-A1")) - - // add more to the same changeSetV - changeSetV += "ramanendu" - changeSetV += Map(1 -> "dg", 2 -> "mc") - - // add for a diff transaction - MongoStorageBackend.insertVectorStorageEntriesFor("U-A2", changeSetV.toList) - assertEquals( - 4, - MongoStorageBackend.getVectorStorageSizeFor("U-A2")) - - // previous transaction change set should remain same - assertEquals( - 5, - MongoStorageBackend.getVectorStorageSizeFor("U-A1")) - - // test single element entry - MongoStorageBackend.insertVectorStorageEntryFor("U-A1", Map(1->1, 2->4, 3->9)) - assertEquals( - 6, - MongoStorageBackend.getVectorStorageSizeFor("U-A1")) + override def afterEach { + MongoStorageBackend.drop } - @Test - def testVectorFetchForKeys = { + describe("persistent maps") { + it("should insert with single key and value") { + import MongoStorageBackend._ - // initially everything 0 - assertEquals( - 0, - MongoStorageBackend.getVectorStorageSizeFor("U-A2")) - - assertEquals( - 0, - MongoStorageBackend.getVectorStorageSizeFor("U-A1")) - - // get some stuff - changeSetV += "debasish" - changeSetV += List(BigDecimal(12), BigDecimal(13), BigDecimal(14)) - MongoStorageBackend.insertVectorStorageEntriesFor("U-A1", changeSetV.toList) - - assertEquals( - 2, - MongoStorageBackend.getVectorStorageSizeFor("U-A1")) - - val JsString(str) = MongoStorageBackend.getVectorStorageEntryFor("U-A1", 0).asInstanceOf[JsString] - assertEquals("debasish", str) - - val l = MongoStorageBackend.getVectorStorageEntryFor("U-A1", 1).asInstanceOf[JsValue] - val num_list = list ! num - val num_list(l0) = l - assertEquals(List(12, 13, 14), l0) - - changeSetV.clear - changeSetV += Map(1->1, 2->4, 3->9) - changeSetV += BigInt(2310) - changeSetV += List(100, 200, 300) - MongoStorageBackend.insertVectorStorageEntriesFor("U-A1", changeSetV.toList) - - assertEquals( - 5, - MongoStorageBackend.getVectorStorageSizeFor("U-A1")) - - val r = - MongoStorageBackend.getVectorStorageRangeFor("U-A1", Some(1), None, 3) - - assertEquals(3, r.size) - val lr = r(0).asInstanceOf[JsValue] - val num_list(l1) = lr - assertEquals(List(12, 13, 14), l1) - } - - @Test - def testVectorFetchForNonExistentKeys = { - try { - MongoStorageBackend.getVectorStorageEntryFor("U-A1", 1) - fail("should throw an exception") - } catch {case e: NoSuchElementException => {}} - - try { - MongoStorageBackend.getVectorStorageRangeFor("U-A1", Some(2), None, 12) - fail("should throw an exception") - } catch {case e: NoSuchElementException => {}} - } - - @Test - def testVectorUpdateForTransactionId = { - import MongoStorageBackend._ - - changeSetV += "debasish" // string - changeSetV += List(1, 2, 3) // Scala List - changeSetV += List(100, 200) - - insertVectorStorageEntriesFor("U-A1", changeSetV.toList) - assertEquals(3, getVectorStorageSizeFor("U-A1")) - updateVectorStorageEntryFor("U-A1", 0, "maulindu") - val JsString(str) = getVectorStorageEntryFor("U-A1", 0).asInstanceOf[JsString] - assertEquals("maulindu", str) - - updateVectorStorageEntryFor("U-A1", 1, Map("1"->"dg", "2"->"mc")) - val JsObject(m) = getVectorStorageEntryFor("U-A1", 1).asInstanceOf[JsObject] - assertEquals(m.keySet.size, 2) - } - - @Test - def testMapInsertForTransactionId = { - fillMap - - // add some more to changeSet - changeSetM += "5" -> Foo(12, "dg") - changeSetM += "6" -> java.util.Calendar.getInstance.getTime - - // insert all into Mongo - MongoStorageBackend.insertMapStorageEntriesFor("U-M1", changeSetM.toList) - assertEquals( - 6, - MongoStorageBackend.getMapStorageSizeFor("U-M1")) - - // individual insert api - MongoStorageBackend.insertMapStorageEntryFor("U-M1", "7", "akka") - MongoStorageBackend.insertMapStorageEntryFor("U-M1", "8", List(23, 25)) - assertEquals( - 8, - MongoStorageBackend.getMapStorageSizeFor("U-M1")) - - // add the same changeSet for another transaction - MongoStorageBackend.insertMapStorageEntriesFor("U-M2", changeSetM.toList) - assertEquals( - 6, - MongoStorageBackend.getMapStorageSizeFor("U-M2")) - - // the first transaction should remain the same - assertEquals( - 8, - MongoStorageBackend.getMapStorageSizeFor("U-M1")) - changeSetM.clear - } - - @Test - def testMapContents = { - fillMap - MongoStorageBackend.insertMapStorageEntriesFor("U-M1", changeSetM.toList) - MongoStorageBackend.getMapStorageEntryFor("U-M1", "2") match { - case Some(x) => { - val JsString(str) = x.asInstanceOf[JsValue] - assertEquals("peter", str) - } - case None => fail("should fetch peter") - } - MongoStorageBackend.getMapStorageEntryFor("U-M1", "4") match { - case Some(x) => { - val num_list = list ! num - val num_list(l0) = x.asInstanceOf[JsValue] - assertEquals(3, l0.size) - } - case None => fail("should fetch list") - } - MongoStorageBackend.getMapStorageEntryFor("U-M1", "3") match { - case Some(x) => { - val num_list = list ! num - val num_list(l0) = x.asInstanceOf[JsValue] - assertEquals(2, l0.size) - } - case None => fail("should fetch list") + insertMapStorageEntryFor("t1", "odersky".getBytes, "scala".getBytes) + insertMapStorageEntryFor("t1", "gosling".getBytes, "java".getBytes) + insertMapStorageEntryFor("t1", "stroustrup".getBytes, "c++".getBytes) + getMapStorageSizeFor("t1") should equal(3) + new String(getMapStorageEntryFor("t1", "odersky".getBytes).get) should equal("scala") + new String(getMapStorageEntryFor("t1", "gosling".getBytes).get) should equal("java") + new String(getMapStorageEntryFor("t1", "stroustrup".getBytes).get) should equal("c++") + getMapStorageEntryFor("t1", "torvalds".getBytes) should equal(None) } - // get the entire map - val l: List[Tuple2[AnyRef, AnyRef]] = - MongoStorageBackend.getMapStorageFor("U-M1") + it("should insert with multiple keys and values") { + import MongoStorageBackend._ - assertEquals(4, l.size) - assertTrue(l.map(_._1).contains("1")) - assertTrue(l.map(_._1).contains("2")) - assertTrue(l.map(_._1).contains("3")) - assertTrue(l.map(_._1).contains("4")) + val l = List(("stroustrup", "c++"), ("odersky", "scala"), ("gosling", "java")) + insertMapStorageEntriesFor("t1", l.map { case (k, v) => (k.getBytes, v.getBytes) }) + getMapStorageSizeFor("t1") should equal(3) + new String(getMapStorageEntryFor("t1", "stroustrup".getBytes).get) should equal("c++") + new String(getMapStorageEntryFor("t1", "gosling".getBytes).get) should equal("java") + new String(getMapStorageEntryFor("t1", "odersky".getBytes).get) should equal("scala") + getMapStorageEntryFor("t1", "torvalds".getBytes) should equal(None) - val JsString(str) = l.filter(_._1 == "2").head._2 - assertEquals(str, "peter") + getMapStorageEntryFor("t2", "torvalds".getBytes) should equal(None) - // trying to fetch for a non-existent transaction will throw - try { - MongoStorageBackend.getMapStorageFor("U-M2") - fail("should throw an exception") - } catch {case e: NoSuchElementException => {}} + getMapStorageFor("t1").map { case (k, v) => (new String(k), new String(v)) } should equal (l) - changeSetM.clear - } + removeMapStorageFor("t1", "gosling".getBytes) + getMapStorageSizeFor("t1") should equal(2) - @Test - def testMapContentsByRange = { - fillMap - changeSetM += "5" -> Map(1 -> "dg", 2 -> "mc") - MongoStorageBackend.insertMapStorageEntriesFor("U-M1", changeSetM.toList) - - // specify start and count - val l: List[Tuple2[AnyRef, AnyRef]] = - MongoStorageBackend.getMapStorageRangeFor( - "U-M1", Some(Integer.valueOf(2)), None, 3) - - assertEquals(3, l.size) - assertEquals("3", l(0)._1.asInstanceOf[String]) - val lst = l(0)._2.asInstanceOf[JsValue] - val num_list = list ! num - val num_list(l0) = lst - assertEquals(List(100, 200), l0) - assertEquals("4", l(1)._1.asInstanceOf[String]) - val ls = l(1)._2.asInstanceOf[JsValue] - val num_list(l1) = ls - assertEquals(List(10, 20, 30), l1) - - // specify start, finish and count where finish - start == count - assertEquals(3, - MongoStorageBackend.getMapStorageRangeFor( - "U-M1", Some(Integer.valueOf(2)), Some(Integer.valueOf(5)), 3).size) - - // specify start, finish and count where finish - start > count - assertEquals(3, - MongoStorageBackend.getMapStorageRangeFor( - "U-M1", Some(Integer.valueOf(2)), Some(Integer.valueOf(9)), 3).size) - - // do not specify start or finish - assertEquals(3, - MongoStorageBackend.getMapStorageRangeFor( - "U-M1", None, None, 3).size) - - // specify finish and count - assertEquals(3, - MongoStorageBackend.getMapStorageRangeFor( - "U-M1", None, Some(Integer.valueOf(3)), 3).size) - - // specify start, finish and count where finish < start - assertEquals(3, - MongoStorageBackend.getMapStorageRangeFor( - "U-M1", Some(Integer.valueOf(2)), Some(Integer.valueOf(1)), 3).size) - - changeSetM.clear - } - - @Test - def testMapStorageRemove = { - fillMap - changeSetM += "5" -> Map(1 -> "dg", 2 -> "mc") - - MongoStorageBackend.insertMapStorageEntriesFor("U-M1", changeSetM.toList) - assertEquals(5, - MongoStorageBackend.getMapStorageSizeFor("U-M1")) - - // remove key "3" - MongoStorageBackend.removeMapStorageFor("U-M1", "3") - assertEquals(4, - MongoStorageBackend.getMapStorageSizeFor("U-M1")) - - try { - MongoStorageBackend.getMapStorageEntryFor("U-M1", "3") - fail("should throw exception") - } catch { case e => {}} - - // remove key "4" - MongoStorageBackend.removeMapStorageFor("U-M1", "4") - assertEquals(3, - MongoStorageBackend.getMapStorageSizeFor("U-M1")) - - // remove key "2" - MongoStorageBackend.removeMapStorageFor("U-M1", "2") - assertEquals(2, - MongoStorageBackend.getMapStorageSizeFor("U-M1")) - - // remove the whole stuff - MongoStorageBackend.removeMapStorageFor("U-M1") - - try { - MongoStorageBackend.getMapStorageFor("U-M1") - fail("should throw exception") - } catch { case e: NoSuchElementException => {}} - - changeSetM.clear - } - - private def fillMap = { - changeSetM += "1" -> "john" - changeSetM += "2" -> "peter" - changeSetM += "3" -> List(100, 200) - changeSetM += "4" -> List(10, 20, 30) - changeSetM - } - - @Test - def testRefStorage = { - MongoStorageBackend.getRefStorageFor("U-R1") match { - case None => - case Some(o) => fail("should be None") + removeMapStorageFor("t1") + getMapStorageSizeFor("t1") should equal(0) } - val m = Map("1"->1, "2"->4, "3"->9) - MongoStorageBackend.insertRefStorageFor("U-R1", m) - MongoStorageBackend.getRefStorageFor("U-R1") match { - case None => fail("should not be empty") - case Some(r) => { - val a = r.asInstanceOf[JsValue] - val m1 = Symbol("1") ? num - val m2 = Symbol("2") ? num - val m3 = Symbol("3") ? num + it("should do proper range queries") { + import MongoStorageBackend._ + val l = List( + ("bjarne stroustrup", "c++"), + ("martin odersky", "scala"), + ("james gosling", "java"), + ("yukihiro matsumoto", "ruby"), + ("slava pestov", "factor"), + ("rich hickey", "clojure"), + ("ola bini", "ioke"), + ("dennis ritchie", "c"), + ("larry wall", "perl"), + ("guido van rossum", "python"), + ("james strachan", "groovy")) + insertMapStorageEntriesFor("t1", l.map { case (k, v) => (k.getBytes, v.getBytes) }) + getMapStorageSizeFor("t1") should equal(l.size) + getMapStorageRangeFor("t1", None, None, 100).map { case (k, v) => (new String(k), new String(v)) } should equal(l.sortWith(_._1 < _._1)) + getMapStorageRangeFor("t1", None, None, 5).map { case (k, v) => (new String(k), new String(v)) }.size should equal(5) + } + } - val m1(n1) = a - val m2(n2) = a - val m3(n3) = a + describe("persistent vectors") { + it("should insert a single value") { + import MongoStorageBackend._ - assertEquals(n1, 1) - assertEquals(n2, 4) - assertEquals(n3, 9) - } + insertVectorStorageEntryFor("t1", "martin odersky".getBytes) + insertVectorStorageEntryFor("t1", "james gosling".getBytes) + new String(getVectorStorageEntryFor("t1", 0)) should equal("james gosling") + new String(getVectorStorageEntryFor("t1", 1)) should equal("martin odersky") } - // insert another one - // the previous one should be replaced - val b = List("100", "jonas") - MongoStorageBackend.insertRefStorageFor("U-R1", b) - MongoStorageBackend.getRefStorageFor("U-R1") match { - case None => fail("should not be empty") - case Some(r) => { - val a = r.asInstanceOf[JsValue] - val str_lst = list ! str - val str_lst(l) = a - assertEquals(b, l) - } + it("should insert multiple values") { + import MongoStorageBackend._ + + insertVectorStorageEntryFor("t1", "martin odersky".getBytes) + insertVectorStorageEntryFor("t1", "james gosling".getBytes) + insertVectorStorageEntriesFor("t1", List("ola bini".getBytes, "james strachan".getBytes, "dennis ritchie".getBytes)) + new String(getVectorStorageEntryFor("t1", 0)) should equal("ola bini") + new String(getVectorStorageEntryFor("t1", 1)) should equal("james strachan") + new String(getVectorStorageEntryFor("t1", 2)) should equal("dennis ritchie") + new String(getVectorStorageEntryFor("t1", 3)) should equal("james gosling") + new String(getVectorStorageEntryFor("t1", 4)) should equal("martin odersky") + } + + it("should fetch a range of values") { + import MongoStorageBackend._ + + insertVectorStorageEntryFor("t1", "martin odersky".getBytes) + insertVectorStorageEntryFor("t1", "james gosling".getBytes) + getVectorStorageSizeFor("t1") should equal(2) + insertVectorStorageEntriesFor("t1", List("ola bini".getBytes, "james strachan".getBytes, "dennis ritchie".getBytes)) + getVectorStorageRangeFor("t1", None, None, 100).map(new String(_)) should equal(List("ola bini", "james strachan", "dennis ritchie", "james gosling", "martin odersky")) + getVectorStorageRangeFor("t1", Some(0), Some(5), 100).map(new String(_)) should equal(List("ola bini", "james strachan", "dennis ritchie", "james gosling", "martin odersky")) + getVectorStorageRangeFor("t1", Some(2), Some(5), 100).map(new String(_)) should equal(List("dennis ritchie", "james gosling", "martin odersky")) + + getVectorStorageSizeFor("t1") should equal(5) + } + + it("should insert and query complex structures") { + import MongoStorageBackend._ + import sjson.json.DefaultProtocol._ + import sjson.json.JsonSerialization._ + + // a list[AnyRef] should be added successfully + val l = List("ola bini".getBytes, tobinary(List(100, 200, 300)), tobinary(List(1, 2, 3))) + + // for id = t1 + insertVectorStorageEntriesFor("t1", l) + new String(getVectorStorageEntryFor("t1", 0)) should equal("ola bini") + frombinary[List[Int]](getVectorStorageEntryFor("t1", 1)) should equal(List(100, 200, 300)) + frombinary[List[Int]](getVectorStorageEntryFor("t1", 2)) should equal(List(1, 2, 3)) + + getVectorStorageSizeFor("t1") should equal(3) + + // some more for id = t1 + val m = List(tobinary(Map(1 -> "dg", 2 -> "mc", 3 -> "nd")), tobinary(List("martin odersky", "james gosling"))) + insertVectorStorageEntriesFor("t1", m) + + // size should add up + getVectorStorageSizeFor("t1") should equal(5) + + // now for a diff id + insertVectorStorageEntriesFor("t2", l) + getVectorStorageSizeFor("t2") should equal(3) + } + } + + describe("persistent refs") { + it("should insert a ref") { + import MongoStorageBackend._ + + insertRefStorageFor("t1", "martin odersky".getBytes) + new String(getRefStorageFor("t1").get) should equal("martin odersky") + insertRefStorageFor("t1", "james gosling".getBytes) + new String(getRefStorageFor("t1").get) should equal("james gosling") + getRefStorageFor("t2") should equal(None) } } } diff --git a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala new file mode 100644 index 0000000000..3b160c8c50 --- /dev/null +++ b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala @@ -0,0 +1,347 @@ +package se.scalablesolutions.akka.persistence.mongo + +import org.scalatest.Spec +import org.scalatest.matchers.ShouldMatchers +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} +import org.scalatest.junit.JUnitRunner +import org.junit.runner.RunWith + +import se.scalablesolutions.akka.actor.{Actor, ActorRef} +import se.scalablesolutions.akka.config.OneForOneStrategy +import Actor._ +import se.scalablesolutions.akka.stm.global._ +import se.scalablesolutions.akka.config.ScalaConfig._ +import se.scalablesolutions.akka.util.Logging + +import MongoStorageBackend._ + +case class GET(k: String) +case class SET(k: String, v: String) +case class REM(k: String) +case class CONTAINS(k: String) +case object MAP_SIZE +case class MSET(kvs: List[(String, String)]) +case class REMOVE_AFTER_PUT(kvsToAdd: List[(String, String)], ksToRem: List[String]) +case class CLEAR_AFTER_PUT(kvsToAdd: List[(String, String)]) +case class PUT_WITH_SLICE(kvsToAdd: List[(String, String)], start: String, cnt: Int) +case class PUT_REM_WITH_SLICE(kvsToAdd: List[(String, String)], ksToRem: List[String], start: String, cnt: Int) + +case class VADD(v: String) +case class VUPD(i: Int, v: String) +case class VUPD_AND_ABORT(i: Int, v: String) +case class VGET(i: Int) +case object VSIZE +case class VGET_AFTER_VADD(vsToAdd: List[String], isToFetch: List[Int]) +case class VADD_WITH_SLICE(vsToAdd: List[String], start: Int, cnt: Int) + +object Storage { + class MongoSampleMapStorage extends Actor { + self.lifeCycle = Some(LifeCycle(Permanent)) + val FOO_MAP = "akka.sample.map" + + private var fooMap = atomic { MongoStorage.getMap(FOO_MAP) } + + def receive = { + case SET(k, v) => + atomic { + fooMap += (k.getBytes, v.getBytes) + } + self.reply((k, v)) + + case GET(k) => + val v = atomic { + fooMap.get(k.getBytes).map(new String(_)).getOrElse(k + " Not found") + } + self.reply(v) + + case REM(k) => + val v = atomic { + fooMap -= k.getBytes + } + self.reply(k) + + case CONTAINS(k) => + val v = atomic { + fooMap contains k.getBytes + } + self.reply(v) + + case MAP_SIZE => + val v = atomic { + fooMap.size + } + self.reply(v) + + case MSET(kvs) => atomic { + kvs.foreach {kv => fooMap += (kv._1.getBytes, kv._2.getBytes) } + } + self.reply(kvs.size) + + case REMOVE_AFTER_PUT(kvs2add, ks2rem) => atomic { + kvs2add.foreach {kv => + fooMap += (kv._1.getBytes, kv._2.getBytes) + } + + ks2rem.foreach {k => + fooMap -= k.getBytes + }} + self.reply(fooMap.size) + + case CLEAR_AFTER_PUT(kvs2add) => atomic { + kvs2add.foreach {kv => + fooMap += (kv._1.getBytes, kv._2.getBytes) + } + fooMap.clear + } + self.reply(true) + + case PUT_WITH_SLICE(kvs2add, from, cnt) => + val v = atomic { + kvs2add.foreach {kv => + fooMap += (kv._1.getBytes, kv._2.getBytes) + } + fooMap.slice(Some(from.getBytes), cnt) + } + self.reply(v: List[(Array[Byte], Array[Byte])]) + + case PUT_REM_WITH_SLICE(kvs2add, ks2rem, from, cnt) => + val v = atomic { + kvs2add.foreach {kv => + fooMap += (kv._1.getBytes, kv._2.getBytes) + } + ks2rem.foreach {k => + fooMap -= k.getBytes + } + fooMap.slice(Some(from.getBytes), cnt) + } + self.reply(v: List[(Array[Byte], Array[Byte])]) + } + } + + class MongoSampleVectorStorage extends Actor { + self.lifeCycle = Some(LifeCycle(Permanent)) + val FOO_VECTOR = "akka.sample.vector" + + private var fooVector = atomic { MongoStorage.getVector(FOO_VECTOR) } + + def receive = { + case VADD(v) => + val size = + atomic { + fooVector + v.getBytes + fooVector length + } + self.reply(size) + + case VGET(index) => + val ind = + atomic { + fooVector get index + } + self.reply(ind) + + case VGET_AFTER_VADD(vs, is) => + val els = + atomic { + vs.foreach(fooVector + _.getBytes) + (is.foldRight(List[Array[Byte]]())(fooVector.get(_) :: _)).map(new String(_)) + } + self.reply(els) + + case VUPD_AND_ABORT(index, value) => + val l = + atomic { + fooVector.update(index, value.getBytes) + // force fail + fooVector get 100 + } + self.reply(index) + + case VADD_WITH_SLICE(vs, s, c) => + val l = + atomic { + vs.foreach(fooVector + _.getBytes) + fooVector.slice(Some(s), None, c) + } + self.reply(l.map(new String(_))) + } + } +} + +import Storage._ + +@RunWith(classOf[JUnitRunner]) +class MongoTicket343Spec extends + Spec with + ShouldMatchers with + BeforeAndAfterAll with + BeforeAndAfterEach { + + + override def beforeAll { + MongoStorageBackend.drop + println("** destroyed database") + } + + override def beforeEach { + MongoStorageBackend.drop + println("** destroyed database") + } + + override def afterEach { + MongoStorageBackend.drop + println("** destroyed database") + } + + describe("Ticket 343 Issue #1") { + it("remove after put should work within the same transaction") { + val proc = actorOf[MongoSampleMapStorage] + proc.start + + (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft")) + (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft") + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1) + + (proc !! MSET(List(("dg", "1"), ("mc", "2"), ("nd", "3")))).getOrElse("Mset failed") should equal(3) + + (proc !! GET("dg")).getOrElse("Get failed") should equal("1") + (proc !! GET("mc")).getOrElse("Get failed") should equal("2") + (proc !! GET("nd")).getOrElse("Get failed") should equal("3") + + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(4) + + val add = List(("a", "1"), ("b", "2"), ("c", "3")) + val rem = List("a", "debasish") + (proc !! REMOVE_AFTER_PUT(add, rem)).getOrElse("REMOVE_AFTER_PUT failed") should equal(5) + + (proc !! GET("debasish")).getOrElse("debasish not found") should equal("debasish Not found") + (proc !! GET("a")).getOrElse("a not found") should equal("a Not found") + + (proc !! GET("b")).getOrElse("b not found") should equal("2") + + (proc !! CONTAINS("b")).getOrElse("b not found") should equal(true) + (proc !! CONTAINS("debasish")).getOrElse("debasish not found") should equal(false) + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(5) + proc.stop + } + } + + describe("Ticket 343 Issue #2") { + it("clear after put should work within the same transaction") { + val proc = actorOf[MongoSampleMapStorage] + proc.start + + (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft")) + (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft") + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1) + + val add = List(("a", "1"), ("b", "2"), ("c", "3")) + (proc !! CLEAR_AFTER_PUT(add)).getOrElse("CLEAR_AFTER_PUT failed") should equal(true) + + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1) + proc.stop + } + } + + describe("Ticket 343 Issue #3") { + it("map size should change after the transaction") { + val proc = actorOf[MongoSampleMapStorage] + proc.start + + (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft")) + (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft") + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1) + + (proc !! MSET(List(("dg", "1"), ("mc", "2"), ("nd", "3")))).getOrElse("Mset failed") should equal(3) + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(4) + + (proc !! GET("dg")).getOrElse("Get failed") should equal("1") + (proc !! GET("mc")).getOrElse("Get failed") should equal("2") + (proc !! GET("nd")).getOrElse("Get failed") should equal("3") + proc.stop + } + } + + describe("slice test") { + it("should pass") { + val proc = actorOf[MongoSampleMapStorage] + proc.start + + (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft")) + (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft") + // (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1) + + (proc !! MSET(List(("dg", "1"), ("mc", "2"), ("nd", "3")))).getOrElse("Mset failed") should equal(3) + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(4) + + (proc !! PUT_WITH_SLICE(List(("ec", "1"), ("tb", "2"), ("mc", "10")), "dg", 3)).get.asInstanceOf[List[(Array[Byte], Array[Byte])]].map { case (k, v) => (new String(k), new String(v)) } should equal(List(("dg", "1"), ("ec", "1"), ("mc", "10"))) + + (proc !! PUT_REM_WITH_SLICE(List(("fc", "1"), ("gb", "2"), ("xy", "10")), List("tb", "fc"), "dg", 5)).get.asInstanceOf[List[(Array[Byte], Array[Byte])]].map { case (k, v) => (new String(k), new String(v)) } should equal(List(("dg", "1"), ("ec", "1"), ("gb", "2"), ("mc", "10"), ("nd", "3"))) + proc.stop + } + } + + describe("Ticket 343 Issue #4") { + it("vector get should not ignore elements that were in vector before transaction") { + + val proc = actorOf[MongoSampleVectorStorage] + proc.start + + // add 4 elements in separate transactions + (proc !! VADD("debasish")).getOrElse("VADD failed") should equal(1) + (proc !! VADD("maulindu")).getOrElse("VADD failed") should equal(2) + (proc !! VADD("ramanendu")).getOrElse("VADD failed") should equal(3) + (proc !! VADD("nilanjan")).getOrElse("VADD failed") should equal(4) + + new String((proc !! VGET(0)).get.asInstanceOf[Array[Byte]] ) should equal("nilanjan") + new String((proc !! VGET(1)).get.asInstanceOf[Array[Byte]] ) should equal("ramanendu") + new String((proc !! VGET(2)).get.asInstanceOf[Array[Byte]] ) should equal("maulindu") + new String((proc !! VGET(3)).get.asInstanceOf[Array[Byte]] ) should equal("debasish") + + // now add 3 more and do gets in the same transaction + (proc !! VGET_AFTER_VADD(List("a", "b", "c"), List(0, 2, 4))).get.asInstanceOf[List[String]] should equal(List("c", "a", "ramanendu")) + proc.stop + } + } + + describe("Ticket 343 Issue #6") { + it("vector update should not ignore transaction") { + val proc = actorOf[MongoSampleVectorStorage] + proc.start + + // add 4 elements in separate transactions + (proc !! VADD("debasish")).getOrElse("VADD failed") should equal(1) + (proc !! VADD("maulindu")).getOrElse("VADD failed") should equal(2) + (proc !! VADD("ramanendu")).getOrElse("VADD failed") should equal(3) + (proc !! VADD("nilanjan")).getOrElse("VADD failed") should equal(4) + + evaluating { + (proc !! VUPD_AND_ABORT(0, "virat")).getOrElse("VUPD_AND_ABORT failed") + } should produce [Exception] + + // update aborts and hence values will remain unchanged + new String((proc !! VGET(0)).get.asInstanceOf[Array[Byte]] ) should equal("nilanjan") + proc.stop + } + } + + describe("Ticket 343 Issue #5") { + it("vector slice() should not ignore elements added in current transaction") { + val proc = actorOf[MongoSampleVectorStorage] + proc.start + + // add 4 elements in separate transactions + (proc !! VADD("debasish")).getOrElse("VADD failed") should equal(1) + (proc !! VADD("maulindu")).getOrElse("VADD failed") should equal(2) + (proc !! VADD("ramanendu")).getOrElse("VADD failed") should equal(3) + (proc !! VADD("nilanjan")).getOrElse("VADD failed") should equal(4) + + // slice with no new elements added in current transaction + (proc !! VADD_WITH_SLICE(List(), 2, 2)).getOrElse("VADD_WITH_SLICE failed") should equal(Vector("maulindu", "debasish")) + + // slice with new elements added in current transaction + (proc !! VADD_WITH_SLICE(List("a", "b", "c", "d"), 2, 2)).getOrElse("VADD_WITH_SLICE failed") should equal(Vector("b", "a")) + proc.stop + } + } +} diff --git a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorage.scala b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorage.scala index c92761beea..1eca775567 100644 --- a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorage.scala +++ b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorage.scala @@ -36,7 +36,7 @@ object RedisStorage extends Storage { * * @author Debasish Ghosh */ -class RedisPersistentMap(id: String) extends PersistentMap[Array[Byte], Array[Byte]] { +class RedisPersistentMap(id: String) extends PersistentMapBinary { val uuid = id val storage = RedisStorageBackend } diff --git a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala index eef60784a0..9fd3142019 100644 --- a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala +++ b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala @@ -11,34 +11,17 @@ import se.scalablesolutions.akka.config.Config.config import com.redis._ -trait Base64Encoder { - def encode(bytes: Array[Byte]): Array[Byte] - def decode(bytes: Array[Byte]): Array[Byte] -} - trait Base64StringEncoder { def byteArrayToString(bytes: Array[Byte]): String def stringToByteArray(str: String): Array[Byte] } -trait NullBase64 { - def encode(bytes: Array[Byte]): Array[Byte] = bytes - def decode(bytes: Array[Byte]): Array[Byte] = bytes -} - object CommonsCodec { import org.apache.commons.codec.binary.Base64 import org.apache.commons.codec.binary.Base64._ val b64 = new Base64(true) - trait CommonsCodecBase64 { - def encode(bytes: Array[Byte]): Array[Byte] = encodeBase64(bytes) - def decode(bytes: Array[Byte]): Array[Byte] = decodeBase64(bytes) - } - - object Base64Encoder extends Base64Encoder with CommonsCodecBase64 - trait CommonsCodecBase64StringEncoder { def byteArrayToString(bytes: Array[Byte]) = encodeBase64URLSafeString(bytes) def stringToByteArray(str: String) = b64.decode(str) @@ -48,7 +31,6 @@ object CommonsCodec { } import CommonsCodec._ -import CommonsCodec.Base64Encoder._ import CommonsCodec.Base64StringEncoder._ /** @@ -94,27 +76,7 @@ private [akka] object RedisStorageBackend extends /** * Map storage in Redis. *

    - * Maps are stored as key/value pairs in redis. Redis keys cannot contain spaces. But with - * our use case, the keys will be specified by the user. Hence we need to encode the key - * ourselves before sending to Redis. We use base64 encoding. - *

    - * Also since we are storing the key/value in the global namespace, we need to construct the - * key suitably so as to avoid namespace clash. The following strategy is used: - * - * Unique identifier for the map = T1 (say) - *

    -   * Map(
    -   *   "debasish.address" -> "kolkata, India",
    -   *   "debasish.company" -> "anshinsoft",
    -   *   "debasish.programming_language" -> "scala",
    -   * )
    - * will be stored as the following key-value pair in Redis: - * - * - * base64(T1):base64("debasish.address") -> "kolkata, India" - * base64(T1):base64("debasish.company") -> "anshinsoft" - * base64(T1):base64("debasish.programming_language") -> "scala" - * + * Maps are stored as key/value pairs in redis. */ def insertMapStorageEntryFor(name: String, key: Array[Byte], value: Array[Byte]): Unit = withErrorHandling { insertMapStorageEntriesFor(name, List((key, value))) @@ -134,12 +96,12 @@ private [akka] object RedisStorageBackend extends *
  • both parts of the key need to be based64 encoded since there can be spaces within each of them
  • */ private [this] def makeRedisKey(name: String, key: Array[Byte]): String = withErrorHandling { - "%s:%s".format(new String(encode(name.getBytes)), new String(encode(key))) + "%s:%s".format(name, new String(key)) } private [this] def makeKeyFromRedisKey(redisKey: String) = withErrorHandling { - val nk = redisKey.split(':').map{e: String => decode(e.getBytes)} - (nk(0), nk(1)) + val nk = redisKey.split(':') + (nk(0), nk(1).getBytes) } private [this] def mset(entries: List[(String, String)]): Unit = withErrorHandling { @@ -149,11 +111,11 @@ private [akka] object RedisStorageBackend extends } def removeMapStorageFor(name: String): Unit = withErrorHandling { - db.keys("%s:*".format(new String(encode(name.getBytes)))) match { + db.keys("%s:*".format(name)) match { case None => throw new NoSuchElementException(name + " not present") case Some(keys) => - keys.foreach(db.del(_)) + keys.foreach(k => db.del(k.get)) } } @@ -162,28 +124,22 @@ private [akka] object RedisStorageBackend extends } def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = withErrorHandling { - db.get(makeRedisKey(name, key)) match { - case None => - throw new NoSuchElementException(new String(key) + " not present") - case Some(s) => Some(stringToByteArray(s)) + db.get(makeRedisKey(name, key)) + .map(stringToByteArray(_)) + .orElse(throw new NoSuchElementException(new String(key) + " not present")) } - } def getMapStorageSizeFor(name: String): Int = withErrorHandling { - db.keys("%s:*".format(new String(encode(name.getBytes)))) match { - case None => 0 - case Some(keys) => - keys.length - } + db.keys("%s:*".format(name)).map(_.length).getOrElse(0) } def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = withErrorHandling { - db.keys("%s:*".format(new String(encode(name.getBytes)))) match { - case None => + db.keys("%s:*".format(name)) + .map { keys => + keys.map(key => (makeKeyFromRedisKey(key.get)._2, stringToByteArray(db.get(key.get).get))).toList + }.getOrElse { throw new NoSuchElementException(name + " not present") - case Some(keys) => - keys.map(key => (makeKeyFromRedisKey(key)._2, stringToByteArray(db.get(key).get))).toList - } + } } def getMapStorageRangeFor(name: String, start: Option[Array[Byte]], @@ -234,7 +190,7 @@ private [akka] object RedisStorageBackend extends } def insertVectorStorageEntryFor(name: String, element: Array[Byte]): Unit = withErrorHandling { - db.lpush(new String(encode(name.getBytes)), byteArrayToString(element)) + db.lpush(name, byteArrayToString(element)) } def insertVectorStorageEntriesFor(name: String, elements: List[Array[Byte]]): Unit = withErrorHandling { @@ -242,16 +198,15 @@ private [akka] object RedisStorageBackend extends } def updateVectorStorageEntryFor(name: String, index: Int, elem: Array[Byte]): Unit = withErrorHandling { - db.lset(new String(encode(name.getBytes)), index, byteArrayToString(elem)) + db.lset(name, index, byteArrayToString(elem)) } def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = withErrorHandling { - db.lindex(new String(encode(name.getBytes)), index) match { - case None => + db.lindex(name, index) + .map(stringToByteArray(_)) + .getOrElse { throw new NoSuchElementException(name + " does not have element at " + index) - case Some(e) => - stringToByteArray(e) - } + } } /** @@ -270,60 +225,51 @@ private [akka] object RedisStorageBackend extends else count if (s == 0 && cnt == 0) List() else - db.lrange(new String(encode(name.getBytes)), s, s + cnt - 1) match { + db.lrange(name, s, s + cnt - 1) match { case None => throw new NoSuchElementException(name + " does not have elements in the range specified") case Some(l) => - l map ( e => stringToByteArray(e.get)) + l map (e => stringToByteArray(e.get)) } } def getVectorStorageSizeFor(name: String): Int = withErrorHandling { - db.llen(new String(encode(name.getBytes))) match { - case None => - throw new NoSuchElementException(name + " not present") - case Some(l) => - l - } + db.llen(name).getOrElse { throw new NoSuchElementException(name + " not present") } } def insertRefStorageFor(name: String, element: Array[Byte]): Unit = withErrorHandling { - db.set(new String(encode(name.getBytes)), byteArrayToString(element)) + db.set(name, byteArrayToString(element)) } def insertRefStorageFor(name: String, element: String): Unit = withErrorHandling { - db.set(new String(encode(name.getBytes)), element) + db.set(name, element) } def getRefStorageFor(name: String): Option[Array[Byte]] = withErrorHandling { - db.get(new String(encode(name.getBytes))) match { - case None => + db.get(name) + .map(stringToByteArray(_)) + .orElse { throw new NoSuchElementException(name + " not present") - case Some(s) => Some(stringToByteArray(s)) - } + } } // add to the end of the queue - def enqueue(name: String, item: Array[Byte]): Boolean = withErrorHandling { - db.rpush(new String(encode(name.getBytes)), byteArrayToString(item)) + def enqueue(name: String, item: Array[Byte]): Option[Int] = withErrorHandling { + db.rpush(name, byteArrayToString(item)) } // pop from the front of the queue def dequeue(name: String): Option[Array[Byte]] = withErrorHandling { - db.lpop(new String(encode(name.getBytes))) match { - case None => + db.lpop(name) + .map(stringToByteArray(_)) + .orElse { throw new NoSuchElementException(name + " not present") - case Some(s) => Some(stringToByteArray(s)) - } + } } // get the size of the queue def size(name: String): Int = withErrorHandling { - db.llen(new String(encode(name.getBytes))) match { - case None => - throw new NoSuchElementException(name + " not present") - case Some(l) => l - } + db.llen(name).getOrElse { throw new NoSuchElementException(name + " not present") } } // return an array of items currently stored in the queue @@ -331,14 +277,14 @@ private [akka] object RedisStorageBackend extends def peek(name: String, start: Int, count: Int): List[Array[Byte]] = withErrorHandling { count match { case 1 => - db.lindex(new String(encode(name.getBytes)), start) match { + db.lindex(name, start) match { case None => throw new NoSuchElementException("No element at " + start) case Some(s) => List(stringToByteArray(s)) } case n => - db.lrange(new String(encode(name.getBytes)), start, start + count - 1) match { + db.lrange(name, start, start + count - 1) match { case None => throw new NoSuchElementException( "No element found between " + start + " and " + (start + count - 1)) @@ -350,61 +296,54 @@ private [akka] object RedisStorageBackend extends // completely delete the queue def remove(name: String): Boolean = withErrorHandling { - db.del(new String(encode(name.getBytes))) match { - case Some(1) => true - case _ => false - } + db.del(name).map { case 1 => true }.getOrElse(false) } // add item to sorted set identified by name def zadd(name: String, zscore: String, item: Array[Byte]): Boolean = withErrorHandling { - db.zadd(new String(encode(name.getBytes)), zscore, byteArrayToString(item)) match { - case Some(1) => true - case _ => false - } + db.zadd(name, zscore, byteArrayToString(item)) + .map { e => + e match { + case 1 => true + case _ => false + } + }.getOrElse(false) } // remove item from sorted set identified by name def zrem(name: String, item: Array[Byte]): Boolean = withErrorHandling { - db.zrem(new String(encode(name.getBytes)), byteArrayToString(item)) match { - case Some(1) => true - case _ => false - } + db.zrem(name, byteArrayToString(item)) + .map { e => + e match { + case 1 => true + case _ => false + } + }.getOrElse(false) } // cardinality of the set identified by name def zcard(name: String): Int = withErrorHandling { - db.zcard(new String(encode(name.getBytes))) match { - case None => - throw new NoSuchElementException(name + " not present") - case Some(l) => l - } + db.zcard(name).getOrElse { throw new NoSuchElementException(name + " not present") } } def zscore(name: String, item: Array[Byte]): Option[Float] = withErrorHandling { - db.zscore(new String(encode(name.getBytes)), byteArrayToString(item)) match { - case Some(s) => Some(s.toFloat) - case None => None - } + db.zscore(name, byteArrayToString(item)).map(_.toFloat) } def zrange(name: String, start: Int, end: Int): List[Array[Byte]] = withErrorHandling { - db.zrange(new String(encode(name.getBytes)), start.toString, end.toString, RedisClient.ASC, false) match { - case None => + db.zrange(name, start.toString, end.toString, RedisClient.ASC, false) + .map(_.map(e => stringToByteArray(e.get))) + .getOrElse { throw new NoSuchElementException(name + " not present") - case Some(s) => - s.map(e => stringToByteArray(e.get)) - } + } } def zrangeWithScore(name: String, start: Int, end: Int): List[(Array[Byte], Float)] = withErrorHandling { - db.zrangeWithScore( - new String(encode(name.getBytes)), start.toString, end.toString, RedisClient.ASC) match { - case None => - throw new NoSuchElementException(name + " not present") - case Some(l) => - l.map{ case (elem, score) => (stringToByteArray(elem.get), score.get.toFloat) } - } + db.zrangeWithScore(name, start.toString, end.toString, RedisClient.ASC) + .map(_.map { case (elem, score) => (stringToByteArray(elem.get), score.get.toFloat) }) + .getOrElse { + throw new NoSuchElementException(name + " not present") + } } def flushDB = withErrorHandling(db.flushdb) @@ -420,6 +359,7 @@ private [akka] object RedisStorageBackend extends case e: java.lang.NullPointerException => throw new StorageException("Could not connect to Redis server") case e => + e.printStackTrace throw new StorageException("Error in Redis: " + e.getMessage) } } diff --git a/akka-persistence/akka-persistence-redis/src/test/scala/RedisTicket343Spec.scala b/akka-persistence/akka-persistence-redis/src/test/scala/RedisTicket343Spec.scala new file mode 100644 index 0000000000..de236b9a5a --- /dev/null +++ b/akka-persistence/akka-persistence-redis/src/test/scala/RedisTicket343Spec.scala @@ -0,0 +1,351 @@ +package se.scalablesolutions.akka.persistence.redis + +import org.scalatest.Spec +import org.scalatest.matchers.ShouldMatchers +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} +import org.scalatest.junit.JUnitRunner +import org.junit.runner.RunWith + +import se.scalablesolutions.akka.actor.{Actor} +import se.scalablesolutions.akka.config.OneForOneStrategy +import Actor._ +import se.scalablesolutions.akka.persistence.common.PersistentVector +import se.scalablesolutions.akka.stm.global._ +import se.scalablesolutions.akka.config.ScalaConfig._ +import se.scalablesolutions.akka.util.Logging + +import RedisStorageBackend._ + +case class GET(k: String) +case class SET(k: String, v: String) +case class REM(k: String) +case class CONTAINS(k: String) +case object MAP_SIZE +case class MSET(kvs: List[(String, String)]) +case class REMOVE_AFTER_PUT(kvsToAdd: List[(String, String)], ksToRem: List[String]) +case class CLEAR_AFTER_PUT(kvsToAdd: List[(String, String)]) +case class PUT_WITH_SLICE(kvsToAdd: List[(String, String)], start: String, cnt: Int) +case class PUT_REM_WITH_SLICE(kvsToAdd: List[(String, String)], ksToRem: List[String], start: String, cnt: Int) + +case class VADD(v: String) +case class VUPD(i: Int, v: String) +case class VUPD_AND_ABORT(i: Int, v: String) +case class VGET(i: Int) +case object VSIZE +case class VGET_AFTER_VADD(vsToAdd: List[String], isToFetch: List[Int]) +case class VADD_WITH_SLICE(vsToAdd: List[String], start: Int, cnt: Int) + +object Storage { + class RedisSampleMapStorage extends Actor { + self.lifeCycle = Some(LifeCycle(Permanent)) + val FOO_MAP = "akka.sample.map" + + private var fooMap = atomic { RedisStorage.getMap(FOO_MAP) } + + def receive = { + case SET(k, v) => + atomic { + fooMap += (k.getBytes, v.getBytes) + } + self.reply((k, v)) + + case GET(k) => + val v = atomic { + fooMap.get(k.getBytes) + } + self.reply(v.collect {case byte => new String(byte)}.getOrElse(k + " Not found")) + + case REM(k) => + val v = atomic { + fooMap -= k.getBytes + } + self.reply(k) + + case CONTAINS(k) => + val v = atomic { + fooMap contains k.getBytes + } + self.reply(v) + + case MAP_SIZE => + val v = atomic { + fooMap.size + } + self.reply(v) + + case MSET(kvs) => + atomic { + kvs.foreach {kv => + fooMap += (kv._1.getBytes, kv._2.getBytes) + } + } + self.reply(kvs.size) + + case REMOVE_AFTER_PUT(kvs2add, ks2rem) => + val v = + atomic { + kvs2add.foreach {kv => + fooMap += (kv._1.getBytes, kv._2.getBytes) + } + + ks2rem.foreach {k => + fooMap -= k.getBytes + } + fooMap.size + } + self.reply(v) + + case CLEAR_AFTER_PUT(kvs2add) => + atomic { + kvs2add.foreach {kv => + fooMap += (kv._1.getBytes, kv._2.getBytes) + } + fooMap.clear + } + self.reply(true) + + case PUT_WITH_SLICE(kvs2add, from, cnt) => + val v = + atomic { + kvs2add.foreach {kv => + fooMap += (kv._1.getBytes, kv._2.getBytes) + } + fooMap.slice(Some(from.getBytes), cnt) + } + self.reply(v: List[(Array[Byte], Array[Byte])]) + + case PUT_REM_WITH_SLICE(kvs2add, ks2rem, from, cnt) => + val v = + atomic { + kvs2add.foreach {kv => + fooMap += (kv._1.getBytes, kv._2.getBytes) + } + ks2rem.foreach {k => + fooMap -= k.getBytes + } + fooMap.slice(Some(from.getBytes), cnt) + } + self.reply(v: List[(Array[Byte], Array[Byte])]) + } + } + + class RedisSampleVectorStorage extends Actor { + self.lifeCycle = Some(LifeCycle(Permanent)) + val FOO_VECTOR = "akka.sample.vector" + + private var fooVector = atomic { RedisStorage.getVector(FOO_VECTOR) } + + def receive = { + case VADD(v) => + val size = + atomic { + fooVector + v.getBytes + fooVector length + } + self.reply(size) + + case VGET(index) => + val ind = + atomic { + fooVector get index + } + self.reply(ind) + + case VGET_AFTER_VADD(vs, is) => + val els = + atomic { + vs.foreach(fooVector + _.getBytes) + (is.foldRight(List[Array[Byte]]())(fooVector.get(_) :: _)).map(new String(_)) + } + self.reply(els) + + case VUPD_AND_ABORT(index, value) => + val l = + atomic { + fooVector.update(index, value.getBytes) + // force fail + fooVector get 100 + } + self.reply(index) + + case VADD_WITH_SLICE(vs, s, c) => + val l = + atomic { + vs.foreach(fooVector + _.getBytes) + fooVector.slice(Some(s), None, c) + } + self.reply(l.map(new String(_))) + } + } +} + +import Storage._ + +@RunWith(classOf[JUnitRunner]) +class RedisTicket343Spec extends + Spec with + ShouldMatchers with + BeforeAndAfterAll with + BeforeAndAfterEach { + + override def beforeAll { + flushDB + println("** destroyed database") + } + + override def afterEach { + flushDB + println("** destroyed database") + } + + describe("Ticket 343 Issue #1") { + it("remove after put should work within the same transaction") { + val proc = actorOf[RedisSampleMapStorage] + proc.start + + (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft")) + (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft") + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1) + + (proc !! MSET(List(("dg", "1"), ("mc", "2"), ("nd", "3")))).getOrElse("Mset failed") should equal(3) + + (proc !! GET("dg")).getOrElse("Get failed") should equal("1") + (proc !! GET("mc")).getOrElse("Get failed") should equal("2") + (proc !! GET("nd")).getOrElse("Get failed") should equal("3") + + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(4) + + val add = List(("a", "1"), ("b", "2"), ("c", "3")) + val rem = List("a", "debasish") + (proc !! REMOVE_AFTER_PUT(add, rem)).getOrElse("REMOVE_AFTER_PUT failed") should equal(5) + + (proc !! GET("debasish")).getOrElse("debasish not found") should equal("debasish Not found") + (proc !! GET("a")).getOrElse("a not found") should equal("a Not found") + + (proc !! GET("b")).getOrElse("b not found") should equal("2") + + (proc !! CONTAINS("b")).getOrElse("b not found") should equal(true) + (proc !! CONTAINS("debasish")).getOrElse("debasish not found") should equal(false) + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(5) + proc.stop + } + } + + describe("Ticket 343 Issue #2") { + it("clear after put should work within the same transaction") { + val proc = actorOf[RedisSampleMapStorage] + proc.start + + (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft")) + (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft") + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1) + + val add = List(("a", "1"), ("b", "2"), ("c", "3")) + (proc !! CLEAR_AFTER_PUT(add)).getOrElse("CLEAR_AFTER_PUT failed") should equal(true) + + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1) + proc.stop + } + } + + describe("Ticket 343 Issue #3") { + it("map size should change after the transaction") { + val proc = actorOf[RedisSampleMapStorage] + proc.start + + (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft")) + (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft") + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1) + + (proc !! MSET(List(("dg", "1"), ("mc", "2"), ("nd", "3")))).getOrElse("Mset failed") should equal(3) + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(4) + + (proc !! GET("dg")).getOrElse("Get failed") should equal("1") + (proc !! GET("mc")).getOrElse("Get failed") should equal("2") + (proc !! GET("nd")).getOrElse("Get failed") should equal("3") + proc.stop + } + } + + describe("slice test") { + it("should pass") { + val proc = actorOf[RedisSampleMapStorage] + proc.start + + (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft")) + (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft") + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1) + + (proc !! MSET(List(("dg", "1"), ("mc", "2"), ("nd", "3")))).getOrElse("Mset failed") should equal(3) + (proc !! MAP_SIZE).getOrElse("Size failed") should equal(4) + + (proc !! PUT_WITH_SLICE(List(("ec", "1"), ("tb", "2"), ("mc", "10")), "dg", 3)).get.asInstanceOf[List[(Array[Byte], Array[Byte])]].map { case (k, v) => (new String(k), new String(v)) } should equal(List(("dg", "1"), ("ec", "1"), ("mc", "10"))) + + (proc !! PUT_REM_WITH_SLICE(List(("fc", "1"), ("gb", "2"), ("xy", "10")), List("tb", "fc"), "dg", 5)).get.asInstanceOf[List[(Array[Byte], Array[Byte])]].map { case (k, v) => (new String(k), new String(v)) } should equal(List(("dg", "1"), ("ec", "1"), ("gb", "2"), ("mc", "10"), ("nd", "3"))) + proc.stop + } + } + + describe("Ticket 343 Issue #4") { + it("vector get should not ignore elements that were in vector before transaction") { + val proc = actorOf[RedisSampleVectorStorage] + proc.start + + // add 4 elements in separate transactions + (proc !! VADD("debasish")).getOrElse("VADD failed") should equal(1) + (proc !! VADD("maulindu")).getOrElse("VADD failed") should equal(2) + (proc !! VADD("ramanendu")).getOrElse("VADD failed") should equal(3) + (proc !! VADD("nilanjan")).getOrElse("VADD failed") should equal(4) + + new String((proc !! VGET(0)).get.asInstanceOf[Array[Byte]] ) should equal("nilanjan") + new String((proc !! VGET(1)).get.asInstanceOf[Array[Byte]] ) should equal("ramanendu") + new String((proc !! VGET(2)).get.asInstanceOf[Array[Byte]] ) should equal("maulindu") + new String((proc !! VGET(3)).get.asInstanceOf[Array[Byte]] ) should equal("debasish") + + // now add 3 more and do gets in the same transaction + (proc !! VGET_AFTER_VADD(List("a", "b", "c"), List(0, 2, 4))).get.asInstanceOf[List[String]] should equal(List("c", "a", "ramanendu")) + proc.stop + } + } + + describe("Ticket 343 Issue #6") { + it("vector update should not ignore transaction") { + val proc = actorOf[RedisSampleVectorStorage] + proc.start + + // add 4 elements in separate transactions + (proc !! VADD("debasish")).getOrElse("VADD failed") should equal(1) + (proc !! VADD("maulindu")).getOrElse("VADD failed") should equal(2) + (proc !! VADD("ramanendu")).getOrElse("VADD failed") should equal(3) + (proc !! VADD("nilanjan")).getOrElse("VADD failed") should equal(4) + + evaluating { + (proc !! VUPD_AND_ABORT(0, "virat")).getOrElse("VUPD_AND_ABORT failed") + } should produce [Exception] + + // update aborts and hence values will remain unchanged + new String((proc !! VGET(0)).get.asInstanceOf[Array[Byte]] ) should equal("nilanjan") + proc.stop + } + } + + describe("Ticket 343 Issue #5") { + it("vector slice() should not ignore elements added in current transaction") { + val proc = actorOf[RedisSampleVectorStorage] + proc.start + + // add 4 elements in separate transactions + (proc !! VADD("debasish")).getOrElse("VADD failed") should equal(1) + (proc !! VADD("maulindu")).getOrElse("VADD failed") should equal(2) + (proc !! VADD("ramanendu")).getOrElse("VADD failed") should equal(3) + (proc !! VADD("nilanjan")).getOrElse("VADD failed") should equal(4) + + // slice with no new elements added in current transaction + (proc !! VADD_WITH_SLICE(List(), 2, 2)).getOrElse("VADD_WITH_SLICE failed") should equal(Vector("maulindu", "debasish")) + + // slice with new elements added in current transaction + (proc !! VADD_WITH_SLICE(List("a", "b", "c", "d"), 2, 2)).getOrElse("VADD_WITH_SLICE failed") should equal(Vector("b", "a")) + proc.stop + } + } +} diff --git a/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java b/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java index 0ab1a0aa10..ab05937f61 100644 --- a/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java +++ b/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java @@ -14,10 +14,10 @@ public final class RemoteProtocol { JAVA_ACTOR(1, 2), TYPED_ACTOR(2, 3), ; - - + + public final int getNumber() { return value; } - + public static ActorType valueOf(int value) { switch (value) { case 1: return SCALA_ACTOR; @@ -26,7 +26,7 @@ public final class RemoteProtocol { default: return null; } } - + public static com.google.protobuf.Internal.EnumLiteMap internalGetValueMap() { return internalValueMap; @@ -38,7 +38,7 @@ public final class RemoteProtocol { return ActorType.valueOf(number) ; } }; - + public final com.google.protobuf.Descriptors.EnumValueDescriptor getValueDescriptor() { return getDescriptor().getValues().get(index); @@ -51,9 +51,9 @@ public final class RemoteProtocol { getDescriptor() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(0); } - + private static final ActorType[] VALUES = { - SCALA_ACTOR, JAVA_ACTOR, TYPED_ACTOR, + SCALA_ACTOR, JAVA_ACTOR, TYPED_ACTOR, }; public static ActorType valueOf( com.google.protobuf.Descriptors.EnumValueDescriptor desc) { @@ -69,14 +69,14 @@ public final class RemoteProtocol { this.index = index; this.value = value; } - + static { se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor(); } - + // @@protoc_insertion_point(enum_scope:ActorType) } - + public enum SerializationSchemeType implements com.google.protobuf.ProtocolMessageEnum { JAVA(0, 1), @@ -85,10 +85,10 @@ public final class RemoteProtocol { JAVA_JSON(3, 4), PROTOBUF(4, 5), ; - - + + public final int getNumber() { return value; } - + public static SerializationSchemeType valueOf(int value) { switch (value) { case 1: return JAVA; @@ -99,7 +99,7 @@ public final class RemoteProtocol { default: return null; } } - + public static com.google.protobuf.Internal.EnumLiteMap internalGetValueMap() { return internalValueMap; @@ -111,7 +111,7 @@ public final class RemoteProtocol { return SerializationSchemeType.valueOf(number) ; } }; - + public final com.google.protobuf.Descriptors.EnumValueDescriptor getValueDescriptor() { return getDescriptor().getValues().get(index); @@ -124,9 +124,9 @@ public final class RemoteProtocol { getDescriptor() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(1); } - + private static final SerializationSchemeType[] VALUES = { - JAVA, SBINARY, SCALA_JSON, JAVA_JSON, PROTOBUF, + JAVA, SBINARY, SCALA_JSON, JAVA_JSON, PROTOBUF, }; public static SerializationSchemeType valueOf( com.google.protobuf.Descriptors.EnumValueDescriptor desc) { @@ -142,23 +142,23 @@ public final class RemoteProtocol { this.index = index; this.value = value; } - + static { se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor(); } - + // @@protoc_insertion_point(enum_scope:SerializationSchemeType) } - + public enum LifeCycleType implements com.google.protobuf.ProtocolMessageEnum { PERMANENT(0, 1), TEMPORARY(1, 2), ; - - + + public final int getNumber() { return value; } - + public static LifeCycleType valueOf(int value) { switch (value) { case 1: return PERMANENT; @@ -166,7 +166,7 @@ public final class RemoteProtocol { default: return null; } } - + public static com.google.protobuf.Internal.EnumLiteMap internalGetValueMap() { return internalValueMap; @@ -178,7 +178,7 @@ public final class RemoteProtocol { return LifeCycleType.valueOf(number) ; } }; - + public final com.google.protobuf.Descriptors.EnumValueDescriptor getValueDescriptor() { return getDescriptor().getValues().get(index); @@ -191,9 +191,9 @@ public final class RemoteProtocol { getDescriptor() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(2); } - + private static final LifeCycleType[] VALUES = { - PERMANENT, TEMPORARY, + PERMANENT, TEMPORARY, }; public static LifeCycleType valueOf( com.google.protobuf.Descriptors.EnumValueDescriptor desc) { @@ -209,14 +209,14 @@ public final class RemoteProtocol { this.index = index; this.value = value; } - + static { se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor(); } - + // @@protoc_insertion_point(enum_scope:LifeCycleType) } - + public static final class RemoteActorRefProtocol extends com.google.protobuf.GeneratedMessage { // Use RemoteActorRefProtocol.newBuilder() to construct. @@ -224,54 +224,54 @@ public final class RemoteProtocol { initFields(); } private RemoteActorRefProtocol(boolean noInit) {} - + private static final RemoteActorRefProtocol defaultInstance; public static RemoteActorRefProtocol getDefaultInstance() { return defaultInstance; } - + public RemoteActorRefProtocol getDefaultInstanceForType() { return defaultInstance; } - + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteActorRefProtocol_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { 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 actorClassname = 2; public static final int ACTORCLASSNAME_FIELD_NUMBER = 2; private boolean hasActorClassname; private java.lang.String actorClassname_ = ""; public boolean hasActorClassname() { return hasActorClassname; } public java.lang.String getActorClassname() { return actorClassname_; } - + // required .AddressProtocol homeAddress = 3; public static final int HOMEADDRESS_FIELD_NUMBER = 3; private boolean hasHomeAddress; private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol homeAddress_; public boolean hasHomeAddress() { return hasHomeAddress; } public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getHomeAddress() { return homeAddress_; } - + // optional uint64 timeout = 4; public static final int TIMEOUT_FIELD_NUMBER = 4; private boolean hasTimeout; private long timeout_ = 0L; public boolean hasTimeout() { return hasTimeout; } public long getTimeout() { return timeout_; } - + private void initFields() { homeAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); } @@ -282,7 +282,7 @@ public final class RemoteProtocol { if (!getHomeAddress().isInitialized()) return false; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -300,12 +300,12 @@ public final class RemoteProtocol { } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (hasUuid()) { size += com.google.protobuf.CodedOutputStream @@ -327,7 +327,7 @@ public final class RemoteProtocol { memoizedSerializedSize = size; return size; } - + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -394,31 +394,31 @@ public final class RemoteProtocol { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder { private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol result; - + // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder() private Builder() {} - + private static Builder create() { Builder builder = new Builder(); builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol(); return builder; } - + protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol internalGetResult() { return result; } - + public Builder clear() { if (result == null) { throw new IllegalStateException( @@ -427,20 +427,20 @@ public final class RemoteProtocol { result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol(); return this; } - + public Builder clone() { return create().mergeFrom(result); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDescriptor(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getDefaultInstanceForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); } - + public boolean isInitialized() { return result.isInitialized(); } @@ -450,7 +450,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { @@ -459,7 +459,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol buildPartial() { if (result == null) { throw new IllegalStateException( @@ -469,7 +469,7 @@ public final class RemoteProtocol { result = null; return returnMe; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol) { return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol)other); @@ -478,7 +478,7 @@ public final class RemoteProtocol { return this; } } - + 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()) { @@ -496,7 +496,7 @@ public final class RemoteProtocol { this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -542,8 +542,8 @@ public final class RemoteProtocol { } } } - - + + // required string uuid = 1; public boolean hasUuid() { return result.hasUuid(); @@ -564,7 +564,7 @@ public final class RemoteProtocol { result.uuid_ = getDefaultInstance().getUuid(); return this; } - + // required string actorClassname = 2; public boolean hasActorClassname() { return result.hasActorClassname(); @@ -585,7 +585,7 @@ public final class RemoteProtocol { result.actorClassname_ = getDefaultInstance().getActorClassname(); return this; } - + // required .AddressProtocol homeAddress = 3; public boolean hasHomeAddress() { return result.hasHomeAddress(); @@ -622,7 +622,7 @@ public final class RemoteProtocol { result.homeAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); return this; } - + // optional uint64 timeout = 4; public boolean hasTimeout() { return result.hasTimeout(); @@ -640,19 +640,19 @@ public final class RemoteProtocol { result.timeout_ = 0L; return this; } - + // @@protoc_insertion_point(builder_scope:RemoteActorRefProtocol) } - + static { defaultInstance = new RemoteActorRefProtocol(true); se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:RemoteActorRefProtocol) } - + public static final class SerializedActorRefProtocol extends com.google.protobuf.GeneratedMessage { // Use SerializedActorRefProtocol.newBuilder() to construct. @@ -660,110 +660,110 @@ public final class RemoteProtocol { initFields(); } private SerializedActorRefProtocol(boolean noInit) {} - + private static final SerializedActorRefProtocol defaultInstance; public static SerializedActorRefProtocol getDefaultInstance() { return defaultInstance; } - + public SerializedActorRefProtocol getDefaultInstanceForType() { return defaultInstance; } - + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_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 id = 2; public static final int ID_FIELD_NUMBER = 2; private boolean hasId; private java.lang.String id_ = ""; public boolean hasId() { return hasId; } public java.lang.String getId() { return id_; } - + // required string actorClassname = 3; public static final int ACTORCLASSNAME_FIELD_NUMBER = 3; private boolean hasActorClassname; private java.lang.String actorClassname_ = ""; public boolean hasActorClassname() { return hasActorClassname; } public java.lang.String getActorClassname() { return actorClassname_; } - + // required .AddressProtocol originalAddress = 4; public static final int ORIGINALADDRESS_FIELD_NUMBER = 4; private boolean hasOriginalAddress; private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol originalAddress_; public boolean hasOriginalAddress() { return hasOriginalAddress; } public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getOriginalAddress() { return originalAddress_; } - + // optional bytes actorInstance = 5; public static final int ACTORINSTANCE_FIELD_NUMBER = 5; private boolean hasActorInstance; private com.google.protobuf.ByteString actorInstance_ = com.google.protobuf.ByteString.EMPTY; public boolean hasActorInstance() { return hasActorInstance; } public com.google.protobuf.ByteString getActorInstance() { return actorInstance_; } - + // optional string serializerClassname = 6; public static final int SERIALIZERCLASSNAME_FIELD_NUMBER = 6; private boolean hasSerializerClassname; private java.lang.String serializerClassname_ = ""; public boolean hasSerializerClassname() { return hasSerializerClassname; } public java.lang.String getSerializerClassname() { return serializerClassname_; } - + // optional bool isTransactor = 7; public static final int ISTRANSACTOR_FIELD_NUMBER = 7; private boolean hasIsTransactor; private boolean isTransactor_ = false; public boolean hasIsTransactor() { return hasIsTransactor; } public boolean getIsTransactor() { return isTransactor_; } - + // optional uint64 timeout = 8; public static final int TIMEOUT_FIELD_NUMBER = 8; private boolean hasTimeout; private long timeout_ = 0L; public boolean hasTimeout() { return hasTimeout; } public long getTimeout() { return timeout_; } - + // optional uint64 receiveTimeout = 9; public static final int RECEIVETIMEOUT_FIELD_NUMBER = 9; private boolean hasReceiveTimeout; private long receiveTimeout_ = 0L; public boolean hasReceiveTimeout() { return hasReceiveTimeout; } public long getReceiveTimeout() { return receiveTimeout_; } - + // optional .LifeCycleProtocol lifeCycle = 10; public static final int LIFECYCLE_FIELD_NUMBER = 10; private boolean hasLifeCycle; private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol lifeCycle_; public boolean hasLifeCycle() { return hasLifeCycle; } public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getLifeCycle() { return lifeCycle_; } - + // optional .RemoteActorRefProtocol supervisor = 11; public static final int SUPERVISOR_FIELD_NUMBER = 11; private boolean hasSupervisor; private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol supervisor_; public boolean hasSupervisor() { return hasSupervisor; } public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor() { return supervisor_; } - + // optional bytes hotswapStack = 12; public static final int HOTSWAPSTACK_FIELD_NUMBER = 12; private boolean hasHotswapStack; private com.google.protobuf.ByteString hotswapStack_ = com.google.protobuf.ByteString.EMPTY; public boolean hasHotswapStack() { return hasHotswapStack; } public com.google.protobuf.ByteString getHotswapStack() { return hotswapStack_; } - + // repeated .RemoteRequestProtocol messages = 13; public static final int MESSAGES_FIELD_NUMBER = 13; private java.util.List messages_ = @@ -775,7 +775,7 @@ public final class RemoteProtocol { public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getMessages(int index) { return messages_.get(index); } - + private void initFields() { originalAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); @@ -798,7 +798,7 @@ public final class RemoteProtocol { } return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -843,12 +843,12 @@ public final class RemoteProtocol { } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (hasUuid()) { size += com.google.protobuf.CodedOutputStream @@ -906,7 +906,7 @@ public final class RemoteProtocol { memoizedSerializedSize = size; return size; } - + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -973,31 +973,31 @@ public final class RemoteProtocol { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder { private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol result; - + // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder() private Builder() {} - + private static Builder create() { Builder builder = new Builder(); builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol(); return builder; } - + protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol internalGetResult() { return result; } - + public Builder clear() { if (result == null) { throw new IllegalStateException( @@ -1006,20 +1006,20 @@ public final class RemoteProtocol { result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol(); return this; } - + public Builder clone() { return create().mergeFrom(result); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDescriptor(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getDefaultInstanceForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance(); } - + public boolean isInitialized() { return result.isInitialized(); } @@ -1029,7 +1029,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { @@ -1038,7 +1038,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildPartial() { if (result == null) { throw new IllegalStateException( @@ -1052,7 +1052,7 @@ public final class RemoteProtocol { result = null; return returnMe; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol) { return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol)other); @@ -1061,7 +1061,7 @@ public final class RemoteProtocol { return this; } } - + 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()) { @@ -1109,7 +1109,7 @@ public final class RemoteProtocol { this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -1203,8 +1203,8 @@ public final class RemoteProtocol { } } } - - + + // required string uuid = 1; public boolean hasUuid() { return result.hasUuid(); @@ -1225,7 +1225,7 @@ public final class RemoteProtocol { result.uuid_ = getDefaultInstance().getUuid(); return this; } - + // required string id = 2; public boolean hasId() { return result.hasId(); @@ -1246,7 +1246,7 @@ public final class RemoteProtocol { result.id_ = getDefaultInstance().getId(); return this; } - + // required string actorClassname = 3; public boolean hasActorClassname() { return result.hasActorClassname(); @@ -1267,7 +1267,7 @@ public final class RemoteProtocol { result.actorClassname_ = getDefaultInstance().getActorClassname(); return this; } - + // required .AddressProtocol originalAddress = 4; public boolean hasOriginalAddress() { return result.hasOriginalAddress(); @@ -1304,7 +1304,7 @@ public final class RemoteProtocol { result.originalAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); return this; } - + // optional bytes actorInstance = 5; public boolean hasActorInstance() { return result.hasActorInstance(); @@ -1325,7 +1325,7 @@ public final class RemoteProtocol { result.actorInstance_ = getDefaultInstance().getActorInstance(); return this; } - + // optional string serializerClassname = 6; public boolean hasSerializerClassname() { return result.hasSerializerClassname(); @@ -1346,7 +1346,7 @@ public final class RemoteProtocol { result.serializerClassname_ = getDefaultInstance().getSerializerClassname(); return this; } - + // optional bool isTransactor = 7; public boolean hasIsTransactor() { return result.hasIsTransactor(); @@ -1364,7 +1364,7 @@ public final class RemoteProtocol { result.isTransactor_ = false; return this; } - + // optional uint64 timeout = 8; public boolean hasTimeout() { return result.hasTimeout(); @@ -1382,7 +1382,7 @@ public final class RemoteProtocol { result.timeout_ = 0L; return this; } - + // optional uint64 receiveTimeout = 9; public boolean hasReceiveTimeout() { return result.hasReceiveTimeout(); @@ -1400,7 +1400,7 @@ public final class RemoteProtocol { result.receiveTimeout_ = 0L; return this; } - + // optional .LifeCycleProtocol lifeCycle = 10; public boolean hasLifeCycle() { return result.hasLifeCycle(); @@ -1437,7 +1437,7 @@ public final class RemoteProtocol { result.lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); return this; } - + // optional .RemoteActorRefProtocol supervisor = 11; public boolean hasSupervisor() { return result.hasSupervisor(); @@ -1474,7 +1474,7 @@ public final class RemoteProtocol { result.supervisor_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); return this; } - + // optional bytes hotswapStack = 12; public boolean hasHotswapStack() { return result.hasHotswapStack(); @@ -1495,7 +1495,7 @@ public final class RemoteProtocol { result.hotswapStack_ = getDefaultInstance().getHotswapStack(); return this; } - + // repeated .RemoteRequestProtocol messages = 13; public java.util.List getMessagesList() { return java.util.Collections.unmodifiableList(result.messages_); @@ -1546,19 +1546,19 @@ public final class RemoteProtocol { result.messages_ = java.util.Collections.emptyList(); return this; } - + // @@protoc_insertion_point(builder_scope:SerializedActorRefProtocol) } - + static { defaultInstance = new SerializedActorRefProtocol(true); se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:SerializedActorRefProtocol) } - + public static final class MessageProtocol extends com.google.protobuf.GeneratedMessage { // Use MessageProtocol.newBuilder() to construct. @@ -1566,47 +1566,47 @@ public final class RemoteProtocol { initFields(); } private MessageProtocol(boolean noInit) {} - + private static final MessageProtocol defaultInstance; public static MessageProtocol getDefaultInstance() { return defaultInstance; } - + public MessageProtocol getDefaultInstanceForType() { return defaultInstance; } - + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MessageProtocol_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MessageProtocol_fieldAccessorTable; } - + // required .SerializationSchemeType serializationScheme = 1; public static final int SERIALIZATIONSCHEME_FIELD_NUMBER = 1; private boolean hasSerializationScheme; private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType serializationScheme_; public boolean hasSerializationScheme() { return hasSerializationScheme; } public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType getSerializationScheme() { return serializationScheme_; } - + // required bytes message = 2; public static final int MESSAGE_FIELD_NUMBER = 2; private boolean hasMessage; private com.google.protobuf.ByteString message_ = com.google.protobuf.ByteString.EMPTY; public boolean hasMessage() { return hasMessage; } public com.google.protobuf.ByteString getMessage() { return message_; } - + // optional bytes messageManifest = 3; public static final int MESSAGEMANIFEST_FIELD_NUMBER = 3; private boolean hasMessageManifest; private com.google.protobuf.ByteString messageManifest_ = com.google.protobuf.ByteString.EMPTY; public boolean hasMessageManifest() { return hasMessageManifest; } public com.google.protobuf.ByteString getMessageManifest() { return messageManifest_; } - + private void initFields() { serializationScheme_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA; } @@ -1615,7 +1615,7 @@ public final class RemoteProtocol { if (!hasMessage) return false; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -1630,12 +1630,12 @@ public final class RemoteProtocol { } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (hasSerializationScheme()) { size += com.google.protobuf.CodedOutputStream @@ -1653,7 +1653,7 @@ public final class RemoteProtocol { memoizedSerializedSize = size; return size; } - + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -1720,31 +1720,31 @@ public final class RemoteProtocol { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder { private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol result; - + // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder() private Builder() {} - + private static Builder create() { Builder builder = new Builder(); builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol(); return builder; } - + protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol internalGetResult() { return result; } - + public Builder clear() { if (result == null) { throw new IllegalStateException( @@ -1753,20 +1753,20 @@ public final class RemoteProtocol { result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol(); return this; } - + public Builder clone() { return create().mergeFrom(result); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDescriptor(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getDefaultInstanceForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); } - + public boolean isInitialized() { return result.isInitialized(); } @@ -1776,7 +1776,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { @@ -1785,7 +1785,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol buildPartial() { if (result == null) { throw new IllegalStateException( @@ -1795,7 +1795,7 @@ public final class RemoteProtocol { result = null; return returnMe; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol) { return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol)other); @@ -1804,7 +1804,7 @@ public final class RemoteProtocol { return this; } } - + public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol other) { if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) return this; if (other.hasSerializationScheme()) { @@ -1819,7 +1819,7 @@ public final class RemoteProtocol { this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -1862,8 +1862,8 @@ public final class RemoteProtocol { } } } - - + + // required .SerializationSchemeType serializationScheme = 1; public boolean hasSerializationScheme() { return result.hasSerializationScheme(); @@ -1884,7 +1884,7 @@ public final class RemoteProtocol { result.serializationScheme_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA; return this; } - + // required bytes message = 2; public boolean hasMessage() { return result.hasMessage(); @@ -1905,7 +1905,7 @@ public final class RemoteProtocol { result.message_ = getDefaultInstance().getMessage(); return this; } - + // optional bytes messageManifest = 3; public boolean hasMessageManifest() { return result.hasMessageManifest(); @@ -1926,19 +1926,19 @@ public final class RemoteProtocol { result.messageManifest_ = getDefaultInstance().getMessageManifest(); return this; } - + // @@protoc_insertion_point(builder_scope:MessageProtocol) } - + static { defaultInstance = new MessageProtocol(true); se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:MessageProtocol) } - + public static final class ActorInfoProtocol extends com.google.protobuf.GeneratedMessage { // Use ActorInfoProtocol.newBuilder() to construct. @@ -1946,61 +1946,68 @@ public final class RemoteProtocol { initFields(); } private ActorInfoProtocol(boolean noInit) {} - + private static final ActorInfoProtocol defaultInstance; public static ActorInfoProtocol getDefaultInstance() { return defaultInstance; } - + public ActorInfoProtocol getDefaultInstanceForType() { return defaultInstance; } - + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_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 target = 2; public static final int TARGET_FIELD_NUMBER = 2; private boolean hasTarget; private java.lang.String target_ = ""; public boolean hasTarget() { return hasTarget; } public java.lang.String getTarget() { return target_; } - + // required uint64 timeout = 3; public static final int TIMEOUT_FIELD_NUMBER = 3; private boolean hasTimeout; private long timeout_ = 0L; public boolean hasTimeout() { return hasTimeout; } public long getTimeout() { return timeout_; } - + // required .ActorType actorType = 4; public static final int ACTORTYPE_FIELD_NUMBER = 4; private boolean hasActorType; private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType actorType_; public boolean hasActorType() { return hasActorType; } public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType getActorType() { return actorType_; } - + // optional .TypedActorInfoProtocol typedActorInfo = 5; public static final int TYPEDACTORINFO_FIELD_NUMBER = 5; private boolean hasTypedActorInfo; private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol typedActorInfo_; public boolean hasTypedActorInfo() { return hasTypedActorInfo; } public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getTypedActorInfo() { return typedActorInfo_; } - + + // optional string id = 6; + public static final int ID_FIELD_NUMBER = 6; + private boolean hasId; + private java.lang.String id_ = ""; + public boolean hasId() { return hasId; } + public java.lang.String getId() { return id_; } + private void initFields() { actorType_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.SCALA_ACTOR; typedActorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance(); @@ -2015,7 +2022,7 @@ public final class RemoteProtocol { } return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -2034,14 +2041,17 @@ public final class RemoteProtocol { if (hasTypedActorInfo()) { output.writeMessage(5, getTypedActorInfo()); } + if (hasId()) { + output.writeString(6, getId()); + } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (hasUuid()) { size += com.google.protobuf.CodedOutputStream @@ -2063,11 +2073,15 @@ public final class RemoteProtocol { size += com.google.protobuf.CodedOutputStream .computeMessageSize(5, getTypedActorInfo()); } + if (hasId()) { + size += com.google.protobuf.CodedOutputStream + .computeStringSize(6, getId()); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } - + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -2134,31 +2148,31 @@ public final class RemoteProtocol { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder { private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol result; - + // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder() private Builder() {} - + private static Builder create() { Builder builder = new Builder(); builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol(); return builder; } - + protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol internalGetResult() { return result; } - + public Builder clear() { if (result == null) { throw new IllegalStateException( @@ -2167,20 +2181,20 @@ public final class RemoteProtocol { result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol(); return this; } - + public Builder clone() { return create().mergeFrom(result); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDescriptor(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getDefaultInstanceForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); } - + public boolean isInitialized() { return result.isInitialized(); } @@ -2190,7 +2204,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { @@ -2199,7 +2213,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol buildPartial() { if (result == null) { throw new IllegalStateException( @@ -2209,7 +2223,7 @@ public final class RemoteProtocol { result = null; return returnMe; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol) { return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol)other); @@ -2218,7 +2232,7 @@ public final class RemoteProtocol { return this; } } - + 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()) { @@ -2236,10 +2250,13 @@ public final class RemoteProtocol { if (other.hasTypedActorInfo()) { mergeTypedActorInfo(other.getTypedActorInfo()); } + if (other.hasId()) { + setId(other.getId()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -2292,11 +2309,15 @@ public final class RemoteProtocol { setTypedActorInfo(subBuilder.buildPartial()); break; } + case 50: { + setId(input.readString()); + break; + } } } } - - + + // required string uuid = 1; public boolean hasUuid() { return result.hasUuid(); @@ -2317,7 +2338,7 @@ public final class RemoteProtocol { result.uuid_ = getDefaultInstance().getUuid(); return this; } - + // required string target = 2; public boolean hasTarget() { return result.hasTarget(); @@ -2338,7 +2359,7 @@ public final class RemoteProtocol { result.target_ = getDefaultInstance().getTarget(); return this; } - + // required uint64 timeout = 3; public boolean hasTimeout() { return result.hasTimeout(); @@ -2356,7 +2377,7 @@ public final class RemoteProtocol { result.timeout_ = 0L; return this; } - + // required .ActorType actorType = 4; public boolean hasActorType() { return result.hasActorType(); @@ -2377,7 +2398,7 @@ public final class RemoteProtocol { result.actorType_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.SCALA_ACTOR; return this; } - + // optional .TypedActorInfoProtocol typedActorInfo = 5; public boolean hasTypedActorInfo() { return result.hasTypedActorInfo(); @@ -2414,19 +2435,40 @@ public final class RemoteProtocol { result.typedActorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance(); return this; } - + + // optional string id = 6; + public boolean hasId() { + return result.hasId(); + } + public java.lang.String getId() { + return result.getId(); + } + public Builder setId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + result.hasId = true; + result.id_ = value; + return this; + } + public Builder clearId() { + result.hasId = false; + result.id_ = getDefaultInstance().getId(); + return this; + } + // @@protoc_insertion_point(builder_scope:ActorInfoProtocol) } - + static { defaultInstance = new ActorInfoProtocol(true); se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:ActorInfoProtocol) } - + public static final class TypedActorInfoProtocol extends com.google.protobuf.GeneratedMessage { // Use TypedActorInfoProtocol.newBuilder() to construct. @@ -2434,40 +2476,40 @@ public final class RemoteProtocol { initFields(); } private TypedActorInfoProtocol(boolean noInit) {} - + private static final TypedActorInfoProtocol defaultInstance; public static TypedActorInfoProtocol getDefaultInstance() { return defaultInstance; } - + public TypedActorInfoProtocol getDefaultInstanceForType() { return defaultInstance; } - + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_TypedActorInfoProtocol_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_TypedActorInfoProtocol_fieldAccessorTable; } - + // required string interface = 1; public static final int INTERFACE_FIELD_NUMBER = 1; private boolean hasInterface; private java.lang.String interface_ = ""; public boolean hasInterface() { return hasInterface; } public java.lang.String getInterface() { return interface_; } - + // required string method = 2; public static final int METHOD_FIELD_NUMBER = 2; private boolean hasMethod; private java.lang.String method_ = ""; public boolean hasMethod() { return hasMethod; } public java.lang.String getMethod() { return method_; } - + private void initFields() { } public final boolean isInitialized() { @@ -2475,7 +2517,7 @@ public final class RemoteProtocol { if (!hasMethod) return false; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -2487,12 +2529,12 @@ public final class RemoteProtocol { } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (hasInterface()) { size += com.google.protobuf.CodedOutputStream @@ -2506,7 +2548,7 @@ public final class RemoteProtocol { memoizedSerializedSize = size; return size; } - + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -2573,31 +2615,31 @@ public final class RemoteProtocol { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder { private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol result; - + // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder() private Builder() {} - + private static Builder create() { Builder builder = new Builder(); builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol(); return builder; } - + protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol internalGetResult() { return result; } - + public Builder clear() { if (result == null) { throw new IllegalStateException( @@ -2606,20 +2648,20 @@ public final class RemoteProtocol { result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol(); return this; } - + public Builder clone() { return create().mergeFrom(result); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDescriptor(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getDefaultInstanceForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance(); } - + public boolean isInitialized() { return result.isInitialized(); } @@ -2629,7 +2671,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { @@ -2638,7 +2680,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol buildPartial() { if (result == null) { throw new IllegalStateException( @@ -2648,7 +2690,7 @@ public final class RemoteProtocol { result = null; return returnMe; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol) { return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol)other); @@ -2657,7 +2699,7 @@ public final class RemoteProtocol { return this; } } - + public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol other) { if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance()) return this; if (other.hasInterface()) { @@ -2669,7 +2711,7 @@ public final class RemoteProtocol { this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -2702,8 +2744,8 @@ public final class RemoteProtocol { } } } - - + + // required string interface = 1; public boolean hasInterface() { return result.hasInterface(); @@ -2724,7 +2766,7 @@ public final class RemoteProtocol { result.interface_ = getDefaultInstance().getInterface(); return this; } - + // required string method = 2; public boolean hasMethod() { return result.hasMethod(); @@ -2745,19 +2787,19 @@ public final class RemoteProtocol { result.method_ = getDefaultInstance().getMethod(); return this; } - + // @@protoc_insertion_point(builder_scope:TypedActorInfoProtocol) } - + static { defaultInstance = new TypedActorInfoProtocol(true); se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:TypedActorInfoProtocol) } - + public static final class RemoteRequestProtocol extends com.google.protobuf.GeneratedMessage { // Use RemoteRequestProtocol.newBuilder() to construct. @@ -2765,68 +2807,80 @@ public final class RemoteProtocol { initFields(); } private RemoteRequestProtocol(boolean noInit) {} - + private static final RemoteRequestProtocol defaultInstance; public static RemoteRequestProtocol getDefaultInstance() { return defaultInstance; } - + public RemoteRequestProtocol getDefaultInstanceForType() { return defaultInstance; } - + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteRequestProtocol_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { 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 .MessageProtocol message = 2; public static final int MESSAGE_FIELD_NUMBER = 2; private boolean hasMessage; private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol message_; public boolean hasMessage() { return hasMessage; } public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { return message_; } - + // required .ActorInfoProtocol actorInfo = 3; public static final int ACTORINFO_FIELD_NUMBER = 3; private boolean hasActorInfo; private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol actorInfo_; public boolean hasActorInfo() { return hasActorInfo; } public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo() { return actorInfo_; } - + // required bool isOneWay = 4; public static final int ISONEWAY_FIELD_NUMBER = 4; private boolean hasIsOneWay; private boolean isOneWay_ = false; public boolean hasIsOneWay() { return hasIsOneWay; } public boolean getIsOneWay() { return isOneWay_; } - + // optional string supervisorUuid = 5; public static final int SUPERVISORUUID_FIELD_NUMBER = 5; private boolean hasSupervisorUuid; private java.lang.String supervisorUuid_ = ""; public boolean hasSupervisorUuid() { return hasSupervisorUuid; } public java.lang.String getSupervisorUuid() { return supervisorUuid_; } - + // optional .RemoteActorRefProtocol sender = 6; public static final int SENDER_FIELD_NUMBER = 6; private boolean hasSender; private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol sender_; public boolean hasSender() { return hasSender; } public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSender() { return sender_; } - + + // repeated .MetadataEntryProtocol metadata = 7; + public static final int METADATA_FIELD_NUMBER = 7; + private java.util.List metadata_ = + java.util.Collections.emptyList(); + public java.util.List getMetadataList() { + return metadata_; + } + public int getMetadataCount() { return metadata_.size(); } + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getMetadata(int index) { + return metadata_.get(index); + } + private void initFields() { message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); actorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); @@ -2842,9 +2896,12 @@ public final class RemoteProtocol { if (hasSender()) { if (!getSender().isInitialized()) return false; } + for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { + if (!element.isInitialized()) return false; + } return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -2866,14 +2923,17 @@ public final class RemoteProtocol { if (hasSender()) { output.writeMessage(6, getSender()); } + for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { + output.writeMessage(7, element); + } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (hasId()) { size += com.google.protobuf.CodedOutputStream @@ -2899,11 +2959,15 @@ public final class RemoteProtocol { size += com.google.protobuf.CodedOutputStream .computeMessageSize(6, getSender()); } + for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(7, element); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } - + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -2970,31 +3034,31 @@ public final class RemoteProtocol { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder { private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol result; - + // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.newBuilder() private Builder() {} - + private static Builder create() { Builder builder = new Builder(); builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol(); return builder; } - + protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol internalGetResult() { return result; } - + public Builder clear() { if (result == null) { throw new IllegalStateException( @@ -3003,20 +3067,20 @@ public final class RemoteProtocol { result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol(); return this; } - + public Builder clone() { return create().mergeFrom(result); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDescriptor(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getDefaultInstanceForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDefaultInstance(); } - + public boolean isInitialized() { return result.isInitialized(); } @@ -3026,7 +3090,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { @@ -3035,17 +3099,21 @@ public final class RemoteProtocol { } return buildPartial(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol buildPartial() { if (result == null) { throw new IllegalStateException( "build() has already been called on this Builder."); } + if (result.metadata_ != java.util.Collections.EMPTY_LIST) { + result.metadata_ = + java.util.Collections.unmodifiableList(result.metadata_); + } se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol returnMe = result; result = null; return returnMe; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol) { return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol)other); @@ -3054,7 +3122,7 @@ public final class RemoteProtocol { return this; } } - + 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()) { @@ -3075,10 +3143,16 @@ public final class RemoteProtocol { if (other.hasSender()) { mergeSender(other.getSender()); } + if (!other.metadata_.isEmpty()) { + if (result.metadata_.isEmpty()) { + result.metadata_ = new java.util.ArrayList(); + } + result.metadata_.addAll(other.metadata_); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -3139,11 +3213,17 @@ public final class RemoteProtocol { setSender(subBuilder.buildPartial()); break; } + case 58: { + se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addMetadata(subBuilder.buildPartial()); + break; + } } } } - - + + // required uint64 id = 1; public boolean hasId() { return result.hasId(); @@ -3161,7 +3241,7 @@ public final class RemoteProtocol { result.id_ = 0L; return this; } - + // required .MessageProtocol message = 2; public boolean hasMessage() { return result.hasMessage(); @@ -3198,7 +3278,7 @@ public final class RemoteProtocol { result.message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); return this; } - + // required .ActorInfoProtocol actorInfo = 3; public boolean hasActorInfo() { return result.hasActorInfo(); @@ -3235,7 +3315,7 @@ public final class RemoteProtocol { result.actorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); return this; } - + // required bool isOneWay = 4; public boolean hasIsOneWay() { return result.hasIsOneWay(); @@ -3253,7 +3333,7 @@ public final class RemoteProtocol { result.isOneWay_ = false; return this; } - + // optional string supervisorUuid = 5; public boolean hasSupervisorUuid() { return result.hasSupervisorUuid(); @@ -3274,7 +3354,7 @@ public final class RemoteProtocol { result.supervisorUuid_ = getDefaultInstance().getSupervisorUuid(); return this; } - + // optional .RemoteActorRefProtocol sender = 6; public boolean hasSender() { return result.hasSender(); @@ -3311,19 +3391,70 @@ public final class RemoteProtocol { result.sender_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); return this; } - + + // repeated .MetadataEntryProtocol metadata = 7; + public java.util.List getMetadataList() { + return java.util.Collections.unmodifiableList(result.metadata_); + } + public int getMetadataCount() { + return result.getMetadataCount(); + } + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getMetadata(int index) { + return result.getMetadata(index); + } + public Builder setMetadata(int index, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol value) { + if (value == null) { + throw new NullPointerException(); + } + result.metadata_.set(index, value); + return this; + } + public Builder setMetadata(int index, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) { + result.metadata_.set(index, builderForValue.build()); + return this; + } + public Builder addMetadata(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol value) { + if (value == null) { + throw new NullPointerException(); + } + if (result.metadata_.isEmpty()) { + result.metadata_ = new java.util.ArrayList(); + } + result.metadata_.add(value); + return this; + } + public Builder addMetadata(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) { + if (result.metadata_.isEmpty()) { + result.metadata_ = new java.util.ArrayList(); + } + result.metadata_.add(builderForValue.build()); + return this; + } + public Builder addAllMetadata( + java.lang.Iterable values) { + if (result.metadata_.isEmpty()) { + result.metadata_ = new java.util.ArrayList(); + } + super.addAll(values, result.metadata_); + return this; + } + public Builder clearMetadata() { + result.metadata_ = java.util.Collections.emptyList(); + return this; + } + // @@protoc_insertion_point(builder_scope:RemoteRequestProtocol) } - + static { defaultInstance = new RemoteRequestProtocol(true); se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:RemoteRequestProtocol) } - + public static final class RemoteReplyProtocol extends com.google.protobuf.GeneratedMessage { // Use RemoteReplyProtocol.newBuilder() to construct. @@ -3331,68 +3462,80 @@ public final class RemoteProtocol { initFields(); } private RemoteReplyProtocol(boolean noInit) {} - + private static final RemoteReplyProtocol defaultInstance; public static RemoteReplyProtocol getDefaultInstance() { return defaultInstance; } - + public RemoteReplyProtocol getDefaultInstanceForType() { return defaultInstance; } - + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteReplyProtocol_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { 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_; } - + // optional .MessageProtocol message = 2; public static final int MESSAGE_FIELD_NUMBER = 2; private boolean hasMessage; private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol message_; public boolean hasMessage() { return hasMessage; } public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { return message_; } - + // optional .ExceptionProtocol exception = 3; public static final int EXCEPTION_FIELD_NUMBER = 3; private boolean hasException; private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol exception_; public boolean hasException() { return hasException; } public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() { return exception_; } - + // optional string supervisorUuid = 4; public static final int SUPERVISORUUID_FIELD_NUMBER = 4; private boolean hasSupervisorUuid; private java.lang.String supervisorUuid_ = ""; public boolean hasSupervisorUuid() { return hasSupervisorUuid; } public java.lang.String getSupervisorUuid() { return supervisorUuid_; } - + // required bool isActor = 5; public static final int ISACTOR_FIELD_NUMBER = 5; private boolean hasIsActor; private boolean isActor_ = false; public boolean hasIsActor() { return hasIsActor; } public boolean getIsActor() { return isActor_; } - + // required bool isSuccessful = 6; public static final int ISSUCCESSFUL_FIELD_NUMBER = 6; private boolean hasIsSuccessful; private boolean isSuccessful_ = false; public boolean hasIsSuccessful() { return hasIsSuccessful; } public boolean getIsSuccessful() { return isSuccessful_; } - + + // repeated .MetadataEntryProtocol metadata = 7; + public static final int METADATA_FIELD_NUMBER = 7; + private java.util.List metadata_ = + java.util.Collections.emptyList(); + public java.util.List getMetadataList() { + return metadata_; + } + public int getMetadataCount() { return metadata_.size(); } + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getMetadata(int index) { + return metadata_.get(index); + } + private void initFields() { message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); exception_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); @@ -3407,9 +3550,12 @@ public final class RemoteProtocol { if (hasException()) { if (!getException().isInitialized()) return false; } + for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { + if (!element.isInitialized()) return false; + } return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -3431,14 +3577,17 @@ public final class RemoteProtocol { if (hasIsSuccessful()) { output.writeBool(6, getIsSuccessful()); } + for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { + output.writeMessage(7, element); + } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (hasId()) { size += com.google.protobuf.CodedOutputStream @@ -3464,11 +3613,15 @@ public final class RemoteProtocol { size += com.google.protobuf.CodedOutputStream .computeBoolSize(6, getIsSuccessful()); } + for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(7, element); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } - + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -3535,31 +3688,31 @@ public final class RemoteProtocol { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder { private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol result; - + // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.newBuilder() private Builder() {} - + private static Builder create() { Builder builder = new Builder(); builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol(); return builder; } - + protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol internalGetResult() { return result; } - + public Builder clear() { if (result == null) { throw new IllegalStateException( @@ -3568,20 +3721,20 @@ public final class RemoteProtocol { result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol(); return this; } - + public Builder clone() { return create().mergeFrom(result); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDescriptor(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol getDefaultInstanceForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDefaultInstance(); } - + public boolean isInitialized() { return result.isInitialized(); } @@ -3591,7 +3744,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { @@ -3600,17 +3753,21 @@ public final class RemoteProtocol { } return buildPartial(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol buildPartial() { if (result == null) { throw new IllegalStateException( "build() has already been called on this Builder."); } + if (result.metadata_ != java.util.Collections.EMPTY_LIST) { + result.metadata_ = + java.util.Collections.unmodifiableList(result.metadata_); + } se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol returnMe = result; result = null; return returnMe; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol) { return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol)other); @@ -3619,7 +3776,7 @@ public final class RemoteProtocol { return this; } } - + 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()) { @@ -3640,10 +3797,16 @@ public final class RemoteProtocol { if (other.hasIsSuccessful()) { setIsSuccessful(other.getIsSuccessful()); } + if (!other.metadata_.isEmpty()) { + if (result.metadata_.isEmpty()) { + result.metadata_ = new java.util.ArrayList(); + } + result.metadata_.addAll(other.metadata_); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -3699,11 +3862,17 @@ public final class RemoteProtocol { setIsSuccessful(input.readBool()); break; } + case 58: { + se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addMetadata(subBuilder.buildPartial()); + break; + } } } } - - + + // required uint64 id = 1; public boolean hasId() { return result.hasId(); @@ -3721,7 +3890,7 @@ public final class RemoteProtocol { result.id_ = 0L; return this; } - + // optional .MessageProtocol message = 2; public boolean hasMessage() { return result.hasMessage(); @@ -3758,7 +3927,7 @@ public final class RemoteProtocol { result.message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); return this; } - + // optional .ExceptionProtocol exception = 3; public boolean hasException() { return result.hasException(); @@ -3795,7 +3964,7 @@ public final class RemoteProtocol { result.exception_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); return this; } - + // optional string supervisorUuid = 4; public boolean hasSupervisorUuid() { return result.hasSupervisorUuid(); @@ -3816,7 +3985,7 @@ public final class RemoteProtocol { result.supervisorUuid_ = getDefaultInstance().getSupervisorUuid(); return this; } - + // required bool isActor = 5; public boolean hasIsActor() { return result.hasIsActor(); @@ -3834,7 +4003,7 @@ public final class RemoteProtocol { result.isActor_ = false; return this; } - + // required bool isSuccessful = 6; public boolean hasIsSuccessful() { return result.hasIsSuccessful(); @@ -3852,19 +4021,726 @@ public final class RemoteProtocol { result.isSuccessful_ = false; return this; } - + + // repeated .MetadataEntryProtocol metadata = 7; + public java.util.List getMetadataList() { + return java.util.Collections.unmodifiableList(result.metadata_); + } + public int getMetadataCount() { + return result.getMetadataCount(); + } + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getMetadata(int index) { + return result.getMetadata(index); + } + public Builder setMetadata(int index, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol value) { + if (value == null) { + throw new NullPointerException(); + } + result.metadata_.set(index, value); + return this; + } + public Builder setMetadata(int index, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) { + result.metadata_.set(index, builderForValue.build()); + return this; + } + public Builder addMetadata(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol value) { + if (value == null) { + throw new NullPointerException(); + } + if (result.metadata_.isEmpty()) { + result.metadata_ = new java.util.ArrayList(); + } + result.metadata_.add(value); + return this; + } + public Builder addMetadata(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) { + if (result.metadata_.isEmpty()) { + result.metadata_ = new java.util.ArrayList(); + } + result.metadata_.add(builderForValue.build()); + return this; + } + public Builder addAllMetadata( + java.lang.Iterable values) { + if (result.metadata_.isEmpty()) { + result.metadata_ = new java.util.ArrayList(); + } + super.addAll(values, result.metadata_); + return this; + } + public Builder clearMetadata() { + result.metadata_ = java.util.Collections.emptyList(); + return this; + } + // @@protoc_insertion_point(builder_scope:RemoteReplyProtocol) } - + static { defaultInstance = new RemoteReplyProtocol(true); se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:RemoteReplyProtocol) } - + + public static final class UuidProtocol extends + com.google.protobuf.GeneratedMessage { + // Use UuidProtocol.newBuilder() to construct. + private UuidProtocol() { + initFields(); + } + private UuidProtocol(boolean noInit) {} + + private static final UuidProtocol defaultInstance; + public static UuidProtocol getDefaultInstance() { + return defaultInstance; + } + + public UuidProtocol getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_UuidProtocol_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_UuidProtocol_fieldAccessorTable; + } + + // required uint64 high = 1; + public static final int HIGH_FIELD_NUMBER = 1; + private boolean hasHigh; + private long high_ = 0L; + public boolean hasHigh() { return hasHigh; } + public long getHigh() { return high_; } + + // required uint64 low = 2; + public static final int LOW_FIELD_NUMBER = 2; + private boolean hasLow; + private long low_ = 0L; + public boolean hasLow() { return hasLow; } + public long getLow() { return low_; } + + private void initFields() { + } + public final boolean isInitialized() { + if (!hasHigh) return false; + if (!hasLow) return false; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (hasHigh()) { + output.writeUInt64(1, getHigh()); + } + if (hasLow()) { + output.writeUInt64(2, getLow()); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (hasHigh()) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(1, getHigh()); + } + if (hasLow()) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(2, getLow()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder { + private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol result; + + // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder() + private Builder() {} + + private static Builder create() { + Builder builder = new Builder(); + builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol(); + return builder; + } + + protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol internalGetResult() { + return result; + } + + public Builder clear() { + if (result == null) { + throw new IllegalStateException( + "Cannot call clear() after build()."); + } + result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol(); + return this; + } + + public Builder clone() { + return create().mergeFrom(result); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDescriptor(); + } + + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getDefaultInstanceForType() { + return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + } + + public boolean isInitialized() { + return result.isInitialized(); + } + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol build() { + if (result != null && !isInitialized()) { + throw newUninitializedMessageException(result); + } + return buildPartial(); + } + + private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + if (!isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return buildPartial(); + } + + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol buildPartial() { + if (result == null) { + throw new IllegalStateException( + "build() has already been called on this Builder."); + } + se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol returnMe = result; + result = null; + return returnMe; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol) { + return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol other) { + if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) return this; + if (other.hasHigh()) { + setHigh(other.getHigh()); + } + if (other.hasLow()) { + setLow(other.getLow()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + return this; + } + break; + } + case 8: { + setHigh(input.readUInt64()); + break; + } + case 16: { + setLow(input.readUInt64()); + break; + } + } + } + } + + + // required uint64 high = 1; + public boolean hasHigh() { + return result.hasHigh(); + } + public long getHigh() { + return result.getHigh(); + } + public Builder setHigh(long value) { + result.hasHigh = true; + result.high_ = value; + return this; + } + public Builder clearHigh() { + result.hasHigh = false; + result.high_ = 0L; + return this; + } + + // required uint64 low = 2; + public boolean hasLow() { + return result.hasLow(); + } + public long getLow() { + return result.getLow(); + } + public Builder setLow(long value) { + result.hasLow = true; + result.low_ = value; + return this; + } + public Builder clearLow() { + result.hasLow = false; + result.low_ = 0L; + return this; + } + + // @@protoc_insertion_point(builder_scope:UuidProtocol) + } + + static { + defaultInstance = new UuidProtocol(true); + se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:UuidProtocol) + } + + public static final class MetadataEntryProtocol extends + com.google.protobuf.GeneratedMessage { + // Use MetadataEntryProtocol.newBuilder() to construct. + private MetadataEntryProtocol() { + initFields(); + } + private MetadataEntryProtocol(boolean noInit) {} + + private static final MetadataEntryProtocol defaultInstance; + public static MetadataEntryProtocol getDefaultInstance() { + return defaultInstance; + } + + public MetadataEntryProtocol getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MetadataEntryProtocol_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MetadataEntryProtocol_fieldAccessorTable; + } + + // required string key = 1; + public static final int KEY_FIELD_NUMBER = 1; + private boolean hasKey; + private java.lang.String key_ = ""; + public boolean hasKey() { return hasKey; } + public java.lang.String getKey() { return key_; } + + // required bytes value = 2; + public static final int VALUE_FIELD_NUMBER = 2; + private boolean hasValue; + private com.google.protobuf.ByteString value_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasValue() { return hasValue; } + public com.google.protobuf.ByteString getValue() { return value_; } + + private void initFields() { + } + public final boolean isInitialized() { + if (!hasKey) return false; + if (!hasValue) return false; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (hasKey()) { + output.writeString(1, getKey()); + } + if (hasValue()) { + output.writeBytes(2, getValue()); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (hasKey()) { + size += com.google.protobuf.CodedOutputStream + .computeStringSize(1, getKey()); + } + if (hasValue()) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, getValue()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder { + private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol result; + + // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.newBuilder() + private Builder() {} + + private static Builder create() { + Builder builder = new Builder(); + builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol(); + return builder; + } + + protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol internalGetResult() { + return result; + } + + public Builder clear() { + if (result == null) { + throw new IllegalStateException( + "Cannot call clear() after build()."); + } + result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol(); + return this; + } + + public Builder clone() { + return create().mergeFrom(result); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.getDescriptor(); + } + + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getDefaultInstanceForType() { + return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.getDefaultInstance(); + } + + public boolean isInitialized() { + return result.isInitialized(); + } + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol build() { + if (result != null && !isInitialized()) { + throw newUninitializedMessageException(result); + } + return buildPartial(); + } + + private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + if (!isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return buildPartial(); + } + + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol buildPartial() { + if (result == null) { + throw new IllegalStateException( + "build() has already been called on this Builder."); + } + se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol returnMe = result; + result = null; + return returnMe; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol) { + return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol other) { + if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.getDefaultInstance()) return this; + if (other.hasKey()) { + setKey(other.getKey()); + } + if (other.hasValue()) { + setValue(other.getValue()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + return this; + } + break; + } + case 10: { + setKey(input.readString()); + break; + } + case 18: { + setValue(input.readBytes()); + break; + } + } + } + } + + + // required string key = 1; + public boolean hasKey() { + return result.hasKey(); + } + public java.lang.String getKey() { + return result.getKey(); + } + public Builder setKey(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + result.hasKey = true; + result.key_ = value; + return this; + } + public Builder clearKey() { + result.hasKey = false; + result.key_ = getDefaultInstance().getKey(); + return this; + } + + // required bytes value = 2; + public boolean hasValue() { + return result.hasValue(); + } + public com.google.protobuf.ByteString getValue() { + return result.getValue(); + } + public Builder setValue(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + result.hasValue = true; + result.value_ = value; + return this; + } + public Builder clearValue() { + result.hasValue = false; + result.value_ = getDefaultInstance().getValue(); + return this; + } + + // @@protoc_insertion_point(builder_scope:MetadataEntryProtocol) + } + + static { + defaultInstance = new MetadataEntryProtocol(true); + se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:MetadataEntryProtocol) + } + public static final class LifeCycleProtocol extends com.google.protobuf.GeneratedMessage { // Use LifeCycleProtocol.newBuilder() to construct. @@ -3872,61 +4748,33 @@ public final class RemoteProtocol { initFields(); } private LifeCycleProtocol(boolean noInit) {} - + private static final LifeCycleProtocol defaultInstance; public static LifeCycleProtocol getDefaultInstance() { return defaultInstance; } - + public LifeCycleProtocol getDefaultInstanceForType() { return defaultInstance; } - + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_LifeCycleProtocol_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_LifeCycleProtocol_fieldAccessorTable; } - + // required .LifeCycleType lifeCycle = 1; public static final int LIFECYCLE_FIELD_NUMBER = 1; private boolean hasLifeCycle; private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType lifeCycle_; public boolean hasLifeCycle() { return hasLifeCycle; } public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle() { return lifeCycle_; } - - // optional string preRestart = 2; - public static final int PRERESTART_FIELD_NUMBER = 2; - private boolean hasPreRestart; - private java.lang.String preRestart_ = ""; - public boolean hasPreRestart() { return hasPreRestart; } - public java.lang.String getPreRestart() { return preRestart_; } - - // optional string postRestart = 3; - public static final int POSTRESTART_FIELD_NUMBER = 3; - private boolean hasPostRestart; - private java.lang.String postRestart_ = ""; - public boolean hasPostRestart() { return hasPostRestart; } - public java.lang.String getPostRestart() { return postRestart_; } - - // optional string init = 4; - public static final int INIT_FIELD_NUMBER = 4; - private boolean hasInit; - private java.lang.String init_ = ""; - public boolean hasInit() { return hasInit; } - public java.lang.String getInit() { return init_; } - - // optional string shutdown = 5; - public static final int SHUTDOWN_FIELD_NUMBER = 5; - private boolean hasShutdown; - private java.lang.String shutdown_ = ""; - public boolean hasShutdown() { return hasShutdown; } - public java.lang.String getShutdown() { return shutdown_; } - + private void initFields() { lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT; } @@ -3934,59 +4782,31 @@ public final class RemoteProtocol { if (!hasLifeCycle) return false; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); if (hasLifeCycle()) { output.writeEnum(1, getLifeCycle().getNumber()); } - if (hasPreRestart()) { - output.writeString(2, getPreRestart()); - } - if (hasPostRestart()) { - output.writeString(3, getPostRestart()); - } - if (hasInit()) { - output.writeString(4, getInit()); - } - if (hasShutdown()) { - output.writeString(5, getShutdown()); - } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (hasLifeCycle()) { size += com.google.protobuf.CodedOutputStream .computeEnumSize(1, getLifeCycle().getNumber()); } - if (hasPreRestart()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(2, getPreRestart()); - } - if (hasPostRestart()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(3, getPostRestart()); - } - if (hasInit()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(4, getInit()); - } - if (hasShutdown()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(5, getShutdown()); - } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } - + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -4053,31 +4873,31 @@ public final class RemoteProtocol { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder { private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol result; - + // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder() private Builder() {} - + private static Builder create() { Builder builder = new Builder(); builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol(); return builder; } - + protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol internalGetResult() { return result; } - + public Builder clear() { if (result == null) { throw new IllegalStateException( @@ -4086,20 +4906,20 @@ public final class RemoteProtocol { result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol(); return this; } - + public Builder clone() { return create().mergeFrom(result); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDescriptor(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getDefaultInstanceForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); } - + public boolean isInitialized() { return result.isInitialized(); } @@ -4109,7 +4929,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { @@ -4118,7 +4938,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol buildPartial() { if (result == null) { throw new IllegalStateException( @@ -4128,7 +4948,7 @@ public final class RemoteProtocol { result = null; return returnMe; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol) { return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol)other); @@ -4137,28 +4957,16 @@ public final class RemoteProtocol { return this; } } - + public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol other) { if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance()) return this; if (other.hasLifeCycle()) { setLifeCycle(other.getLifeCycle()); } - if (other.hasPreRestart()) { - setPreRestart(other.getPreRestart()); - } - if (other.hasPostRestart()) { - setPostRestart(other.getPostRestart()); - } - if (other.hasInit()) { - setInit(other.getInit()); - } - if (other.hasShutdown()) { - setShutdown(other.getShutdown()); - } this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -4190,27 +4998,11 @@ public final class RemoteProtocol { } break; } - case 18: { - setPreRestart(input.readString()); - break; - } - case 26: { - setPostRestart(input.readString()); - break; - } - case 34: { - setInit(input.readString()); - break; - } - case 42: { - setShutdown(input.readString()); - break; - } } } } - - + + // required .LifeCycleType lifeCycle = 1; public boolean hasLifeCycle() { return result.hasLifeCycle(); @@ -4231,103 +5023,19 @@ public final class RemoteProtocol { result.lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT; return this; } - - // optional string preRestart = 2; - public boolean hasPreRestart() { - return result.hasPreRestart(); - } - public java.lang.String getPreRestart() { - return result.getPreRestart(); - } - public Builder setPreRestart(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasPreRestart = true; - result.preRestart_ = value; - return this; - } - public Builder clearPreRestart() { - result.hasPreRestart = false; - result.preRestart_ = getDefaultInstance().getPreRestart(); - return this; - } - - // optional string postRestart = 3; - public boolean hasPostRestart() { - return result.hasPostRestart(); - } - public java.lang.String getPostRestart() { - return result.getPostRestart(); - } - public Builder setPostRestart(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasPostRestart = true; - result.postRestart_ = value; - return this; - } - public Builder clearPostRestart() { - result.hasPostRestart = false; - result.postRestart_ = getDefaultInstance().getPostRestart(); - return this; - } - - // optional string init = 4; - public boolean hasInit() { - return result.hasInit(); - } - public java.lang.String getInit() { - return result.getInit(); - } - public Builder setInit(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasInit = true; - result.init_ = value; - return this; - } - public Builder clearInit() { - result.hasInit = false; - result.init_ = getDefaultInstance().getInit(); - return this; - } - - // optional string shutdown = 5; - public boolean hasShutdown() { - return result.hasShutdown(); - } - public java.lang.String getShutdown() { - return result.getShutdown(); - } - public Builder setShutdown(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasShutdown = true; - result.shutdown_ = value; - return this; - } - public Builder clearShutdown() { - result.hasShutdown = false; - result.shutdown_ = getDefaultInstance().getShutdown(); - return this; - } - + // @@protoc_insertion_point(builder_scope:LifeCycleProtocol) } - + static { defaultInstance = new LifeCycleProtocol(true); se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:LifeCycleProtocol) } - + public static final class AddressProtocol extends com.google.protobuf.GeneratedMessage { // Use AddressProtocol.newBuilder() to construct. @@ -4335,40 +5043,40 @@ public final class RemoteProtocol { initFields(); } private AddressProtocol(boolean noInit) {} - + private static final AddressProtocol defaultInstance; public static AddressProtocol getDefaultInstance() { return defaultInstance; } - + public AddressProtocol getDefaultInstanceForType() { return defaultInstance; } - + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_AddressProtocol_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_AddressProtocol_fieldAccessorTable; } - + // required string hostname = 1; public static final int HOSTNAME_FIELD_NUMBER = 1; private boolean hasHostname; private java.lang.String hostname_ = ""; public boolean hasHostname() { return hasHostname; } public java.lang.String getHostname() { return hostname_; } - + // required uint32 port = 2; public static final int PORT_FIELD_NUMBER = 2; private boolean hasPort; private int port_ = 0; public boolean hasPort() { return hasPort; } public int getPort() { return port_; } - + private void initFields() { } public final boolean isInitialized() { @@ -4376,7 +5084,7 @@ public final class RemoteProtocol { if (!hasPort) return false; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -4388,12 +5096,12 @@ public final class RemoteProtocol { } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (hasHostname()) { size += com.google.protobuf.CodedOutputStream @@ -4407,7 +5115,7 @@ public final class RemoteProtocol { memoizedSerializedSize = size; return size; } - + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -4474,31 +5182,31 @@ public final class RemoteProtocol { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder { private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol result; - + // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder() private Builder() {} - + private static Builder create() { Builder builder = new Builder(); builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol(); return builder; } - + protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol internalGetResult() { return result; } - + public Builder clear() { if (result == null) { throw new IllegalStateException( @@ -4507,20 +5215,20 @@ public final class RemoteProtocol { result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol(); return this; } - + public Builder clone() { return create().mergeFrom(result); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDescriptor(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getDefaultInstanceForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); } - + public boolean isInitialized() { return result.isInitialized(); } @@ -4530,7 +5238,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { @@ -4539,7 +5247,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol buildPartial() { if (result == null) { throw new IllegalStateException( @@ -4549,7 +5257,7 @@ public final class RemoteProtocol { result = null; return returnMe; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol) { return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol)other); @@ -4558,7 +5266,7 @@ public final class RemoteProtocol { return this; } } - + public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol other) { if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) return this; if (other.hasHostname()) { @@ -4570,7 +5278,7 @@ public final class RemoteProtocol { this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -4603,8 +5311,8 @@ public final class RemoteProtocol { } } } - - + + // required string hostname = 1; public boolean hasHostname() { return result.hasHostname(); @@ -4625,7 +5333,7 @@ public final class RemoteProtocol { result.hostname_ = getDefaultInstance().getHostname(); return this; } - + // required uint32 port = 2; public boolean hasPort() { return result.hasPort(); @@ -4643,19 +5351,19 @@ public final class RemoteProtocol { result.port_ = 0; return this; } - + // @@protoc_insertion_point(builder_scope:AddressProtocol) } - + static { defaultInstance = new AddressProtocol(true); se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:AddressProtocol) } - + public static final class ExceptionProtocol extends com.google.protobuf.GeneratedMessage { // Use ExceptionProtocol.newBuilder() to construct. @@ -4663,40 +5371,40 @@ public final class RemoteProtocol { initFields(); } private ExceptionProtocol(boolean noInit) {} - + private static final ExceptionProtocol defaultInstance; public static ExceptionProtocol getDefaultInstance() { return defaultInstance; } - + public ExceptionProtocol getDefaultInstanceForType() { return defaultInstance; } - + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ExceptionProtocol_descriptor; } - + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ExceptionProtocol_fieldAccessorTable; } - + // required string classname = 1; public static final int CLASSNAME_FIELD_NUMBER = 1; private boolean hasClassname; private java.lang.String classname_ = ""; public boolean hasClassname() { return hasClassname; } public java.lang.String getClassname() { return classname_; } - + // required string message = 2; public static final int MESSAGE_FIELD_NUMBER = 2; private boolean hasMessage; private java.lang.String message_ = ""; public boolean hasMessage() { return hasMessage; } public java.lang.String getMessage() { return message_; } - + private void initFields() { } public final boolean isInitialized() { @@ -4704,7 +5412,7 @@ public final class RemoteProtocol { if (!hasMessage) return false; return true; } - + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); @@ -4716,12 +5424,12 @@ public final class RemoteProtocol { } getUnknownFields().writeTo(output); } - + private int memoizedSerializedSize = -1; public int getSerializedSize() { int size = memoizedSerializedSize; if (size != -1) return size; - + size = 0; if (hasClassname()) { size += com.google.protobuf.CodedOutputStream @@ -4735,7 +5443,7 @@ public final class RemoteProtocol { memoizedSerializedSize = size; return size; } - + public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -4802,31 +5510,31 @@ public final class RemoteProtocol { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - + public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } - + public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder { private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol result; - + // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder() private Builder() {} - + private static Builder create() { Builder builder = new Builder(); builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol(); return builder; } - + protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol internalGetResult() { return result; } - + public Builder clear() { if (result == null) { throw new IllegalStateException( @@ -4835,20 +5543,20 @@ public final class RemoteProtocol { result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol(); return this; } - + public Builder clone() { return create().mergeFrom(result); } - + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDescriptor(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getDefaultInstanceForType() { return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); } - + public boolean isInitialized() { return result.isInitialized(); } @@ -4858,7 +5566,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { @@ -4867,7 +5575,7 @@ public final class RemoteProtocol { } return buildPartial(); } - + public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol buildPartial() { if (result == null) { throw new IllegalStateException( @@ -4877,7 +5585,7 @@ public final class RemoteProtocol { result = null; return returnMe; } - + public Builder mergeFrom(com.google.protobuf.Message other) { if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol) { return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol)other); @@ -4886,7 +5594,7 @@ public final class RemoteProtocol { return this; } } - + public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol other) { if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance()) return this; if (other.hasClassname()) { @@ -4898,7 +5606,7 @@ public final class RemoteProtocol { this.mergeUnknownFields(other.getUnknownFields()); return this; } - + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -4931,8 +5639,8 @@ public final class RemoteProtocol { } } } - - + + // required string classname = 1; public boolean hasClassname() { return result.hasClassname(); @@ -4953,7 +5661,7 @@ public final class RemoteProtocol { result.classname_ = getDefaultInstance().getClassname(); return this; } - + // required string message = 2; public boolean hasMessage() { return result.hasMessage(); @@ -4974,19 +5682,19 @@ public final class RemoteProtocol { result.message_ = getDefaultInstance().getMessage(); return this; } - + // @@protoc_insertion_point(builder_scope:ExceptionProtocol) } - + static { defaultInstance = new ExceptionProtocol(true); se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:ExceptionProtocol) } - + private static com.google.protobuf.Descriptors.Descriptor internal_static_RemoteActorRefProtocol_descriptor; private static @@ -5022,6 +5730,16 @@ public final class RemoteProtocol { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_RemoteReplyProtocol_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_UuidProtocol_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_UuidProtocol_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_MetadataEntryProtocol_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_MetadataEntryProtocol_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor internal_static_LifeCycleProtocol_descriptor; private static @@ -5037,7 +5755,7 @@ public final class RemoteProtocol { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_ExceptionProtocol_fieldAccessorTable; - + public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { return descriptor; @@ -5062,33 +5780,36 @@ public final class RemoteProtocol { "tProtocol\"r\n\017MessageProtocol\0225\n\023serializ" + "ationScheme\030\001 \002(\0162\030.SerializationSchemeT" + "ype\022\017\n\007message\030\002 \002(\014\022\027\n\017messageManifest\030" + - "\003 \001(\014\"\222\001\n\021ActorInfoProtocol\022\014\n\004uuid\030\001 \002(" + + "\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\tac" + "torType\030\004 \002(\0162\n.ActorType\022/\n\016typedActorI" + - "nfo\030\005 \001(\0132\027.TypedActorInfoProtocol\";\n\026Ty", - "pedActorInfoProtocol\022\021\n\tinterface\030\001 \002(\t\022" + - "\016\n\006method\030\002 \002(\t\"\300\001\n\025RemoteRequestProtoco" + - "l\022\n\n\002id\030\001 \002(\004\022!\n\007message\030\002 \002(\0132\020.Message" + - "Protocol\022%\n\tactorInfo\030\003 \002(\0132\022.ActorInfoP" + - "rotocol\022\020\n\010isOneWay\030\004 \002(\010\022\026\n\016supervisorU" + - "uid\030\005 \001(\t\022\'\n\006sender\030\006 \001(\0132\027.RemoteActorR" + - "efProtocol\"\252\001\n\023RemoteReplyProtocol\022\n\n\002id" + - "\030\001 \002(\004\022!\n\007message\030\002 \001(\0132\020.MessageProtoco" + - "l\022%\n\texception\030\003 \001(\0132\022.ExceptionProtocol" + - "\022\026\n\016supervisorUuid\030\004 \001(\t\022\017\n\007isActor\030\005 \002(", - "\010\022\024\n\014isSuccessful\030\006 \002(\010\"\177\n\021LifeCycleProt" + - "ocol\022!\n\tlifeCycle\030\001 \002(\0162\016.LifeCycleType\022" + - "\022\n\npreRestart\030\002 \001(\t\022\023\n\013postRestart\030\003 \001(\t" + - "\022\014\n\004init\030\004 \001(\t\022\020\n\010shutdown\030\005 \001(\t\"1\n\017Addr" + - "essProtocol\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\013SCA" + - "LA_ACTOR\020\001\022\016\n\nJAVA_ACTOR\020\002\022\017\n\013TYPED_ACTO" + - "R\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_JS", - "ON\020\004\022\014\n\010PROTOBUF\020\005*-\n\rLifeCycleType\022\r\n\tP" + - "ERMANENT\020\001\022\r\n\tTEMPORARY\020\002B-\n)se.scalable" + - "solutions.akka.remote.protocolH\001" + "nfo\030\005 \001(\0132\027.TypedActorInfoProtocol\022\n\n\002id", + "\030\006 \001(\t\";\n\026TypedActorInfoProtocol\022\021\n\tinte" + + "rface\030\001 \002(\t\022\016\n\006method\030\002 \002(\t\"\352\001\n\025RemoteRe" + + "questProtocol\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(\013" + + "2\026.MetadataEntryProtocol\"\324\001\n\023RemoteReply" + + "Protocol\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.Exc", + "eptionProtocol\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.MetadataEntryProtocol\"" + + ")\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\t" + + "lifeCycle\030\001 \002(\0162\016.LifeCycleType\"1\n\017Addre" + + "ssProtocol\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\013SCAL", + "A_ACTOR\020\001\022\016\n\nJAVA_ACTOR\020\002\022\017\n\013TYPED_ACTOR" + + "\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_JSO" + + "N\020\004\022\014\n\010PROTOBUF\020\005*-\n\rLifeCycleType\022\r\n\tPE" + + "RMANENT\020\001\022\r\n\tTEMPORARY\020\002B-\n)se.scalables" + + "olutions.akka.remote.protocolH\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -5124,7 +5845,7 @@ public final class RemoteProtocol { internal_static_ActorInfoProtocol_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ActorInfoProtocol_descriptor, - new java.lang.String[] { "Uuid", "Target", "Timeout", "ActorType", "TypedActorInfo", }, + new java.lang.String[] { "Uuid", "Target", "Timeout", "ActorType", "TypedActorInfo", "Id", }, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.class, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder.class); internal_static_TypedActorInfoProtocol_descriptor = @@ -5140,7 +5861,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", }, + new java.lang.String[] { "Id", "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 = @@ -5148,19 +5869,35 @@ 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", }, + new java.lang.String[] { "Id", "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_LifeCycleProtocol_descriptor = + internal_static_UuidProtocol_descriptor = getDescriptor().getMessageTypes().get(7); + internal_static_UuidProtocol_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_UuidProtocol_descriptor, + new java.lang.String[] { "High", "Low", }, + se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.class, + se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder.class); + internal_static_MetadataEntryProtocol_descriptor = + getDescriptor().getMessageTypes().get(8); + internal_static_MetadataEntryProtocol_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_MetadataEntryProtocol_descriptor, + new java.lang.String[] { "Key", "Value", }, + se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.class, + se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder.class); + internal_static_LifeCycleProtocol_descriptor = + getDescriptor().getMessageTypes().get(9); internal_static_LifeCycleProtocol_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_LifeCycleProtocol_descriptor, - new java.lang.String[] { "LifeCycle", "PreRestart", "PostRestart", "Init", "Shutdown", }, + new java.lang.String[] { "LifeCycle", }, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.class, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder.class); internal_static_AddressProtocol_descriptor = - getDescriptor().getMessageTypes().get(8); + getDescriptor().getMessageTypes().get(10); internal_static_AddressProtocol_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_AddressProtocol_descriptor, @@ -5168,7 +5905,7 @@ public final class RemoteProtocol { se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.class, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder.class); internal_static_ExceptionProtocol_descriptor = - getDescriptor().getMessageTypes().get(9); + getDescriptor().getMessageTypes().get(11); internal_static_ExceptionProtocol_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ExceptionProtocol_descriptor, @@ -5183,8 +5920,8 @@ public final class RemoteProtocol { new com.google.protobuf.Descriptors.FileDescriptor[] { }, assigner); } - + public static void internalForceInit() {} - + // @@protoc_insertion_point(outer_class_scope) } diff --git a/akka-remote/src/main/protocol/RemoteProtocol.proto b/akka-remote/src/main/protocol/RemoteProtocol.proto index 567bf54eba..4f2fa5c2dd 100644 --- a/akka-remote/src/main/protocol/RemoteProtocol.proto +++ b/akka-remote/src/main/protocol/RemoteProtocol.proto @@ -16,7 +16,7 @@ option optimize_for = SPEED; * on the original node. */ message RemoteActorRefProtocol { - required string uuid = 1; + required UuidProtocol uuid = 1; required string actorClassname = 2; required AddressProtocol homeAddress = 3; optional uint64 timeout = 4; @@ -28,7 +28,7 @@ message RemoteActorRefProtocol { * 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; @@ -56,11 +56,12 @@ 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; optional TypedActorInfoProtocol typedActorInfo = 5; + optional string id = 6; } /** @@ -75,24 +76,42 @@ 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 RemoteActorRefProtocol sender = 6; + repeated MetadataEntryProtocol metadata = 7; } /** * 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; required bool isActor = 5; required bool isSuccessful = 6; + repeated MetadataEntryProtocol metadata = 7; +} + +/** + * Defines a UUID. + */ +message UuidProtocol { + required uint64 high = 1; + required uint64 low = 2; +} + +/** + * Defines a meta data entry. + */ +message MetadataEntryProtocol { + required string key = 1; + required bytes value = 2; } /** @@ -154,4 +173,4 @@ message AddressProtocol { message ExceptionProtocol { required string classname = 1; required string message = 2; -} +} \ No newline at end of file diff --git a/akka-remote/src/main/scala/remote/Cluster.scala b/akka-remote/src/main/scala/remote/Cluster.scala index 8a5864a51b..6e1e99f0b2 100644 --- a/akka-remote/src/main/scala/remote/Cluster.scala +++ b/akka-remote/src/main/scala/remote/Cluster.scala @@ -35,6 +35,11 @@ trait Cluster { * The order of application is undefined and may vary */ def foreach(f: (RemoteAddress) => Unit): Unit + + /** + * Returns all the endpoints in the cluster. + */ + def endpoints: Array[RemoteAddress] } /** @@ -81,11 +86,11 @@ abstract class BasicClusterActor extends ClusterActor with Logging { @volatile private var local: Node = Node(Nil) @volatile private var remotes: Map[ADDR_T, Node] = Map() - override def init = { + override def preStart = { remotes = new HashMap[ADDR_T, Node] } - override def shutdown = { + override def postStop = { remotes = Map() } @@ -196,6 +201,11 @@ abstract class BasicClusterActor extends ClusterActor with Logging { * Applies the given function to all remote addresses known */ def foreach(f: (RemoteAddress) => Unit): Unit = remotes.valuesIterator.toList.flatMap(_.endpoints).foreach(f) + + /** + * Returns all the endpoints in the cluster. + */ + def endpoints: Array[RemoteAddress] = remotes.toArray.asInstanceOf[Array[RemoteAddress]] } /** @@ -211,7 +221,7 @@ object Cluster extends Cluster with Logging { lazy val DEFAULT_CLUSTER_ACTOR_CLASS_NAME = classOf[JGroupsClusterActor].getName @volatile private[remote] var clusterActorRef: Option[ActorRef] = None - @volatile private[akka] var classLoader : Option[ClassLoader] = Some(getClass.getClassLoader) + @volatile private[akka] var classLoader: Option[ClassLoader] = Some(getClass.getClassLoader) private[remote] def createClusterActor(): Option[ActorRef] = { val name = config.getString("akka.remote.cluster.actor", DEFAULT_CLUSTER_ACTOR_CLASS_NAME) @@ -233,7 +243,7 @@ object Cluster extends Cluster with Logging { RestartStrategy(OneForOne, 5, 1000, List(classOf[Exception])), Supervise(actor, LifeCycle(Permanent)) :: Nil))) - private[this] def clusterActor = if(clusterActorRef.isEmpty) None else Some(clusterActorRef.get.actor.asInstanceOf[ClusterActor]) + private[this] def clusterActor = if (clusterActorRef.isEmpty) None else Some(clusterActorRef.get.actor.asInstanceOf[ClusterActor]) def name = clusterActor.map(_.name).getOrElse("No cluster") @@ -257,6 +267,10 @@ object Cluster extends Cluster with Logging { def foreach(f: (RemoteAddress) => Unit): Unit = clusterActor.foreach(_.foreach(f)) + def endpoints: Array[RemoteAddress] = clusterActor + .getOrElse(throw new IllegalStateException("No cluster actor is defined")) + .endpoints + def start(): Unit = start(None) def start(serializerClassLoader: Option[ClassLoader]): Unit = synchronized { diff --git a/akka-remote/src/main/scala/remote/JGroupsClusterActor.scala b/akka-remote/src/main/scala/remote/JGroupsClusterActor.scala index 54ef3807d4..07cbf4d65b 100644 --- a/akka-remote/src/main/scala/remote/JGroupsClusterActor.scala +++ b/akka-remote/src/main/scala/remote/JGroupsClusterActor.scala @@ -54,8 +54,8 @@ class JGroupsClusterActor extends BasicClusterActor { protected def toAllNodes(msg : Array[Byte]): Unit = for (c <- channel) c.send(new JG_MSG(null, null, msg)) - override def shutdown = { - super.shutdown + override def postStop = { + super.postStop log info ("Shutting down %s", toString) isActive = false channel.foreach(Util shutdown _) diff --git a/akka-remote/src/main/scala/remote/RemoteClient.scala b/akka-remote/src/main/scala/remote/RemoteClient.scala index 459c260a62..26cc275956 100644 --- a/akka-remote/src/main/scala/remote/RemoteClient.scala +++ b/akka-remote/src/main/scala/remote/RemoteClient.scala @@ -4,14 +4,14 @@ package se.scalablesolutions.akka.remote -import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._ -import se.scalablesolutions.akka.actor.{Exit, Actor, ActorRef, ActorType, RemoteActorRef, RemoteActorSerialization, IllegalActorStateException} +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.config.Config._ +import se.scalablesolutions.akka.serialization.RemoteActorSerialization._ import se.scalablesolutions.akka.AkkaException import Actor._ -import RemoteActorSerialization._ import org.jboss.netty.channel._ import group.DefaultChannelGroup @@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable.{HashSet, HashMap} import scala.reflect.BeanProperty +import se.scalablesolutions.akka.actor._ /** * Life-cycle events for RemoteClient. @@ -64,8 +65,6 @@ object RemoteClient extends Logging { private val remoteClients = new HashMap[String, RemoteClient] private val remoteActors = new HashMap[RemoteServer.Address, HashSet[String]] - // FIXME: simplify overloaded methods when we have Scala 2.8 - def actorFor(classNameOrServiceId: String, hostname: String, port: Int): ActorRef = actorFor(classNameOrServiceId, classNameOrServiceId, 5000L, hostname, port, None) @@ -87,6 +86,27 @@ object RemoteClient extends Logging { def actorFor(serviceId: String, className: String, timeout: Long, hostname: String, port: Int): ActorRef = RemoteActorRef(serviceId, className, hostname, port, timeout, None) + def typedActorFor[T](intfClass: Class[T], serviceIdOrClassName: String, hostname: String, port: Int) : T = { + typedActorFor(intfClass, serviceIdOrClassName, serviceIdOrClassName, 5000L, hostname, port, None) + } + + def typedActorFor[T](intfClass: Class[T], serviceIdOrClassName: String, timeout: Long, hostname: String, port: Int) : T = { + typedActorFor(intfClass, serviceIdOrClassName, serviceIdOrClassName, timeout, hostname, port, None) + } + + def typedActorFor[T](intfClass: Class[T], serviceIdOrClassName: String, timeout: Long, hostname: String, port: Int, loader: ClassLoader) : T = { + typedActorFor(intfClass, serviceIdOrClassName, serviceIdOrClassName, timeout, hostname, port, Some(loader)) + } + + def typedActorFor[T](intfClass: Class[T], serviceId: String, implClassName: String, timeout: Long, hostname: String, port: Int, loader: ClassLoader) : T = { + typedActorFor(intfClass, serviceId, implClassName, timeout, hostname, port, Some(loader)) + } + + private[akka] def typedActorFor[T](intfClass: Class[T], serviceId: String, implClassName: String, timeout: Long, hostname: String, port: Int, loader: Option[ClassLoader]) : T = { + val actorRef = RemoteActorRef(serviceId, implClassName, hostname, port, timeout, loader, ActorType.TypedActor) + TypedActor.createProxyForRemoteActorRef(intfClass, actorRef) + } + private[akka] def actorFor(serviceId: String, className: String, timeout: Long, hostname: String, port: Int, loader: ClassLoader): ActorRef = RemoteActorRef(serviceId, className, hostname, port, timeout, Some(loader)) @@ -173,7 +193,7 @@ class RemoteClient private[akka] ( extends Logging with ListenerManagement { val name = "RemoteClient@" + hostname + "::" + port - //FIXME Should these be clear:ed on shutdown? + //FIXME Should these be clear:ed on postStop? private val futures = new ConcurrentHashMap[Long, CompletableFuture[_]] private val supervisors = new ConcurrentHashMap[String, ActorRef] @@ -208,10 +228,10 @@ class RemoteClient private[akka] ( val channel = connection.awaitUninterruptibly.getChannel openChannels.add(channel) if (!connection.isSuccess) { - foreachListener(_ ! RemoteClientError(connection.getCause, this)) + notifyListeners(RemoteClientError(connection.getCause, this)) log.error(connection.getCause, "Remote client connection to [%s:%s] has failed", hostname, port) } - foreachListener(_ ! RemoteClientStarted(this)) + notifyListeners(RemoteClientStarted(this)) isRunning = true } } @@ -220,7 +240,7 @@ class RemoteClient private[akka] ( log.info("Shutting down %s", name) if (isRunning) { isRunning = false - foreachListener(_ ! RemoteClientShutdown(this)) + notifyListeners(RemoteClientShutdown(this)) timer.stop timer = null openChannels.close.awaitUninterruptibly @@ -238,7 +258,7 @@ class RemoteClient private[akka] ( @deprecated("Use removeListener instead") def deregisterListener(actorRef: ActorRef) = removeListener(actorRef) - override def foreachListener(f: (ActorRef) => Unit): Unit = super.foreachListener(f) + override def notifyListeners(message: => Any): Unit = super.notifyListeners(message) protected override def manageLifeCycleOfListeners = false @@ -275,7 +295,7 @@ class RemoteClient private[akka] ( } else { val exception = new RemoteClientException( "Remote client is not running, make sure you have invoked 'RemoteClient.connect' before using it.", this) - foreachListener(l => l ! RemoteClientError(exception, this)) + notifyListeners(RemoteClientError(exception, this)) throw exception } @@ -391,12 +411,12 @@ class RemoteClientHandler( futures.remove(reply.getId) } else { val exception = new RemoteClientException("Unknown message received in remote client handler: " + result, client) - client.foreachListener(_ ! RemoteClientError(exception, client)) + client.notifyListeners(RemoteClientError(exception, client)) throw exception } } catch { case e: Exception => - client.foreachListener(_ ! RemoteClientError(e, client)) + client.notifyListeners(RemoteClientError(e, client)) log.error("Unexpected exception in remote client handler: %s", e) throw e } @@ -411,7 +431,7 @@ class RemoteClientHandler( client.connection = bootstrap.connect(remoteAddress) client.connection.awaitUninterruptibly // Wait until the connection attempt succeeds or fails. if (!client.connection.isSuccess) { - client.foreachListener(_ ! RemoteClientError(client.connection.getCause, client)) + client.notifyListeners(RemoteClientError(client.connection.getCause, client)) log.error(client.connection.getCause, "Reconnection to [%s] has failed", remoteAddress) } } @@ -421,7 +441,7 @@ class RemoteClientHandler( override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { def connect = { - client.foreachListener(_ ! RemoteClientConnected(client)) + client.notifyListeners(RemoteClientConnected(client)) log.debug("Remote client connected to [%s]", ctx.getChannel.getRemoteAddress) client.resetReconnectionTimeWindow } @@ -438,12 +458,12 @@ class RemoteClientHandler( } override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { - client.foreachListener(_ ! RemoteClientDisconnected(client)) + client.notifyListeners(RemoteClientDisconnected(client)) log.debug("Remote client disconnected from [%s]", ctx.getChannel.getRemoteAddress) } override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { - client.foreachListener(_ ! RemoteClientError(event.getCause, client)) + client.notifyListeners(RemoteClientError(event.getCause, client)) log.error(event.getCause, "Unexpected exception from downstream in remote client") event.getChannel.close } diff --git a/akka-remote/src/main/scala/remote/RemoteServer.scala b/akka-remote/src/main/scala/remote/RemoteServer.scala index 2ce24b3fff..c1f25b6d4f 100644 --- a/akka-remote/src/main/scala/remote/RemoteServer.scala +++ b/akka-remote/src/main/scala/remote/RemoteServer.scala @@ -9,12 +9,15 @@ import java.net.InetSocketAddress import java.util.concurrent.{ConcurrentHashMap, Executors} import java.util.{Map => JMap} -import se.scalablesolutions.akka.actor._ +import se.scalablesolutions.akka.actor.{ + Actor, TypedActor, ActorRef, IllegalActorStateException, RemoteActorSystemMessage} import se.scalablesolutions.akka.actor.Actor._ import se.scalablesolutions.akka.util._ import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._ import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType._ import se.scalablesolutions.akka.config.Config._ +import se.scalablesolutions.akka.serialization.RemoteActorSerialization +import se.scalablesolutions.akka.serialization.RemoteActorSerialization._ import org.jboss.netty.bootstrap.ServerBootstrap import org.jboss.netty.channel._ @@ -27,6 +30,7 @@ import org.jboss.netty.handler.ssl.SslHandler import scala.collection.mutable.Map import scala.reflect.BeanProperty +import se.scalablesolutions.akka.dispatch.{DefaultCompletableFuture, CompletableFuture} /** * Use this object if you need a single remote server on a specific node. @@ -130,8 +134,8 @@ object RemoteServer { actorsFor(RemoteServer.Address(address.getHostName, address.getPort)).actors.put(uuid, actor) } - private[akka] def registerTypedActor(address: InetSocketAddress, name: String, typedActor: AnyRef) = guard.withWriteGuard { - actorsFor(RemoteServer.Address(address.getHostName, address.getPort)).typedActors.put(name, typedActor) + private[akka] def registerTypedActor(address: InetSocketAddress, uuid: String, typedActor: AnyRef) = guard.withWriteGuard { + actorsFor(RemoteServer.Address(address.getHostName, address.getPort)).typedActors.put(uuid, typedActor) } private[akka] def getOrCreateServer(address: InetSocketAddress): RemoteServer = guard.withWriteGuard { @@ -190,8 +194,10 @@ case class RemoteServerClientDisconnected(@BeanProperty val server: RemoteServer class RemoteServer extends Logging with ListenerManagement { def name = "RemoteServer@" + hostname + ":" + port - private[akka] var hostname = RemoteServer.HOSTNAME - private[akka] var port = RemoteServer.PORT + private[akka] var address = RemoteServer.Address(RemoteServer.HOSTNAME,RemoteServer.PORT) + + def hostname = address.hostname + def port = address.port @volatile private var _isRunning = false @@ -227,13 +233,11 @@ class RemoteServer extends Logging with ListenerManagement { private def start(_hostname: String, _port: Int, loader: Option[ClassLoader]): RemoteServer = synchronized { try { if (!_isRunning) { - hostname = _hostname - port = _port + address = RemoteServer.Address(_hostname,_port) log.info("Starting remote server at [%s:%s]", hostname, port) RemoteServer.register(hostname, port, this) - val remoteActorSet = RemoteServer.actorsFor(RemoteServer.Address(hostname, port)) val pipelineFactory = new RemoteServerPipelineFactory( - name, openChannels, loader, remoteActorSet.actors, remoteActorSet.typedActors,this) + name, openChannels, loader, this) bootstrap.setPipelineFactory(pipelineFactory) bootstrap.setOption("child.tcpNoDelay", true) bootstrap.setOption("child.keepAlive", true) @@ -242,12 +246,12 @@ class RemoteServer extends Logging with ListenerManagement { openChannels.add(bootstrap.bind(new InetSocketAddress(hostname, port))) _isRunning = true Cluster.registerLocalNode(hostname, port) - foreachListener(_ ! RemoteServerStarted(this)) + notifyListeners(RemoteServerStarted(this)) } } catch { case e => log.error(e, "Could not start up remote server") - foreachListener(_ ! RemoteServerError(e, this)) + notifyListeners(RemoteServerError(e, this)) } this } @@ -260,7 +264,7 @@ class RemoteServer extends Logging with ListenerManagement { openChannels.close.awaitUninterruptibly bootstrap.releaseExternalResources Cluster.deregisterLocalNode(hostname, port) - foreachListener(_ ! RemoteServerShutdown(this)) + notifyListeners(RemoteServerShutdown(this)) } catch { case e: java.nio.channels.ClosedChannelException => {} case e => log.warning("Could not close remote server channel in a graceful way") @@ -268,12 +272,28 @@ class RemoteServer extends Logging with ListenerManagement { } } - // TODO: register typed actor in RemoteServer as well + /** + * Register typed actor by interface name. + */ + def registerTypedActor(intfClass: Class[_], typedActor: AnyRef) : Unit = registerTypedActor(intfClass.getName, typedActor) + + /** + * Register remote typed actor by a specific id. + * @param id custom actor id + * @param typedActor typed actor to register + */ + def registerTypedActor(id: String, typedActor: AnyRef): Unit = synchronized { + val typedActors = RemoteServer.actorsFor(RemoteServer.Address(hostname, port)).typedActors + if (!typedActors.contains(id)) { + log.debug("Registering server side remote actor [%s] with id [%s] on [%s:%d]", typedActor.getClass.getName, id, hostname, port) + typedActors.put(id, typedActor) + } + } /** * Register Remote Actor by the Actor's 'id' field. It starts the Actor if it is not started already. */ - def register(actorRef: ActorRef): Unit = register(actorRef.id,actorRef) + def register(actorRef: ActorRef): Unit = register(actorRef.id, actorRef) /** * Register Remote Actor by a specific 'id' passed as argument. @@ -282,11 +302,11 @@ class RemoteServer extends Logging with ListenerManagement { */ def register(id: String, actorRef: ActorRef): Unit = synchronized { if (_isRunning) { - val actors = RemoteServer.actorsFor(RemoteServer.Address(hostname, port)).actors - if (!actors.contains(id)) { + val actorMap = actors() + if (!actorMap.contains(id)) { if (!actorRef.isRunning) actorRef.start log.debug("Registering server side remote actor [%s] with id [%s]", actorRef.actorClass.getName, id) - actors.put(id, actorRef) + actorMap.put(id, actorRef) } } } @@ -296,10 +316,10 @@ 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]", actorRef.actorClass.getName, actorRef.id) - val actors = RemoteServer.actorsFor(RemoteServer.Address(hostname, port)).actors - actors.remove(actorRef.id) - if (actorRef.registeredInRemoteNodeDuringSerialization) actors.remove(actorRef.uuid) + 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) actorMap remove actorRef.uuid } } @@ -311,16 +331,32 @@ class RemoteServer extends Logging with ListenerManagement { def unregister(id: String):Unit = synchronized { if (_isRunning) { log.info("Unregistering server side remote actor with id [%s]", id) - val actors = RemoteServer.actorsFor(RemoteServer.Address(hostname, port)).actors - val actorRef = actors.get(id) - actors.remove(id) - if (actorRef.registeredInRemoteNodeDuringSerialization) actors.remove(actorRef.uuid) + val actorMap = actors() + val actorRef = actorMap get id + actorMap remove id + if (actorRef.registeredInRemoteNodeDuringSerialization) actorMap remove actorRef.uuid + } + } + + /** + * Unregister Remote Typed Actor by specific 'id'. + *

    + * NOTE: You need to call this method if you have registered an actor by a custom ID. + */ + def unregisterTypedActor(id: String):Unit = synchronized { + if (_isRunning) { + log.info("Unregistering server side remote typed actor with id [%s]", id) + val registeredTypedActors = typedActors() + registeredTypedActors.remove(id) } } protected override def manageLifeCycleOfListeners = false - protected[akka] override def foreachListener(f: (ActorRef) => Unit): Unit = super.foreachListener(f) + protected[akka] override def notifyListeners(message: => Any): Unit = super.notifyListeners(message) + + private[akka] def actors() = RemoteServer.actorsFor(address).actors + private[akka] def typedActors() = RemoteServer.actorsFor(address).typedActors } object RemoteServerSslContext { @@ -345,8 +381,6 @@ class RemoteServerPipelineFactory( val name: String, val openChannels: ChannelGroup, val loader: Option[ClassLoader], - val actors: JMap[String, ActorRef], - val typedActors: JMap[String, AnyRef], val server: RemoteServer) extends ChannelPipelineFactory { import RemoteServer._ @@ -370,7 +404,7 @@ class RemoteServerPipelineFactory( case _ => (join(), join()) } - val remoteServer = new RemoteServerHandler(name, openChannels, loader, actors, typedActors,server) + val remoteServer = new RemoteServerHandler(name, openChannels, loader, server) val stages = ssl ++ dec ++ join(lenDec, protobufDec) ++ enc ++ join(lenPrep, protobufEnc, remoteServer) new StaticChannelPipeline(stages: _*) } @@ -384,15 +418,13 @@ class RemoteServerHandler( val name: String, val openChannels: ChannelGroup, val applicationLoader: Option[ClassLoader], - val actors: JMap[String, ActorRef], - val typedActors: JMap[String, AnyRef], val server: RemoteServer) extends SimpleChannelUpstreamHandler with Logging { val AW_PROXY_PREFIX = "$$ProxiedByAW".intern applicationLoader.foreach(MessageSerializer.setClassLoader(_)) /** - * ChannelOpen overridden to store open channels for a clean shutdown of a RemoteServer. + * ChannelOpen overridden to store open channels for a clean postStop of a RemoteServer. * If a channel is closed before, it is automatically removed from the open channels group. */ override def channelOpen(ctx: ChannelHandlerContext, event: ChannelStateEvent) = openChannels.add(ctx.getChannel) @@ -407,18 +439,18 @@ class RemoteServerHandler( def operationComplete(future: ChannelFuture): Unit = { if (future.isSuccess) { openChannels.add(future.getChannel) - server.foreachListener(_ ! RemoteServerClientConnected(server)) + server.notifyListeners(RemoteServerClientConnected(server)) } else future.getChannel.close } }) } else { - server.foreachListener(_ ! RemoteServerClientConnected(server)) + server.notifyListeners(RemoteServerClientConnected(server)) } } override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { log.debug("Remote client disconnected from [%s]", server.name) - server.foreachListener(_ ! RemoteServerClientDisconnected(server)) + server.notifyListeners(RemoteServerClientDisconnected(server)) } override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = { @@ -440,7 +472,7 @@ class RemoteServerHandler( override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { log.error(event.getCause, "Unexpected exception from remote downstream") event.getChannel.close - server.foreachListener(_ ! RemoteServerError(event.getCause, server)) + server.notifyListeners(RemoteServerError(event.getCause, server)) } private def handleRemoteRequestProtocol(request: RemoteRequestProtocol, channel: Channel) = { @@ -467,27 +499,36 @@ class RemoteServerHandler( case RemoteActorSystemMessage.Stop => actorRef.stop case _ => // then match on user defined messages if (request.getIsOneWay) actorRef.!(message)(sender) - else { - try { - val resultOrNone = (actorRef.!!(message)(sender)).as[AnyRef] - val result = if (resultOrNone.isDefined) resultOrNone.get else null + else actorRef.postMessageToMailboxAndCreateFutureResultWithTimeout(message,request.getActorInfo.getTimeout,None,Some( + new DefaultCompletableFuture[AnyRef](request.getActorInfo.getTimeout){ + 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) - log.debug("Returning result from actor invocation [%s]", result) - val replyBuilder = RemoteReplyProtocol.newBuilder - .setId(request.getId) - .setMessage(MessageSerializer.serialize(result)) - .setIsSuccessful(true) - .setIsActor(true) + if (request.hasSupervisorUuid) replyBuilder.setSupervisorUuid(request.getSupervisorUuid) - if (request.hasSupervisorUuid) replyBuilder.setSupervisorUuid(request.getSupervisorUuid) - channel.write(replyBuilder.build) + try { + channel.write(replyBuilder.build) + } catch { + case e: Throwable => + server.notifyListeners(RemoteServerError(e, server)) + } + } - } catch { - case e: Throwable => - channel.write(createErrorReplyMessage(e, request, true)) - server.foreachListener(_ ! RemoteServerError(e, server)) - } + override def onCompleteException(exception: Throwable) { + try { + channel.write(createErrorReplyMessage(exception, request, true)) + } catch { + case e: Throwable => + server.notifyListeners(RemoteServerError(e, server)) + } + } } + )) } } @@ -517,13 +558,39 @@ class RemoteServerHandler( } catch { case e: InvocationTargetException => channel.write(createErrorReplyMessage(e.getCause, request, false)) - server.foreachListener(_ ! RemoteServerError(e, server)) + server.notifyListeners(RemoteServerError(e, server)) case e: Throwable => channel.write(createErrorReplyMessage(e, request, false)) - server.foreachListener(_ ! RemoteServerError(e, server)) + server.notifyListeners(RemoteServerError(e, server)) } } + /** + * Find a registered actor by ID (default) or UUID. + * Actors are registered by id apart from registering during serialization see SerializationProtocol. + */ + private def findActorByIdOrUuid(id: String, uuid: String) : ActorRef = { + val registeredActors = server.actors() + var actorRefOrNull = registeredActors get id + if (actorRefOrNull eq null) { + actorRefOrNull = registeredActors get uuid + } + actorRefOrNull + } + + /** + * Find a registered typed actor by ID (default) or UUID. + * Actors are registered by id apart from registering during serialization see SerializationProtocol. + */ + private def findTypedActorByIdOrUUid(id: String, uuid: String) : AnyRef = { + val registeredActors = server.typedActors() + var actorRefOrNull = registeredActors get id + if (actorRefOrNull eq null) { + actorRefOrNull = registeredActors get uuid + } + actorRefOrNull + } + /** * Creates a new instance of the actor with name, uuid and timeout specified as arguments. * @@ -533,10 +600,12 @@ class RemoteServerHandler( */ private def createActor(actorInfo: ActorInfoProtocol): ActorRef = { val uuid = actorInfo.getUuid + val id = actorInfo.getId + val name = actorInfo.getTarget val timeout = actorInfo.getTimeout - val actorRefOrNull = actors get uuid + val actorRefOrNull = findActorByIdOrUuid(id, uuid) if (actorRefOrNull eq null) { try { @@ -545,14 +614,15 @@ class RemoteServerHandler( else Class.forName(name) val actorRef = Actor.actorOf(clazz.newInstance.asInstanceOf[Actor]) actorRef.uuid = uuid + actorRef.id = id actorRef.timeout = timeout actorRef.remoteAddress = None - actors.put(uuid, actorRef) + server.actors.put(id, actorRef) // register by id actorRef } catch { case e => log.error(e, "Could not create remote actor instance") - server.foreachListener(_ ! RemoteServerError(e, server)) + server.notifyListeners(RemoteServerError(e, server)) throw e } } else actorRefOrNull @@ -560,7 +630,9 @@ class RemoteServerHandler( private def createTypedActor(actorInfo: ActorInfoProtocol): AnyRef = { val uuid = actorInfo.getUuid - val typedActorOrNull = typedActors get uuid + val id = actorInfo.getId + + val typedActorOrNull = findTypedActorByIdOrUUid(id, uuid) if (typedActorOrNull eq null) { val typedActorInfo = actorInfo.getTypedActorInfo @@ -577,12 +649,12 @@ class RemoteServerHandler( val newInstance = TypedActor.newInstance( interfaceClass, targetClass.asInstanceOf[Class[_ <: TypedActor]], actorInfo.getTimeout).asInstanceOf[AnyRef] - typedActors.put(uuid, newInstance) + server.typedActors.put(id, newInstance) // register by id newInstance } catch { case e => log.error(e, "Could not create remote typed actor instance") - server.foreachListener(_ ! RemoteServerError(e, server)) + server.notifyListeners(RemoteServerError(e, server)) throw e } } else typedActorOrNull diff --git a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala index 4050c2026f..bc1aa9052f 100644 --- a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala +++ b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala @@ -2,17 +2,18 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package se.scalablesolutions.akka.serialization -import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy} -import se.scalablesolutions.akka.config.ScalaConfig._ +import se.scalablesolutions.akka.actor.{Actor, ActorRef, LocalActorRef, RemoteActorRef, IllegalActorStateException, ActorType} import se.scalablesolutions.akka.stm.global._ import se.scalablesolutions.akka.stm.TransactionManagement._ import se.scalablesolutions.akka.stm.TransactionManagement import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._ import se.scalablesolutions.akka.remote.{RemoteServer, MessageSerializer} -import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType._ -import se.scalablesolutions.akka.serialization.Serializer +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 com.google.protobuf.ByteString import se.scalablesolutions.akka.util.UUID @@ -68,7 +69,7 @@ trait SerializerBasedActorFormat[T <: Actor] extends Format[T] { } /** - * Module for local actor serialization + * Module for local actor serialization. */ object ActorSerialization { @@ -250,6 +251,7 @@ object RemoteActorSerialization { val actorInfoBuilder = ActorInfoProtocol.newBuilder .setUuid(uuid) + .setId(actorRef.id) .setTarget(actorClassName) .setTimeout(timeout) diff --git a/akka-remote/src/test/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java b/akka-remote/src/test/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java deleted file mode 100644 index 0ab1a0aa10..0000000000 --- a/akka-remote/src/test/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java +++ /dev/null @@ -1,5190 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// source: RemoteProtocol.proto - -package se.scalablesolutions.akka.remote.protocol; - -public final class RemoteProtocol { - private RemoteProtocol() {} - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistry registry) { - } - public enum ActorType - implements com.google.protobuf.ProtocolMessageEnum { - SCALA_ACTOR(0, 1), - JAVA_ACTOR(1, 2), - TYPED_ACTOR(2, 3), - ; - - - public final int getNumber() { return value; } - - public static ActorType valueOf(int value) { - switch (value) { - case 1: return SCALA_ACTOR; - case 2: return JAVA_ACTOR; - case 3: return TYPED_ACTOR; - default: return null; - } - } - - public static com.google.protobuf.Internal.EnumLiteMap - internalGetValueMap() { - return internalValueMap; - } - private static com.google.protobuf.Internal.EnumLiteMap - internalValueMap = - new com.google.protobuf.Internal.EnumLiteMap() { - public ActorType findValueByNumber(int number) { - return ActorType.valueOf(number) - ; } - }; - - public final com.google.protobuf.Descriptors.EnumValueDescriptor - getValueDescriptor() { - return getDescriptor().getValues().get(index); - } - public final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptorForType() { - return getDescriptor(); - } - public static final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(0); - } - - private static final ActorType[] VALUES = { - SCALA_ACTOR, JAVA_ACTOR, TYPED_ACTOR, - }; - public static ActorType valueOf( - com.google.protobuf.Descriptors.EnumValueDescriptor desc) { - if (desc.getType() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "EnumValueDescriptor is not for this type."); - } - return VALUES[desc.getIndex()]; - } - private final int index; - private final int value; - private ActorType(int index, int value) { - this.index = index; - this.value = value; - } - - static { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor(); - } - - // @@protoc_insertion_point(enum_scope:ActorType) - } - - public enum SerializationSchemeType - implements com.google.protobuf.ProtocolMessageEnum { - JAVA(0, 1), - SBINARY(1, 2), - SCALA_JSON(2, 3), - JAVA_JSON(3, 4), - PROTOBUF(4, 5), - ; - - - public final int getNumber() { return value; } - - public static SerializationSchemeType valueOf(int value) { - switch (value) { - case 1: return JAVA; - case 2: return SBINARY; - case 3: return SCALA_JSON; - case 4: return JAVA_JSON; - case 5: return PROTOBUF; - default: return null; - } - } - - public static com.google.protobuf.Internal.EnumLiteMap - internalGetValueMap() { - return internalValueMap; - } - private static com.google.protobuf.Internal.EnumLiteMap - internalValueMap = - new com.google.protobuf.Internal.EnumLiteMap() { - public SerializationSchemeType findValueByNumber(int number) { - return SerializationSchemeType.valueOf(number) - ; } - }; - - public final com.google.protobuf.Descriptors.EnumValueDescriptor - getValueDescriptor() { - return getDescriptor().getValues().get(index); - } - public final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptorForType() { - return getDescriptor(); - } - public static final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(1); - } - - private static final SerializationSchemeType[] VALUES = { - JAVA, SBINARY, SCALA_JSON, JAVA_JSON, PROTOBUF, - }; - public static SerializationSchemeType valueOf( - com.google.protobuf.Descriptors.EnumValueDescriptor desc) { - if (desc.getType() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "EnumValueDescriptor is not for this type."); - } - return VALUES[desc.getIndex()]; - } - private final int index; - private final int value; - private SerializationSchemeType(int index, int value) { - this.index = index; - this.value = value; - } - - static { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor(); - } - - // @@protoc_insertion_point(enum_scope:SerializationSchemeType) - } - - public enum LifeCycleType - implements com.google.protobuf.ProtocolMessageEnum { - PERMANENT(0, 1), - TEMPORARY(1, 2), - ; - - - public final int getNumber() { return value; } - - public static LifeCycleType valueOf(int value) { - switch (value) { - case 1: return PERMANENT; - case 2: return TEMPORARY; - default: return null; - } - } - - public static com.google.protobuf.Internal.EnumLiteMap - internalGetValueMap() { - return internalValueMap; - } - private static com.google.protobuf.Internal.EnumLiteMap - internalValueMap = - new com.google.protobuf.Internal.EnumLiteMap() { - public LifeCycleType findValueByNumber(int number) { - return LifeCycleType.valueOf(number) - ; } - }; - - public final com.google.protobuf.Descriptors.EnumValueDescriptor - getValueDescriptor() { - return getDescriptor().getValues().get(index); - } - public final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptorForType() { - return getDescriptor(); - } - public static final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(2); - } - - private static final LifeCycleType[] VALUES = { - PERMANENT, TEMPORARY, - }; - public static LifeCycleType valueOf( - com.google.protobuf.Descriptors.EnumValueDescriptor desc) { - if (desc.getType() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "EnumValueDescriptor is not for this type."); - } - return VALUES[desc.getIndex()]; - } - private final int index; - private final int value; - private LifeCycleType(int index, int value) { - this.index = index; - this.value = value; - } - - static { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor(); - } - - // @@protoc_insertion_point(enum_scope:LifeCycleType) - } - - public static final class RemoteActorRefProtocol extends - com.google.protobuf.GeneratedMessage { - // Use RemoteActorRefProtocol.newBuilder() to construct. - private RemoteActorRefProtocol() { - initFields(); - } - private RemoteActorRefProtocol(boolean noInit) {} - - private static final RemoteActorRefProtocol defaultInstance; - public static RemoteActorRefProtocol getDefaultInstance() { - return defaultInstance; - } - - public RemoteActorRefProtocol getDefaultInstanceForType() { - return defaultInstance; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteActorRefProtocol_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - 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 actorClassname = 2; - public static final int ACTORCLASSNAME_FIELD_NUMBER = 2; - private boolean hasActorClassname; - private java.lang.String actorClassname_ = ""; - public boolean hasActorClassname() { return hasActorClassname; } - public java.lang.String getActorClassname() { return actorClassname_; } - - // required .AddressProtocol homeAddress = 3; - public static final int HOMEADDRESS_FIELD_NUMBER = 3; - private boolean hasHomeAddress; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol homeAddress_; - public boolean hasHomeAddress() { return hasHomeAddress; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getHomeAddress() { return homeAddress_; } - - // optional uint64 timeout = 4; - public static final int TIMEOUT_FIELD_NUMBER = 4; - private boolean hasTimeout; - private long timeout_ = 0L; - public boolean hasTimeout() { return hasTimeout; } - public long getTimeout() { return timeout_; } - - private void initFields() { - homeAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); - } - public final boolean isInitialized() { - if (!hasUuid) return false; - if (!hasActorClassname) return false; - if (!hasHomeAddress) return false; - if (!getHomeAddress().isInitialized()) return false; - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (hasUuid()) { - output.writeString(1, getUuid()); - } - if (hasActorClassname()) { - output.writeString(2, getActorClassname()); - } - if (hasHomeAddress()) { - output.writeMessage(3, getHomeAddress()); - } - if (hasTimeout()) { - output.writeUInt64(4, getTimeout()); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - if (hasUuid()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(1, getUuid()); - } - if (hasActorClassname()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(2, getActorClassname()); - } - if (hasHomeAddress()) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(3, getHomeAddress()); - } - if (hasTimeout()) { - size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(4, getTimeout()); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol result; - - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol(); - return builder; - } - - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol internalGetResult() { - return result; - } - - public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol(); - return this; - } - - public Builder clone() { - return create().mergeFrom(result); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDescriptor(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getDefaultInstanceForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); - } - - public boolean isInitialized() { - return result.isInitialized(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol build() { - if (result != null && !isInitialized()) { - throw newUninitializedMessageException(result); - } - return buildPartial(); - } - - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return buildPartial(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); - } - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol returnMe = result; - result = null; - return returnMe; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol) { - return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol)other); - } else { - super.mergeFrom(other); - return this; - } - } - - 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.hasActorClassname()) { - setActorClassname(other.getActorClassname()); - } - if (other.hasHomeAddress()) { - mergeHomeAddress(other.getHomeAddress()); - } - if (other.hasTimeout()) { - setTimeout(other.getTimeout()); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - return this; - } - break; - } - case 10: { - setUuid(input.readString()); - break; - } - case 18: { - setActorClassname(input.readString()); - break; - } - case 26: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(); - if (hasHomeAddress()) { - subBuilder.mergeFrom(getHomeAddress()); - } - input.readMessage(subBuilder, extensionRegistry); - setHomeAddress(subBuilder.buildPartial()); - break; - } - case 32: { - setTimeout(input.readUInt64()); - break; - } - } - } - } - - - // required string uuid = 1; - public boolean hasUuid() { - return result.hasUuid(); - } - public java.lang.String getUuid() { - return result.getUuid(); - } - public Builder setUuid(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasUuid = true; - result.uuid_ = value; - return this; - } - public Builder clearUuid() { - result.hasUuid = false; - result.uuid_ = getDefaultInstance().getUuid(); - return this; - } - - // required string actorClassname = 2; - public boolean hasActorClassname() { - return result.hasActorClassname(); - } - public java.lang.String getActorClassname() { - return result.getActorClassname(); - } - public Builder setActorClassname(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasActorClassname = true; - result.actorClassname_ = value; - return this; - } - public Builder clearActorClassname() { - result.hasActorClassname = false; - result.actorClassname_ = getDefaultInstance().getActorClassname(); - return this; - } - - // required .AddressProtocol homeAddress = 3; - public boolean hasHomeAddress() { - return result.hasHomeAddress(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getHomeAddress() { - return result.getHomeAddress(); - } - public Builder setHomeAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasHomeAddress = true; - result.homeAddress_ = value; - return this; - } - public Builder setHomeAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder builderForValue) { - result.hasHomeAddress = true; - result.homeAddress_ = builderForValue.build(); - return this; - } - public Builder mergeHomeAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol value) { - if (result.hasHomeAddress() && - result.homeAddress_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) { - result.homeAddress_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(result.homeAddress_).mergeFrom(value).buildPartial(); - } else { - result.homeAddress_ = value; - } - result.hasHomeAddress = true; - return this; - } - public Builder clearHomeAddress() { - result.hasHomeAddress = false; - result.homeAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); - return this; - } - - // optional uint64 timeout = 4; - public boolean hasTimeout() { - return result.hasTimeout(); - } - public long getTimeout() { - return result.getTimeout(); - } - public Builder setTimeout(long value) { - result.hasTimeout = true; - result.timeout_ = value; - return this; - } - public Builder clearTimeout() { - result.hasTimeout = false; - result.timeout_ = 0L; - return this; - } - - // @@protoc_insertion_point(builder_scope:RemoteActorRefProtocol) - } - - static { - defaultInstance = new RemoteActorRefProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:RemoteActorRefProtocol) - } - - public static final class SerializedActorRefProtocol extends - com.google.protobuf.GeneratedMessage { - // Use SerializedActorRefProtocol.newBuilder() to construct. - private SerializedActorRefProtocol() { - initFields(); - } - private SerializedActorRefProtocol(boolean noInit) {} - - private static final SerializedActorRefProtocol defaultInstance; - public static SerializedActorRefProtocol getDefaultInstance() { - return defaultInstance; - } - - public SerializedActorRefProtocol getDefaultInstanceForType() { - return defaultInstance; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_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 id = 2; - public static final int ID_FIELD_NUMBER = 2; - private boolean hasId; - private java.lang.String id_ = ""; - public boolean hasId() { return hasId; } - public java.lang.String getId() { return id_; } - - // required string actorClassname = 3; - public static final int ACTORCLASSNAME_FIELD_NUMBER = 3; - private boolean hasActorClassname; - private java.lang.String actorClassname_ = ""; - public boolean hasActorClassname() { return hasActorClassname; } - public java.lang.String getActorClassname() { return actorClassname_; } - - // required .AddressProtocol originalAddress = 4; - public static final int ORIGINALADDRESS_FIELD_NUMBER = 4; - private boolean hasOriginalAddress; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol originalAddress_; - public boolean hasOriginalAddress() { return hasOriginalAddress; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getOriginalAddress() { return originalAddress_; } - - // optional bytes actorInstance = 5; - public static final int ACTORINSTANCE_FIELD_NUMBER = 5; - private boolean hasActorInstance; - private com.google.protobuf.ByteString actorInstance_ = com.google.protobuf.ByteString.EMPTY; - public boolean hasActorInstance() { return hasActorInstance; } - public com.google.protobuf.ByteString getActorInstance() { return actorInstance_; } - - // optional string serializerClassname = 6; - public static final int SERIALIZERCLASSNAME_FIELD_NUMBER = 6; - private boolean hasSerializerClassname; - private java.lang.String serializerClassname_ = ""; - public boolean hasSerializerClassname() { return hasSerializerClassname; } - public java.lang.String getSerializerClassname() { return serializerClassname_; } - - // optional bool isTransactor = 7; - public static final int ISTRANSACTOR_FIELD_NUMBER = 7; - private boolean hasIsTransactor; - private boolean isTransactor_ = false; - public boolean hasIsTransactor() { return hasIsTransactor; } - public boolean getIsTransactor() { return isTransactor_; } - - // optional uint64 timeout = 8; - public static final int TIMEOUT_FIELD_NUMBER = 8; - private boolean hasTimeout; - private long timeout_ = 0L; - public boolean hasTimeout() { return hasTimeout; } - public long getTimeout() { return timeout_; } - - // optional uint64 receiveTimeout = 9; - public static final int RECEIVETIMEOUT_FIELD_NUMBER = 9; - private boolean hasReceiveTimeout; - private long receiveTimeout_ = 0L; - public boolean hasReceiveTimeout() { return hasReceiveTimeout; } - public long getReceiveTimeout() { return receiveTimeout_; } - - // optional .LifeCycleProtocol lifeCycle = 10; - public static final int LIFECYCLE_FIELD_NUMBER = 10; - private boolean hasLifeCycle; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol lifeCycle_; - public boolean hasLifeCycle() { return hasLifeCycle; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getLifeCycle() { return lifeCycle_; } - - // optional .RemoteActorRefProtocol supervisor = 11; - public static final int SUPERVISOR_FIELD_NUMBER = 11; - private boolean hasSupervisor; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol supervisor_; - public boolean hasSupervisor() { return hasSupervisor; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor() { return supervisor_; } - - // optional bytes hotswapStack = 12; - public static final int HOTSWAPSTACK_FIELD_NUMBER = 12; - private boolean hasHotswapStack; - private com.google.protobuf.ByteString hotswapStack_ = com.google.protobuf.ByteString.EMPTY; - public boolean hasHotswapStack() { return hasHotswapStack; } - public com.google.protobuf.ByteString getHotswapStack() { return hotswapStack_; } - - // repeated .RemoteRequestProtocol messages = 13; - public static final int MESSAGES_FIELD_NUMBER = 13; - private java.util.List messages_ = - java.util.Collections.emptyList(); - public java.util.List getMessagesList() { - return messages_; - } - public int getMessagesCount() { return messages_.size(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getMessages(int index) { - return messages_.get(index); - } - - private void initFields() { - 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(); - } - public final boolean isInitialized() { - if (!hasUuid) return false; - if (!hasId) return false; - if (!hasActorClassname) return false; - if (!hasOriginalAddress) return false; - if (!getOriginalAddress().isInitialized()) return false; - if (hasLifeCycle()) { - if (!getLifeCycle().isInitialized()) return false; - } - if (hasSupervisor()) { - if (!getSupervisor().isInitialized()) return false; - } - for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol element : getMessagesList()) { - if (!element.isInitialized()) return false; - } - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (hasUuid()) { - output.writeString(1, getUuid()); - } - if (hasId()) { - output.writeString(2, getId()); - } - if (hasActorClassname()) { - output.writeString(3, getActorClassname()); - } - if (hasOriginalAddress()) { - output.writeMessage(4, getOriginalAddress()); - } - if (hasActorInstance()) { - output.writeBytes(5, getActorInstance()); - } - if (hasSerializerClassname()) { - output.writeString(6, getSerializerClassname()); - } - if (hasIsTransactor()) { - output.writeBool(7, getIsTransactor()); - } - if (hasTimeout()) { - output.writeUInt64(8, getTimeout()); - } - if (hasReceiveTimeout()) { - output.writeUInt64(9, getReceiveTimeout()); - } - if (hasLifeCycle()) { - output.writeMessage(10, getLifeCycle()); - } - if (hasSupervisor()) { - output.writeMessage(11, getSupervisor()); - } - if (hasHotswapStack()) { - output.writeBytes(12, getHotswapStack()); - } - for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol element : getMessagesList()) { - output.writeMessage(13, element); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - if (hasUuid()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(1, getUuid()); - } - if (hasId()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(2, getId()); - } - if (hasActorClassname()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(3, getActorClassname()); - } - if (hasOriginalAddress()) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(4, getOriginalAddress()); - } - if (hasActorInstance()) { - size += com.google.protobuf.CodedOutputStream - .computeBytesSize(5, getActorInstance()); - } - if (hasSerializerClassname()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(6, getSerializerClassname()); - } - if (hasIsTransactor()) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(7, getIsTransactor()); - } - if (hasTimeout()) { - size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(8, getTimeout()); - } - if (hasReceiveTimeout()) { - size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(9, getReceiveTimeout()); - } - if (hasLifeCycle()) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(10, getLifeCycle()); - } - if (hasSupervisor()) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(11, getSupervisor()); - } - if (hasHotswapStack()) { - size += com.google.protobuf.CodedOutputStream - .computeBytesSize(12, getHotswapStack()); - } - for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol element : getMessagesList()) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(13, element); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol result; - - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol(); - return builder; - } - - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol internalGetResult() { - return result; - } - - public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol(); - return this; - } - - public Builder clone() { - return create().mergeFrom(result); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDescriptor(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getDefaultInstanceForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance(); - } - - public boolean isInitialized() { - return result.isInitialized(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol build() { - if (result != null && !isInitialized()) { - throw newUninitializedMessageException(result); - } - return buildPartial(); - } - - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return buildPartial(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); - } - if (result.messages_ != java.util.Collections.EMPTY_LIST) { - result.messages_ = - java.util.Collections.unmodifiableList(result.messages_); - } - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol returnMe = result; - result = null; - return returnMe; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol) { - return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol)other); - } else { - super.mergeFrom(other); - return this; - } - } - - 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()); - } - if (other.hasId()) { - setId(other.getId()); - } - if (other.hasActorClassname()) { - setActorClassname(other.getActorClassname()); - } - if (other.hasOriginalAddress()) { - mergeOriginalAddress(other.getOriginalAddress()); - } - if (other.hasActorInstance()) { - setActorInstance(other.getActorInstance()); - } - if (other.hasSerializerClassname()) { - setSerializerClassname(other.getSerializerClassname()); - } - if (other.hasIsTransactor()) { - setIsTransactor(other.getIsTransactor()); - } - if (other.hasTimeout()) { - setTimeout(other.getTimeout()); - } - if (other.hasReceiveTimeout()) { - setReceiveTimeout(other.getReceiveTimeout()); - } - if (other.hasLifeCycle()) { - mergeLifeCycle(other.getLifeCycle()); - } - if (other.hasSupervisor()) { - mergeSupervisor(other.getSupervisor()); - } - if (other.hasHotswapStack()) { - setHotswapStack(other.getHotswapStack()); - } - if (!other.messages_.isEmpty()) { - if (result.messages_.isEmpty()) { - result.messages_ = new java.util.ArrayList(); - } - result.messages_.addAll(other.messages_); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - return this; - } - break; - } - case 10: { - setUuid(input.readString()); - break; - } - case 18: { - setId(input.readString()); - break; - } - case 26: { - setActorClassname(input.readString()); - break; - } - case 34: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(); - if (hasOriginalAddress()) { - subBuilder.mergeFrom(getOriginalAddress()); - } - input.readMessage(subBuilder, extensionRegistry); - setOriginalAddress(subBuilder.buildPartial()); - break; - } - case 42: { - setActorInstance(input.readBytes()); - break; - } - case 50: { - setSerializerClassname(input.readString()); - break; - } - case 56: { - setIsTransactor(input.readBool()); - break; - } - case 64: { - setTimeout(input.readUInt64()); - break; - } - case 72: { - setReceiveTimeout(input.readUInt64()); - break; - } - case 82: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder(); - if (hasLifeCycle()) { - subBuilder.mergeFrom(getLifeCycle()); - } - input.readMessage(subBuilder, extensionRegistry); - setLifeCycle(subBuilder.buildPartial()); - break; - } - case 90: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(); - if (hasSupervisor()) { - subBuilder.mergeFrom(getSupervisor()); - } - input.readMessage(subBuilder, extensionRegistry); - setSupervisor(subBuilder.buildPartial()); - break; - } - case 98: { - setHotswapStack(input.readBytes()); - break; - } - case 106: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addMessages(subBuilder.buildPartial()); - break; - } - } - } - } - - - // required string uuid = 1; - public boolean hasUuid() { - return result.hasUuid(); - } - public java.lang.String getUuid() { - return result.getUuid(); - } - public Builder setUuid(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasUuid = true; - result.uuid_ = value; - return this; - } - public Builder clearUuid() { - result.hasUuid = false; - result.uuid_ = getDefaultInstance().getUuid(); - return this; - } - - // required string id = 2; - public boolean hasId() { - return result.hasId(); - } - public java.lang.String getId() { - return result.getId(); - } - public Builder setId(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasId = true; - result.id_ = value; - return this; - } - public Builder clearId() { - result.hasId = false; - result.id_ = getDefaultInstance().getId(); - return this; - } - - // required string actorClassname = 3; - public boolean hasActorClassname() { - return result.hasActorClassname(); - } - public java.lang.String getActorClassname() { - return result.getActorClassname(); - } - public Builder setActorClassname(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasActorClassname = true; - result.actorClassname_ = value; - return this; - } - public Builder clearActorClassname() { - result.hasActorClassname = false; - result.actorClassname_ = getDefaultInstance().getActorClassname(); - return this; - } - - // required .AddressProtocol originalAddress = 4; - public boolean hasOriginalAddress() { - return result.hasOriginalAddress(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getOriginalAddress() { - return result.getOriginalAddress(); - } - public Builder setOriginalAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasOriginalAddress = true; - result.originalAddress_ = value; - return this; - } - public Builder setOriginalAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder builderForValue) { - result.hasOriginalAddress = true; - result.originalAddress_ = builderForValue.build(); - return this; - } - public Builder mergeOriginalAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol value) { - if (result.hasOriginalAddress() && - result.originalAddress_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) { - result.originalAddress_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(result.originalAddress_).mergeFrom(value).buildPartial(); - } else { - result.originalAddress_ = value; - } - result.hasOriginalAddress = true; - return this; - } - public Builder clearOriginalAddress() { - result.hasOriginalAddress = false; - result.originalAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); - return this; - } - - // optional bytes actorInstance = 5; - public boolean hasActorInstance() { - return result.hasActorInstance(); - } - public com.google.protobuf.ByteString getActorInstance() { - return result.getActorInstance(); - } - public Builder setActorInstance(com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasActorInstance = true; - result.actorInstance_ = value; - return this; - } - public Builder clearActorInstance() { - result.hasActorInstance = false; - result.actorInstance_ = getDefaultInstance().getActorInstance(); - return this; - } - - // optional string serializerClassname = 6; - public boolean hasSerializerClassname() { - return result.hasSerializerClassname(); - } - public java.lang.String getSerializerClassname() { - return result.getSerializerClassname(); - } - public Builder setSerializerClassname(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasSerializerClassname = true; - result.serializerClassname_ = value; - return this; - } - public Builder clearSerializerClassname() { - result.hasSerializerClassname = false; - result.serializerClassname_ = getDefaultInstance().getSerializerClassname(); - return this; - } - - // optional bool isTransactor = 7; - public boolean hasIsTransactor() { - return result.hasIsTransactor(); - } - public boolean getIsTransactor() { - return result.getIsTransactor(); - } - public Builder setIsTransactor(boolean value) { - result.hasIsTransactor = true; - result.isTransactor_ = value; - return this; - } - public Builder clearIsTransactor() { - result.hasIsTransactor = false; - result.isTransactor_ = false; - return this; - } - - // optional uint64 timeout = 8; - public boolean hasTimeout() { - return result.hasTimeout(); - } - public long getTimeout() { - return result.getTimeout(); - } - public Builder setTimeout(long value) { - result.hasTimeout = true; - result.timeout_ = value; - return this; - } - public Builder clearTimeout() { - result.hasTimeout = false; - result.timeout_ = 0L; - return this; - } - - // optional uint64 receiveTimeout = 9; - public boolean hasReceiveTimeout() { - return result.hasReceiveTimeout(); - } - public long getReceiveTimeout() { - return result.getReceiveTimeout(); - } - public Builder setReceiveTimeout(long value) { - result.hasReceiveTimeout = true; - result.receiveTimeout_ = value; - return this; - } - public Builder clearReceiveTimeout() { - result.hasReceiveTimeout = false; - result.receiveTimeout_ = 0L; - return this; - } - - // optional .LifeCycleProtocol lifeCycle = 10; - public boolean hasLifeCycle() { - return result.hasLifeCycle(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getLifeCycle() { - return result.getLifeCycle(); - } - public Builder setLifeCycle(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasLifeCycle = true; - result.lifeCycle_ = value; - return this; - } - public Builder setLifeCycle(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder builderForValue) { - result.hasLifeCycle = true; - result.lifeCycle_ = builderForValue.build(); - return this; - } - public Builder mergeLifeCycle(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol value) { - if (result.hasLifeCycle() && - result.lifeCycle_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance()) { - result.lifeCycle_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder(result.lifeCycle_).mergeFrom(value).buildPartial(); - } else { - result.lifeCycle_ = value; - } - result.hasLifeCycle = true; - return this; - } - public Builder clearLifeCycle() { - result.hasLifeCycle = false; - result.lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); - return this; - } - - // optional .RemoteActorRefProtocol supervisor = 11; - public boolean hasSupervisor() { - return result.hasSupervisor(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor() { - return result.getSupervisor(); - } - public Builder setSupervisor(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasSupervisor = true; - result.supervisor_ = value; - return this; - } - public Builder setSupervisor(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder builderForValue) { - result.hasSupervisor = true; - result.supervisor_ = builderForValue.build(); - return this; - } - public Builder mergeSupervisor(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { - if (result.hasSupervisor() && - result.supervisor_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) { - result.supervisor_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(result.supervisor_).mergeFrom(value).buildPartial(); - } else { - result.supervisor_ = value; - } - result.hasSupervisor = true; - return this; - } - public Builder clearSupervisor() { - result.hasSupervisor = false; - result.supervisor_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); - return this; - } - - // optional bytes hotswapStack = 12; - public boolean hasHotswapStack() { - return result.hasHotswapStack(); - } - public com.google.protobuf.ByteString getHotswapStack() { - return result.getHotswapStack(); - } - public Builder setHotswapStack(com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasHotswapStack = true; - result.hotswapStack_ = value; - return this; - } - public Builder clearHotswapStack() { - result.hasHotswapStack = false; - result.hotswapStack_ = getDefaultInstance().getHotswapStack(); - return this; - } - - // repeated .RemoteRequestProtocol messages = 13; - public java.util.List getMessagesList() { - return java.util.Collections.unmodifiableList(result.messages_); - } - public int getMessagesCount() { - return result.getMessagesCount(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getMessages(int index) { - return result.getMessages(index); - } - public Builder setMessages(int index, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol value) { - if (value == null) { - throw new NullPointerException(); - } - result.messages_.set(index, value); - return this; - } - public Builder setMessages(int index, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder builderForValue) { - result.messages_.set(index, builderForValue.build()); - return this; - } - public Builder addMessages(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol value) { - if (value == null) { - throw new NullPointerException(); - } - if (result.messages_.isEmpty()) { - result.messages_ = new java.util.ArrayList(); - } - result.messages_.add(value); - return this; - } - public Builder addMessages(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder builderForValue) { - if (result.messages_.isEmpty()) { - result.messages_ = new java.util.ArrayList(); - } - result.messages_.add(builderForValue.build()); - return this; - } - public Builder addAllMessages( - java.lang.Iterable values) { - if (result.messages_.isEmpty()) { - result.messages_ = new java.util.ArrayList(); - } - super.addAll(values, result.messages_); - return this; - } - public Builder clearMessages() { - result.messages_ = java.util.Collections.emptyList(); - return this; - } - - // @@protoc_insertion_point(builder_scope:SerializedActorRefProtocol) - } - - static { - defaultInstance = new SerializedActorRefProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:SerializedActorRefProtocol) - } - - public static final class MessageProtocol extends - com.google.protobuf.GeneratedMessage { - // Use MessageProtocol.newBuilder() to construct. - private MessageProtocol() { - initFields(); - } - private MessageProtocol(boolean noInit) {} - - private static final MessageProtocol defaultInstance; - public static MessageProtocol getDefaultInstance() { - return defaultInstance; - } - - public MessageProtocol getDefaultInstanceForType() { - return defaultInstance; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MessageProtocol_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MessageProtocol_fieldAccessorTable; - } - - // required .SerializationSchemeType serializationScheme = 1; - public static final int SERIALIZATIONSCHEME_FIELD_NUMBER = 1; - private boolean hasSerializationScheme; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType serializationScheme_; - public boolean hasSerializationScheme() { return hasSerializationScheme; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType getSerializationScheme() { return serializationScheme_; } - - // required bytes message = 2; - public static final int MESSAGE_FIELD_NUMBER = 2; - private boolean hasMessage; - private com.google.protobuf.ByteString message_ = com.google.protobuf.ByteString.EMPTY; - public boolean hasMessage() { return hasMessage; } - public com.google.protobuf.ByteString getMessage() { return message_; } - - // optional bytes messageManifest = 3; - public static final int MESSAGEMANIFEST_FIELD_NUMBER = 3; - private boolean hasMessageManifest; - private com.google.protobuf.ByteString messageManifest_ = com.google.protobuf.ByteString.EMPTY; - public boolean hasMessageManifest() { return hasMessageManifest; } - public com.google.protobuf.ByteString getMessageManifest() { return messageManifest_; } - - private void initFields() { - serializationScheme_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA; - } - public final boolean isInitialized() { - if (!hasSerializationScheme) return false; - if (!hasMessage) return false; - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (hasSerializationScheme()) { - output.writeEnum(1, getSerializationScheme().getNumber()); - } - if (hasMessage()) { - output.writeBytes(2, getMessage()); - } - if (hasMessageManifest()) { - output.writeBytes(3, getMessageManifest()); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - if (hasSerializationScheme()) { - size += com.google.protobuf.CodedOutputStream - .computeEnumSize(1, getSerializationScheme().getNumber()); - } - if (hasMessage()) { - size += com.google.protobuf.CodedOutputStream - .computeBytesSize(2, getMessage()); - } - if (hasMessageManifest()) { - size += com.google.protobuf.CodedOutputStream - .computeBytesSize(3, getMessageManifest()); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol result; - - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol(); - return builder; - } - - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol internalGetResult() { - return result; - } - - public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol(); - return this; - } - - public Builder clone() { - return create().mergeFrom(result); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDescriptor(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getDefaultInstanceForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); - } - - public boolean isInitialized() { - return result.isInitialized(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol build() { - if (result != null && !isInitialized()) { - throw newUninitializedMessageException(result); - } - return buildPartial(); - } - - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return buildPartial(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); - } - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol returnMe = result; - result = null; - return returnMe; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol) { - return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol other) { - if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) return this; - if (other.hasSerializationScheme()) { - setSerializationScheme(other.getSerializationScheme()); - } - if (other.hasMessage()) { - setMessage(other.getMessage()); - } - if (other.hasMessageManifest()) { - setMessageManifest(other.getMessageManifest()); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - return this; - } - break; - } - case 8: { - int rawValue = input.readEnum(); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType value = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.valueOf(rawValue); - if (value == null) { - unknownFields.mergeVarintField(1, rawValue); - } else { - setSerializationScheme(value); - } - break; - } - case 18: { - setMessage(input.readBytes()); - break; - } - case 26: { - setMessageManifest(input.readBytes()); - break; - } - } - } - } - - - // required .SerializationSchemeType serializationScheme = 1; - public boolean hasSerializationScheme() { - return result.hasSerializationScheme(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType getSerializationScheme() { - return result.getSerializationScheme(); - } - public Builder setSerializationScheme(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasSerializationScheme = true; - result.serializationScheme_ = value; - return this; - } - public Builder clearSerializationScheme() { - result.hasSerializationScheme = false; - result.serializationScheme_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA; - return this; - } - - // required bytes message = 2; - public boolean hasMessage() { - return result.hasMessage(); - } - public com.google.protobuf.ByteString getMessage() { - return result.getMessage(); - } - public Builder setMessage(com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasMessage = true; - result.message_ = value; - return this; - } - public Builder clearMessage() { - result.hasMessage = false; - result.message_ = getDefaultInstance().getMessage(); - return this; - } - - // optional bytes messageManifest = 3; - public boolean hasMessageManifest() { - return result.hasMessageManifest(); - } - public com.google.protobuf.ByteString getMessageManifest() { - return result.getMessageManifest(); - } - public Builder setMessageManifest(com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasMessageManifest = true; - result.messageManifest_ = value; - return this; - } - public Builder clearMessageManifest() { - result.hasMessageManifest = false; - result.messageManifest_ = getDefaultInstance().getMessageManifest(); - return this; - } - - // @@protoc_insertion_point(builder_scope:MessageProtocol) - } - - static { - defaultInstance = new MessageProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:MessageProtocol) - } - - public static final class ActorInfoProtocol extends - com.google.protobuf.GeneratedMessage { - // Use ActorInfoProtocol.newBuilder() to construct. - private ActorInfoProtocol() { - initFields(); - } - private ActorInfoProtocol(boolean noInit) {} - - private static final ActorInfoProtocol defaultInstance; - public static ActorInfoProtocol getDefaultInstance() { - return defaultInstance; - } - - public ActorInfoProtocol getDefaultInstanceForType() { - return defaultInstance; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_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 target = 2; - public static final int TARGET_FIELD_NUMBER = 2; - private boolean hasTarget; - private java.lang.String target_ = ""; - public boolean hasTarget() { return hasTarget; } - public java.lang.String getTarget() { return target_; } - - // required uint64 timeout = 3; - public static final int TIMEOUT_FIELD_NUMBER = 3; - private boolean hasTimeout; - private long timeout_ = 0L; - public boolean hasTimeout() { return hasTimeout; } - public long getTimeout() { return timeout_; } - - // required .ActorType actorType = 4; - public static final int ACTORTYPE_FIELD_NUMBER = 4; - private boolean hasActorType; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType actorType_; - public boolean hasActorType() { return hasActorType; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType getActorType() { return actorType_; } - - // optional .TypedActorInfoProtocol typedActorInfo = 5; - public static final int TYPEDACTORINFO_FIELD_NUMBER = 5; - private boolean hasTypedActorInfo; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol typedActorInfo_; - public boolean hasTypedActorInfo() { return hasTypedActorInfo; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getTypedActorInfo() { return typedActorInfo_; } - - private void initFields() { - actorType_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.SCALA_ACTOR; - typedActorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance(); - } - public final boolean isInitialized() { - if (!hasUuid) return false; - if (!hasTarget) return false; - if (!hasTimeout) return false; - if (!hasActorType) return false; - if (hasTypedActorInfo()) { - if (!getTypedActorInfo().isInitialized()) return false; - } - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (hasUuid()) { - output.writeString(1, getUuid()); - } - if (hasTarget()) { - output.writeString(2, getTarget()); - } - if (hasTimeout()) { - output.writeUInt64(3, getTimeout()); - } - if (hasActorType()) { - output.writeEnum(4, getActorType().getNumber()); - } - if (hasTypedActorInfo()) { - output.writeMessage(5, getTypedActorInfo()); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - if (hasUuid()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(1, getUuid()); - } - if (hasTarget()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(2, getTarget()); - } - if (hasTimeout()) { - size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(3, getTimeout()); - } - if (hasActorType()) { - size += com.google.protobuf.CodedOutputStream - .computeEnumSize(4, getActorType().getNumber()); - } - if (hasTypedActorInfo()) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(5, getTypedActorInfo()); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol result; - - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol(); - return builder; - } - - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol internalGetResult() { - return result; - } - - public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol(); - return this; - } - - public Builder clone() { - return create().mergeFrom(result); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDescriptor(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getDefaultInstanceForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); - } - - public boolean isInitialized() { - return result.isInitialized(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol build() { - if (result != null && !isInitialized()) { - throw newUninitializedMessageException(result); - } - return buildPartial(); - } - - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return buildPartial(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); - } - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol returnMe = result; - result = null; - return returnMe; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol) { - return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol)other); - } else { - super.mergeFrom(other); - return this; - } - } - - 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()); - } - if (other.hasTarget()) { - setTarget(other.getTarget()); - } - if (other.hasTimeout()) { - setTimeout(other.getTimeout()); - } - if (other.hasActorType()) { - setActorType(other.getActorType()); - } - if (other.hasTypedActorInfo()) { - mergeTypedActorInfo(other.getTypedActorInfo()); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - return this; - } - break; - } - case 10: { - setUuid(input.readString()); - break; - } - case 18: { - setTarget(input.readString()); - break; - } - case 24: { - setTimeout(input.readUInt64()); - break; - } - case 32: { - int rawValue = input.readEnum(); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType value = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.valueOf(rawValue); - if (value == null) { - unknownFields.mergeVarintField(4, rawValue); - } else { - setActorType(value); - } - break; - } - case 42: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder(); - if (hasTypedActorInfo()) { - subBuilder.mergeFrom(getTypedActorInfo()); - } - input.readMessage(subBuilder, extensionRegistry); - setTypedActorInfo(subBuilder.buildPartial()); - break; - } - } - } - } - - - // required string uuid = 1; - public boolean hasUuid() { - return result.hasUuid(); - } - public java.lang.String getUuid() { - return result.getUuid(); - } - public Builder setUuid(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasUuid = true; - result.uuid_ = value; - return this; - } - public Builder clearUuid() { - result.hasUuid = false; - result.uuid_ = getDefaultInstance().getUuid(); - return this; - } - - // required string target = 2; - public boolean hasTarget() { - return result.hasTarget(); - } - public java.lang.String getTarget() { - return result.getTarget(); - } - public Builder setTarget(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasTarget = true; - result.target_ = value; - return this; - } - public Builder clearTarget() { - result.hasTarget = false; - result.target_ = getDefaultInstance().getTarget(); - return this; - } - - // required uint64 timeout = 3; - public boolean hasTimeout() { - return result.hasTimeout(); - } - public long getTimeout() { - return result.getTimeout(); - } - public Builder setTimeout(long value) { - result.hasTimeout = true; - result.timeout_ = value; - return this; - } - public Builder clearTimeout() { - result.hasTimeout = false; - result.timeout_ = 0L; - return this; - } - - // required .ActorType actorType = 4; - public boolean hasActorType() { - return result.hasActorType(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType getActorType() { - return result.getActorType(); - } - public Builder setActorType(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasActorType = true; - result.actorType_ = value; - return this; - } - public Builder clearActorType() { - result.hasActorType = false; - result.actorType_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.SCALA_ACTOR; - return this; - } - - // optional .TypedActorInfoProtocol typedActorInfo = 5; - public boolean hasTypedActorInfo() { - return result.hasTypedActorInfo(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getTypedActorInfo() { - return result.getTypedActorInfo(); - } - public Builder setTypedActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasTypedActorInfo = true; - result.typedActorInfo_ = value; - return this; - } - public Builder setTypedActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.Builder builderForValue) { - result.hasTypedActorInfo = true; - result.typedActorInfo_ = builderForValue.build(); - return this; - } - public Builder mergeTypedActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol value) { - if (result.hasTypedActorInfo() && - result.typedActorInfo_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance()) { - result.typedActorInfo_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder(result.typedActorInfo_).mergeFrom(value).buildPartial(); - } else { - result.typedActorInfo_ = value; - } - result.hasTypedActorInfo = true; - return this; - } - public Builder clearTypedActorInfo() { - result.hasTypedActorInfo = false; - result.typedActorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance(); - return this; - } - - // @@protoc_insertion_point(builder_scope:ActorInfoProtocol) - } - - static { - defaultInstance = new ActorInfoProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:ActorInfoProtocol) - } - - public static final class TypedActorInfoProtocol extends - com.google.protobuf.GeneratedMessage { - // Use TypedActorInfoProtocol.newBuilder() to construct. - private TypedActorInfoProtocol() { - initFields(); - } - private TypedActorInfoProtocol(boolean noInit) {} - - private static final TypedActorInfoProtocol defaultInstance; - public static TypedActorInfoProtocol getDefaultInstance() { - return defaultInstance; - } - - public TypedActorInfoProtocol getDefaultInstanceForType() { - return defaultInstance; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_TypedActorInfoProtocol_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_TypedActorInfoProtocol_fieldAccessorTable; - } - - // required string interface = 1; - public static final int INTERFACE_FIELD_NUMBER = 1; - private boolean hasInterface; - private java.lang.String interface_ = ""; - public boolean hasInterface() { return hasInterface; } - public java.lang.String getInterface() { return interface_; } - - // required string method = 2; - public static final int METHOD_FIELD_NUMBER = 2; - private boolean hasMethod; - private java.lang.String method_ = ""; - public boolean hasMethod() { return hasMethod; } - public java.lang.String getMethod() { return method_; } - - private void initFields() { - } - public final boolean isInitialized() { - if (!hasInterface) return false; - if (!hasMethod) return false; - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (hasInterface()) { - output.writeString(1, getInterface()); - } - if (hasMethod()) { - output.writeString(2, getMethod()); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - if (hasInterface()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(1, getInterface()); - } - if (hasMethod()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(2, getMethod()); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol result; - - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol(); - return builder; - } - - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol internalGetResult() { - return result; - } - - public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol(); - return this; - } - - public Builder clone() { - return create().mergeFrom(result); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDescriptor(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getDefaultInstanceForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance(); - } - - public boolean isInitialized() { - return result.isInitialized(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol build() { - if (result != null && !isInitialized()) { - throw newUninitializedMessageException(result); - } - return buildPartial(); - } - - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return buildPartial(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); - } - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol returnMe = result; - result = null; - return returnMe; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol) { - return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol other) { - if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance()) return this; - if (other.hasInterface()) { - setInterface(other.getInterface()); - } - if (other.hasMethod()) { - setMethod(other.getMethod()); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - return this; - } - break; - } - case 10: { - setInterface(input.readString()); - break; - } - case 18: { - setMethod(input.readString()); - break; - } - } - } - } - - - // required string interface = 1; - public boolean hasInterface() { - return result.hasInterface(); - } - public java.lang.String getInterface() { - return result.getInterface(); - } - public Builder setInterface(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasInterface = true; - result.interface_ = value; - return this; - } - public Builder clearInterface() { - result.hasInterface = false; - result.interface_ = getDefaultInstance().getInterface(); - return this; - } - - // required string method = 2; - public boolean hasMethod() { - return result.hasMethod(); - } - public java.lang.String getMethod() { - return result.getMethod(); - } - public Builder setMethod(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasMethod = true; - result.method_ = value; - return this; - } - public Builder clearMethod() { - result.hasMethod = false; - result.method_ = getDefaultInstance().getMethod(); - return this; - } - - // @@protoc_insertion_point(builder_scope:TypedActorInfoProtocol) - } - - static { - defaultInstance = new TypedActorInfoProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:TypedActorInfoProtocol) - } - - public static final class RemoteRequestProtocol extends - com.google.protobuf.GeneratedMessage { - // Use RemoteRequestProtocol.newBuilder() to construct. - private RemoteRequestProtocol() { - initFields(); - } - private RemoteRequestProtocol(boolean noInit) {} - - private static final RemoteRequestProtocol defaultInstance; - public static RemoteRequestProtocol getDefaultInstance() { - return defaultInstance; - } - - public RemoteRequestProtocol getDefaultInstanceForType() { - return defaultInstance; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteRequestProtocol_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - 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 .MessageProtocol message = 2; - public static final int MESSAGE_FIELD_NUMBER = 2; - private boolean hasMessage; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol message_; - public boolean hasMessage() { return hasMessage; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { return message_; } - - // required .ActorInfoProtocol actorInfo = 3; - public static final int ACTORINFO_FIELD_NUMBER = 3; - private boolean hasActorInfo; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol actorInfo_; - public boolean hasActorInfo() { return hasActorInfo; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo() { return actorInfo_; } - - // required bool isOneWay = 4; - public static final int ISONEWAY_FIELD_NUMBER = 4; - private boolean hasIsOneWay; - private boolean isOneWay_ = false; - public boolean hasIsOneWay() { return hasIsOneWay; } - public boolean getIsOneWay() { return isOneWay_; } - - // optional string supervisorUuid = 5; - public static final int SUPERVISORUUID_FIELD_NUMBER = 5; - private boolean hasSupervisorUuid; - private java.lang.String supervisorUuid_ = ""; - public boolean hasSupervisorUuid() { return hasSupervisorUuid; } - public java.lang.String getSupervisorUuid() { return supervisorUuid_; } - - // optional .RemoteActorRefProtocol sender = 6; - public static final int SENDER_FIELD_NUMBER = 6; - private boolean hasSender; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol sender_; - public boolean hasSender() { return hasSender; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSender() { return sender_; } - - private void initFields() { - message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); - actorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); - sender_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); - } - public final boolean isInitialized() { - if (!hasId) return false; - if (!hasMessage) return false; - if (!hasActorInfo) return false; - if (!hasIsOneWay) return false; - if (!getMessage().isInitialized()) return false; - if (!getActorInfo().isInitialized()) return false; - if (hasSender()) { - if (!getSender().isInitialized()) return false; - } - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (hasId()) { - output.writeUInt64(1, getId()); - } - if (hasMessage()) { - output.writeMessage(2, getMessage()); - } - if (hasActorInfo()) { - output.writeMessage(3, getActorInfo()); - } - if (hasIsOneWay()) { - output.writeBool(4, getIsOneWay()); - } - if (hasSupervisorUuid()) { - output.writeString(5, getSupervisorUuid()); - } - if (hasSender()) { - output.writeMessage(6, getSender()); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - if (hasId()) { - size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(1, getId()); - } - if (hasMessage()) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, getMessage()); - } - if (hasActorInfo()) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(3, getActorInfo()); - } - if (hasIsOneWay()) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(4, getIsOneWay()); - } - if (hasSupervisorUuid()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(5, getSupervisorUuid()); - } - if (hasSender()) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(6, getSender()); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol result; - - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol(); - return builder; - } - - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol internalGetResult() { - return result; - } - - public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol(); - return this; - } - - public Builder clone() { - return create().mergeFrom(result); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDescriptor(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getDefaultInstanceForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDefaultInstance(); - } - - public boolean isInitialized() { - return result.isInitialized(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol build() { - if (result != null && !isInitialized()) { - throw newUninitializedMessageException(result); - } - return buildPartial(); - } - - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return buildPartial(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); - } - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol returnMe = result; - result = null; - return returnMe; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol) { - return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol)other); - } else { - super.mergeFrom(other); - return this; - } - } - - 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.hasMessage()) { - mergeMessage(other.getMessage()); - } - if (other.hasActorInfo()) { - mergeActorInfo(other.getActorInfo()); - } - if (other.hasIsOneWay()) { - setIsOneWay(other.getIsOneWay()); - } - if (other.hasSupervisorUuid()) { - setSupervisorUuid(other.getSupervisorUuid()); - } - if (other.hasSender()) { - mergeSender(other.getSender()); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - return this; - } - break; - } - case 8: { - setId(input.readUInt64()); - break; - } - case 18: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(); - if (hasMessage()) { - subBuilder.mergeFrom(getMessage()); - } - input.readMessage(subBuilder, extensionRegistry); - setMessage(subBuilder.buildPartial()); - break; - } - case 26: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder(); - if (hasActorInfo()) { - subBuilder.mergeFrom(getActorInfo()); - } - input.readMessage(subBuilder, extensionRegistry); - setActorInfo(subBuilder.buildPartial()); - break; - } - case 32: { - setIsOneWay(input.readBool()); - break; - } - case 42: { - setSupervisorUuid(input.readString()); - break; - } - case 50: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(); - if (hasSender()) { - subBuilder.mergeFrom(getSender()); - } - input.readMessage(subBuilder, extensionRegistry); - setSender(subBuilder.buildPartial()); - break; - } - } - } - } - - - // required uint64 id = 1; - public boolean hasId() { - return result.hasId(); - } - public long getId() { - return result.getId(); - } - public Builder setId(long value) { - result.hasId = true; - result.id_ = value; - return this; - } - public Builder clearId() { - result.hasId = false; - result.id_ = 0L; - return this; - } - - // required .MessageProtocol message = 2; - public boolean hasMessage() { - return result.hasMessage(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { - return result.getMessage(); - } - public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasMessage = true; - result.message_ = value; - return this; - } - public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder builderForValue) { - result.hasMessage = true; - result.message_ = builderForValue.build(); - return this; - } - public Builder mergeMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol value) { - if (result.hasMessage() && - result.message_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) { - result.message_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(result.message_).mergeFrom(value).buildPartial(); - } else { - result.message_ = value; - } - result.hasMessage = true; - return this; - } - public Builder clearMessage() { - result.hasMessage = false; - result.message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); - return this; - } - - // required .ActorInfoProtocol actorInfo = 3; - public boolean hasActorInfo() { - return result.hasActorInfo(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo() { - return result.getActorInfo(); - } - public Builder setActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasActorInfo = true; - result.actorInfo_ = value; - return this; - } - public Builder setActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder builderForValue) { - result.hasActorInfo = true; - result.actorInfo_ = builderForValue.build(); - return this; - } - public Builder mergeActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol value) { - if (result.hasActorInfo() && - result.actorInfo_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance()) { - result.actorInfo_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder(result.actorInfo_).mergeFrom(value).buildPartial(); - } else { - result.actorInfo_ = value; - } - result.hasActorInfo = true; - return this; - } - public Builder clearActorInfo() { - result.hasActorInfo = false; - result.actorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); - return this; - } - - // required bool isOneWay = 4; - public boolean hasIsOneWay() { - return result.hasIsOneWay(); - } - public boolean getIsOneWay() { - return result.getIsOneWay(); - } - public Builder setIsOneWay(boolean value) { - result.hasIsOneWay = true; - result.isOneWay_ = value; - return this; - } - public Builder clearIsOneWay() { - result.hasIsOneWay = false; - result.isOneWay_ = false; - return this; - } - - // optional string supervisorUuid = 5; - public boolean hasSupervisorUuid() { - return result.hasSupervisorUuid(); - } - public java.lang.String getSupervisorUuid() { - return result.getSupervisorUuid(); - } - public Builder setSupervisorUuid(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasSupervisorUuid = true; - result.supervisorUuid_ = value; - return this; - } - public Builder clearSupervisorUuid() { - result.hasSupervisorUuid = false; - result.supervisorUuid_ = getDefaultInstance().getSupervisorUuid(); - return this; - } - - // optional .RemoteActorRefProtocol sender = 6; - public boolean hasSender() { - return result.hasSender(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSender() { - return result.getSender(); - } - public Builder setSender(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasSender = true; - result.sender_ = value; - return this; - } - public Builder setSender(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder builderForValue) { - result.hasSender = true; - result.sender_ = builderForValue.build(); - return this; - } - public Builder mergeSender(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { - if (result.hasSender() && - result.sender_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) { - result.sender_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(result.sender_).mergeFrom(value).buildPartial(); - } else { - result.sender_ = value; - } - result.hasSender = true; - return this; - } - public Builder clearSender() { - result.hasSender = false; - result.sender_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); - return this; - } - - // @@protoc_insertion_point(builder_scope:RemoteRequestProtocol) - } - - static { - defaultInstance = new RemoteRequestProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:RemoteRequestProtocol) - } - - public static final class RemoteReplyProtocol extends - com.google.protobuf.GeneratedMessage { - // Use RemoteReplyProtocol.newBuilder() to construct. - private RemoteReplyProtocol() { - initFields(); - } - private RemoteReplyProtocol(boolean noInit) {} - - private static final RemoteReplyProtocol defaultInstance; - public static RemoteReplyProtocol getDefaultInstance() { - return defaultInstance; - } - - public RemoteReplyProtocol getDefaultInstanceForType() { - return defaultInstance; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteReplyProtocol_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - 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_; } - - // optional .MessageProtocol message = 2; - public static final int MESSAGE_FIELD_NUMBER = 2; - private boolean hasMessage; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol message_; - public boolean hasMessage() { return hasMessage; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { return message_; } - - // optional .ExceptionProtocol exception = 3; - public static final int EXCEPTION_FIELD_NUMBER = 3; - private boolean hasException; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol exception_; - public boolean hasException() { return hasException; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() { return exception_; } - - // optional string supervisorUuid = 4; - public static final int SUPERVISORUUID_FIELD_NUMBER = 4; - private boolean hasSupervisorUuid; - private java.lang.String supervisorUuid_ = ""; - public boolean hasSupervisorUuid() { return hasSupervisorUuid; } - public java.lang.String getSupervisorUuid() { return supervisorUuid_; } - - // required bool isActor = 5; - public static final int ISACTOR_FIELD_NUMBER = 5; - private boolean hasIsActor; - private boolean isActor_ = false; - public boolean hasIsActor() { return hasIsActor; } - public boolean getIsActor() { return isActor_; } - - // required bool isSuccessful = 6; - public static final int ISSUCCESSFUL_FIELD_NUMBER = 6; - private boolean hasIsSuccessful; - private boolean isSuccessful_ = false; - public boolean hasIsSuccessful() { return hasIsSuccessful; } - public boolean getIsSuccessful() { return isSuccessful_; } - - private void initFields() { - message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); - exception_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); - } - public final boolean isInitialized() { - if (!hasId) return false; - if (!hasIsActor) return false; - if (!hasIsSuccessful) return false; - if (hasMessage()) { - if (!getMessage().isInitialized()) return false; - } - if (hasException()) { - if (!getException().isInitialized()) return false; - } - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (hasId()) { - output.writeUInt64(1, getId()); - } - if (hasMessage()) { - output.writeMessage(2, getMessage()); - } - if (hasException()) { - output.writeMessage(3, getException()); - } - if (hasSupervisorUuid()) { - output.writeString(4, getSupervisorUuid()); - } - if (hasIsActor()) { - output.writeBool(5, getIsActor()); - } - if (hasIsSuccessful()) { - output.writeBool(6, getIsSuccessful()); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - if (hasId()) { - size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(1, getId()); - } - if (hasMessage()) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, getMessage()); - } - if (hasException()) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(3, getException()); - } - if (hasSupervisorUuid()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(4, getSupervisorUuid()); - } - if (hasIsActor()) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(5, getIsActor()); - } - if (hasIsSuccessful()) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(6, getIsSuccessful()); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol result; - - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol(); - return builder; - } - - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol internalGetResult() { - return result; - } - - public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol(); - return this; - } - - public Builder clone() { - return create().mergeFrom(result); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDescriptor(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol getDefaultInstanceForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDefaultInstance(); - } - - public boolean isInitialized() { - return result.isInitialized(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol build() { - if (result != null && !isInitialized()) { - throw newUninitializedMessageException(result); - } - return buildPartial(); - } - - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return buildPartial(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); - } - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol returnMe = result; - result = null; - return returnMe; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol) { - return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol)other); - } else { - super.mergeFrom(other); - return this; - } - } - - 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.hasMessage()) { - mergeMessage(other.getMessage()); - } - if (other.hasException()) { - mergeException(other.getException()); - } - if (other.hasSupervisorUuid()) { - setSupervisorUuid(other.getSupervisorUuid()); - } - if (other.hasIsActor()) { - setIsActor(other.getIsActor()); - } - if (other.hasIsSuccessful()) { - setIsSuccessful(other.getIsSuccessful()); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - return this; - } - break; - } - case 8: { - setId(input.readUInt64()); - break; - } - case 18: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(); - if (hasMessage()) { - subBuilder.mergeFrom(getMessage()); - } - input.readMessage(subBuilder, extensionRegistry); - setMessage(subBuilder.buildPartial()); - break; - } - case 26: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder(); - if (hasException()) { - subBuilder.mergeFrom(getException()); - } - input.readMessage(subBuilder, extensionRegistry); - setException(subBuilder.buildPartial()); - break; - } - case 34: { - setSupervisorUuid(input.readString()); - break; - } - case 40: { - setIsActor(input.readBool()); - break; - } - case 48: { - setIsSuccessful(input.readBool()); - break; - } - } - } - } - - - // required uint64 id = 1; - public boolean hasId() { - return result.hasId(); - } - public long getId() { - return result.getId(); - } - public Builder setId(long value) { - result.hasId = true; - result.id_ = value; - return this; - } - public Builder clearId() { - result.hasId = false; - result.id_ = 0L; - return this; - } - - // optional .MessageProtocol message = 2; - public boolean hasMessage() { - return result.hasMessage(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { - return result.getMessage(); - } - public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasMessage = true; - result.message_ = value; - return this; - } - public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder builderForValue) { - result.hasMessage = true; - result.message_ = builderForValue.build(); - return this; - } - public Builder mergeMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol value) { - if (result.hasMessage() && - result.message_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) { - result.message_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(result.message_).mergeFrom(value).buildPartial(); - } else { - result.message_ = value; - } - result.hasMessage = true; - return this; - } - public Builder clearMessage() { - result.hasMessage = false; - result.message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); - return this; - } - - // optional .ExceptionProtocol exception = 3; - public boolean hasException() { - return result.hasException(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() { - return result.getException(); - } - public Builder setException(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasException = true; - result.exception_ = value; - return this; - } - public Builder setException(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder builderForValue) { - result.hasException = true; - result.exception_ = builderForValue.build(); - return this; - } - public Builder mergeException(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol value) { - if (result.hasException() && - result.exception_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance()) { - result.exception_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder(result.exception_).mergeFrom(value).buildPartial(); - } else { - result.exception_ = value; - } - result.hasException = true; - return this; - } - public Builder clearException() { - result.hasException = false; - result.exception_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); - return this; - } - - // optional string supervisorUuid = 4; - public boolean hasSupervisorUuid() { - return result.hasSupervisorUuid(); - } - public java.lang.String getSupervisorUuid() { - return result.getSupervisorUuid(); - } - public Builder setSupervisorUuid(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasSupervisorUuid = true; - result.supervisorUuid_ = value; - return this; - } - public Builder clearSupervisorUuid() { - result.hasSupervisorUuid = false; - result.supervisorUuid_ = getDefaultInstance().getSupervisorUuid(); - return this; - } - - // required bool isActor = 5; - public boolean hasIsActor() { - return result.hasIsActor(); - } - public boolean getIsActor() { - return result.getIsActor(); - } - public Builder setIsActor(boolean value) { - result.hasIsActor = true; - result.isActor_ = value; - return this; - } - public Builder clearIsActor() { - result.hasIsActor = false; - result.isActor_ = false; - return this; - } - - // required bool isSuccessful = 6; - public boolean hasIsSuccessful() { - return result.hasIsSuccessful(); - } - public boolean getIsSuccessful() { - return result.getIsSuccessful(); - } - public Builder setIsSuccessful(boolean value) { - result.hasIsSuccessful = true; - result.isSuccessful_ = value; - return this; - } - public Builder clearIsSuccessful() { - result.hasIsSuccessful = false; - result.isSuccessful_ = false; - return this; - } - - // @@protoc_insertion_point(builder_scope:RemoteReplyProtocol) - } - - static { - defaultInstance = new RemoteReplyProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:RemoteReplyProtocol) - } - - public static final class LifeCycleProtocol extends - com.google.protobuf.GeneratedMessage { - // Use LifeCycleProtocol.newBuilder() to construct. - private LifeCycleProtocol() { - initFields(); - } - private LifeCycleProtocol(boolean noInit) {} - - private static final LifeCycleProtocol defaultInstance; - public static LifeCycleProtocol getDefaultInstance() { - return defaultInstance; - } - - public LifeCycleProtocol getDefaultInstanceForType() { - return defaultInstance; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_LifeCycleProtocol_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_LifeCycleProtocol_fieldAccessorTable; - } - - // required .LifeCycleType lifeCycle = 1; - public static final int LIFECYCLE_FIELD_NUMBER = 1; - private boolean hasLifeCycle; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType lifeCycle_; - public boolean hasLifeCycle() { return hasLifeCycle; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle() { return lifeCycle_; } - - // optional string preRestart = 2; - public static final int PRERESTART_FIELD_NUMBER = 2; - private boolean hasPreRestart; - private java.lang.String preRestart_ = ""; - public boolean hasPreRestart() { return hasPreRestart; } - public java.lang.String getPreRestart() { return preRestart_; } - - // optional string postRestart = 3; - public static final int POSTRESTART_FIELD_NUMBER = 3; - private boolean hasPostRestart; - private java.lang.String postRestart_ = ""; - public boolean hasPostRestart() { return hasPostRestart; } - public java.lang.String getPostRestart() { return postRestart_; } - - // optional string init = 4; - public static final int INIT_FIELD_NUMBER = 4; - private boolean hasInit; - private java.lang.String init_ = ""; - public boolean hasInit() { return hasInit; } - public java.lang.String getInit() { return init_; } - - // optional string shutdown = 5; - public static final int SHUTDOWN_FIELD_NUMBER = 5; - private boolean hasShutdown; - private java.lang.String shutdown_ = ""; - public boolean hasShutdown() { return hasShutdown; } - public java.lang.String getShutdown() { return shutdown_; } - - private void initFields() { - lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT; - } - public final boolean isInitialized() { - if (!hasLifeCycle) return false; - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (hasLifeCycle()) { - output.writeEnum(1, getLifeCycle().getNumber()); - } - if (hasPreRestart()) { - output.writeString(2, getPreRestart()); - } - if (hasPostRestart()) { - output.writeString(3, getPostRestart()); - } - if (hasInit()) { - output.writeString(4, getInit()); - } - if (hasShutdown()) { - output.writeString(5, getShutdown()); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - if (hasLifeCycle()) { - size += com.google.protobuf.CodedOutputStream - .computeEnumSize(1, getLifeCycle().getNumber()); - } - if (hasPreRestart()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(2, getPreRestart()); - } - if (hasPostRestart()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(3, getPostRestart()); - } - if (hasInit()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(4, getInit()); - } - if (hasShutdown()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(5, getShutdown()); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol result; - - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol(); - return builder; - } - - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol internalGetResult() { - return result; - } - - public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol(); - return this; - } - - public Builder clone() { - return create().mergeFrom(result); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDescriptor(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getDefaultInstanceForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); - } - - public boolean isInitialized() { - return result.isInitialized(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol build() { - if (result != null && !isInitialized()) { - throw newUninitializedMessageException(result); - } - return buildPartial(); - } - - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return buildPartial(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); - } - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol returnMe = result; - result = null; - return returnMe; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol) { - return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol other) { - if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance()) return this; - if (other.hasLifeCycle()) { - setLifeCycle(other.getLifeCycle()); - } - if (other.hasPreRestart()) { - setPreRestart(other.getPreRestart()); - } - if (other.hasPostRestart()) { - setPostRestart(other.getPostRestart()); - } - if (other.hasInit()) { - setInit(other.getInit()); - } - if (other.hasShutdown()) { - setShutdown(other.getShutdown()); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - return this; - } - break; - } - case 8: { - int rawValue = input.readEnum(); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType value = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.valueOf(rawValue); - if (value == null) { - unknownFields.mergeVarintField(1, rawValue); - } else { - setLifeCycle(value); - } - break; - } - case 18: { - setPreRestart(input.readString()); - break; - } - case 26: { - setPostRestart(input.readString()); - break; - } - case 34: { - setInit(input.readString()); - break; - } - case 42: { - setShutdown(input.readString()); - break; - } - } - } - } - - - // required .LifeCycleType lifeCycle = 1; - public boolean hasLifeCycle() { - return result.hasLifeCycle(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle() { - return result.getLifeCycle(); - } - public Builder setLifeCycle(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasLifeCycle = true; - result.lifeCycle_ = value; - return this; - } - public Builder clearLifeCycle() { - result.hasLifeCycle = false; - result.lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT; - return this; - } - - // optional string preRestart = 2; - public boolean hasPreRestart() { - return result.hasPreRestart(); - } - public java.lang.String getPreRestart() { - return result.getPreRestart(); - } - public Builder setPreRestart(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasPreRestart = true; - result.preRestart_ = value; - return this; - } - public Builder clearPreRestart() { - result.hasPreRestart = false; - result.preRestart_ = getDefaultInstance().getPreRestart(); - return this; - } - - // optional string postRestart = 3; - public boolean hasPostRestart() { - return result.hasPostRestart(); - } - public java.lang.String getPostRestart() { - return result.getPostRestart(); - } - public Builder setPostRestart(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasPostRestart = true; - result.postRestart_ = value; - return this; - } - public Builder clearPostRestart() { - result.hasPostRestart = false; - result.postRestart_ = getDefaultInstance().getPostRestart(); - return this; - } - - // optional string init = 4; - public boolean hasInit() { - return result.hasInit(); - } - public java.lang.String getInit() { - return result.getInit(); - } - public Builder setInit(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasInit = true; - result.init_ = value; - return this; - } - public Builder clearInit() { - result.hasInit = false; - result.init_ = getDefaultInstance().getInit(); - return this; - } - - // optional string shutdown = 5; - public boolean hasShutdown() { - return result.hasShutdown(); - } - public java.lang.String getShutdown() { - return result.getShutdown(); - } - public Builder setShutdown(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasShutdown = true; - result.shutdown_ = value; - return this; - } - public Builder clearShutdown() { - result.hasShutdown = false; - result.shutdown_ = getDefaultInstance().getShutdown(); - return this; - } - - // @@protoc_insertion_point(builder_scope:LifeCycleProtocol) - } - - static { - defaultInstance = new LifeCycleProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:LifeCycleProtocol) - } - - public static final class AddressProtocol extends - com.google.protobuf.GeneratedMessage { - // Use AddressProtocol.newBuilder() to construct. - private AddressProtocol() { - initFields(); - } - private AddressProtocol(boolean noInit) {} - - private static final AddressProtocol defaultInstance; - public static AddressProtocol getDefaultInstance() { - return defaultInstance; - } - - public AddressProtocol getDefaultInstanceForType() { - return defaultInstance; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_AddressProtocol_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_AddressProtocol_fieldAccessorTable; - } - - // required string hostname = 1; - public static final int HOSTNAME_FIELD_NUMBER = 1; - private boolean hasHostname; - private java.lang.String hostname_ = ""; - public boolean hasHostname() { return hasHostname; } - public java.lang.String getHostname() { return hostname_; } - - // required uint32 port = 2; - public static final int PORT_FIELD_NUMBER = 2; - private boolean hasPort; - private int port_ = 0; - public boolean hasPort() { return hasPort; } - public int getPort() { return port_; } - - private void initFields() { - } - public final boolean isInitialized() { - if (!hasHostname) return false; - if (!hasPort) return false; - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (hasHostname()) { - output.writeString(1, getHostname()); - } - if (hasPort()) { - output.writeUInt32(2, getPort()); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - if (hasHostname()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(1, getHostname()); - } - if (hasPort()) { - size += com.google.protobuf.CodedOutputStream - .computeUInt32Size(2, getPort()); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol result; - - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol(); - return builder; - } - - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol internalGetResult() { - return result; - } - - public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol(); - return this; - } - - public Builder clone() { - return create().mergeFrom(result); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDescriptor(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getDefaultInstanceForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); - } - - public boolean isInitialized() { - return result.isInitialized(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol build() { - if (result != null && !isInitialized()) { - throw newUninitializedMessageException(result); - } - return buildPartial(); - } - - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return buildPartial(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); - } - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol returnMe = result; - result = null; - return returnMe; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol) { - return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol other) { - if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) return this; - if (other.hasHostname()) { - setHostname(other.getHostname()); - } - if (other.hasPort()) { - setPort(other.getPort()); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - return this; - } - break; - } - case 10: { - setHostname(input.readString()); - break; - } - case 16: { - setPort(input.readUInt32()); - break; - } - } - } - } - - - // required string hostname = 1; - public boolean hasHostname() { - return result.hasHostname(); - } - public java.lang.String getHostname() { - return result.getHostname(); - } - public Builder setHostname(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasHostname = true; - result.hostname_ = value; - return this; - } - public Builder clearHostname() { - result.hasHostname = false; - result.hostname_ = getDefaultInstance().getHostname(); - return this; - } - - // required uint32 port = 2; - public boolean hasPort() { - return result.hasPort(); - } - public int getPort() { - return result.getPort(); - } - public Builder setPort(int value) { - result.hasPort = true; - result.port_ = value; - return this; - } - public Builder clearPort() { - result.hasPort = false; - result.port_ = 0; - return this; - } - - // @@protoc_insertion_point(builder_scope:AddressProtocol) - } - - static { - defaultInstance = new AddressProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:AddressProtocol) - } - - public static final class ExceptionProtocol extends - com.google.protobuf.GeneratedMessage { - // Use ExceptionProtocol.newBuilder() to construct. - private ExceptionProtocol() { - initFields(); - } - private ExceptionProtocol(boolean noInit) {} - - private static final ExceptionProtocol defaultInstance; - public static ExceptionProtocol getDefaultInstance() { - return defaultInstance; - } - - public ExceptionProtocol getDefaultInstanceForType() { - return defaultInstance; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ExceptionProtocol_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ExceptionProtocol_fieldAccessorTable; - } - - // required string classname = 1; - public static final int CLASSNAME_FIELD_NUMBER = 1; - private boolean hasClassname; - private java.lang.String classname_ = ""; - public boolean hasClassname() { return hasClassname; } - public java.lang.String getClassname() { return classname_; } - - // required string message = 2; - public static final int MESSAGE_FIELD_NUMBER = 2; - private boolean hasMessage; - private java.lang.String message_ = ""; - public boolean hasMessage() { return hasMessage; } - public java.lang.String getMessage() { return message_; } - - private void initFields() { - } - public final boolean isInitialized() { - if (!hasClassname) return false; - if (!hasMessage) return false; - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (hasClassname()) { - output.writeString(1, getClassname()); - } - if (hasMessage()) { - output.writeString(2, getMessage()); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - if (hasClassname()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(1, getClassname()); - } - if (hasMessage()) { - size += com.google.protobuf.CodedOutputStream - .computeStringSize(2, getMessage()); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol result; - - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol(); - return builder; - } - - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol internalGetResult() { - return result; - } - - public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol(); - return this; - } - - public Builder clone() { - return create().mergeFrom(result); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDescriptor(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getDefaultInstanceForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); - } - - public boolean isInitialized() { - return result.isInitialized(); - } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol build() { - if (result != null && !isInitialized()) { - throw newUninitializedMessageException(result); - } - return buildPartial(); - } - - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return buildPartial(); - } - - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); - } - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol returnMe = result; - result = null; - return returnMe; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol) { - return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol other) { - if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance()) return this; - if (other.hasClassname()) { - setClassname(other.getClassname()); - } - if (other.hasMessage()) { - setMessage(other.getMessage()); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - return this; - } - break; - } - case 10: { - setClassname(input.readString()); - break; - } - case 18: { - setMessage(input.readString()); - break; - } - } - } - } - - - // required string classname = 1; - public boolean hasClassname() { - return result.hasClassname(); - } - public java.lang.String getClassname() { - return result.getClassname(); - } - public Builder setClassname(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasClassname = true; - result.classname_ = value; - return this; - } - public Builder clearClassname() { - result.hasClassname = false; - result.classname_ = getDefaultInstance().getClassname(); - return this; - } - - // required string message = 2; - public boolean hasMessage() { - return result.hasMessage(); - } - public java.lang.String getMessage() { - return result.getMessage(); - } - public Builder setMessage(java.lang.String value) { - if (value == null) { - throw new NullPointerException(); - } - result.hasMessage = true; - result.message_ = value; - return this; - } - public Builder clearMessage() { - result.hasMessage = false; - result.message_ = getDefaultInstance().getMessage(); - return this; - } - - // @@protoc_insertion_point(builder_scope:ExceptionProtocol) - } - - static { - defaultInstance = new ExceptionProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:ExceptionProtocol) - } - - private static com.google.protobuf.Descriptors.Descriptor - internal_static_RemoteActorRefProtocol_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_RemoteActorRefProtocol_fieldAccessorTable; - private static com.google.protobuf.Descriptors.Descriptor - internal_static_SerializedActorRefProtocol_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_SerializedActorRefProtocol_fieldAccessorTable; - private static com.google.protobuf.Descriptors.Descriptor - internal_static_MessageProtocol_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_MessageProtocol_fieldAccessorTable; - private static com.google.protobuf.Descriptors.Descriptor - internal_static_ActorInfoProtocol_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_ActorInfoProtocol_fieldAccessorTable; - private static com.google.protobuf.Descriptors.Descriptor - internal_static_TypedActorInfoProtocol_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_TypedActorInfoProtocol_fieldAccessorTable; - private static com.google.protobuf.Descriptors.Descriptor - internal_static_RemoteRequestProtocol_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_RemoteRequestProtocol_fieldAccessorTable; - private static com.google.protobuf.Descriptors.Descriptor - internal_static_RemoteReplyProtocol_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_RemoteReplyProtocol_fieldAccessorTable; - private static com.google.protobuf.Descriptors.Descriptor - internal_static_LifeCycleProtocol_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_LifeCycleProtocol_fieldAccessorTable; - private static com.google.protobuf.Descriptors.Descriptor - internal_static_AddressProtocol_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_AddressProtocol_fieldAccessorTable; - private static com.google.protobuf.Descriptors.Descriptor - internal_static_ExceptionProtocol_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_ExceptionProtocol_fieldAccessorTable; - - public static com.google.protobuf.Descriptors.FileDescriptor - getDescriptor() { - return descriptor; - } - private static com.google.protobuf.Descriptors.FileDescriptor - 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\"\200\003\n\032SerializedAct" + - "orRefProtocol\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\017originalAddre" + - "ss\030\004 \002(\0132\020.AddressProtocol\022\025\n\ractorInsta" + - "nce\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\tlifeCycle\030\n \001(\013", - "2\022.LifeCycleProtocol\022+\n\nsupervisor\030\013 \001(\013" + - "2\027.RemoteActorRefProtocol\022\024\n\014hotswapStac" + - "k\030\014 \001(\014\022(\n\010messages\030\r \003(\0132\026.RemoteReques" + - "tProtocol\"r\n\017MessageProtocol\0225\n\023serializ" + - "ationScheme\030\001 \002(\0162\030.SerializationSchemeT" + - "ype\022\017\n\007message\030\002 \002(\014\022\027\n\017messageManifest\030" + - "\003 \001(\014\"\222\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\tac" + - "torType\030\004 \002(\0162\n.ActorType\022/\n\016typedActorI" + - "nfo\030\005 \001(\0132\027.TypedActorInfoProtocol\";\n\026Ty", - "pedActorInfoProtocol\022\021\n\tinterface\030\001 \002(\t\022" + - "\016\n\006method\030\002 \002(\t\"\300\001\n\025RemoteRequestProtoco" + - "l\022\n\n\002id\030\001 \002(\004\022!\n\007message\030\002 \002(\0132\020.Message" + - "Protocol\022%\n\tactorInfo\030\003 \002(\0132\022.ActorInfoP" + - "rotocol\022\020\n\010isOneWay\030\004 \002(\010\022\026\n\016supervisorU" + - "uid\030\005 \001(\t\022\'\n\006sender\030\006 \001(\0132\027.RemoteActorR" + - "efProtocol\"\252\001\n\023RemoteReplyProtocol\022\n\n\002id" + - "\030\001 \002(\004\022!\n\007message\030\002 \001(\0132\020.MessageProtoco" + - "l\022%\n\texception\030\003 \001(\0132\022.ExceptionProtocol" + - "\022\026\n\016supervisorUuid\030\004 \001(\t\022\017\n\007isActor\030\005 \002(", - "\010\022\024\n\014isSuccessful\030\006 \002(\010\"\177\n\021LifeCycleProt" + - "ocol\022!\n\tlifeCycle\030\001 \002(\0162\016.LifeCycleType\022" + - "\022\n\npreRestart\030\002 \001(\t\022\023\n\013postRestart\030\003 \001(\t" + - "\022\014\n\004init\030\004 \001(\t\022\020\n\010shutdown\030\005 \001(\t\"1\n\017Addr" + - "essProtocol\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\013SCA" + - "LA_ACTOR\020\001\022\016\n\nJAVA_ACTOR\020\002\022\017\n\013TYPED_ACTO" + - "R\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_JS", - "ON\020\004\022\014\n\010PROTOBUF\020\005*-\n\rLifeCycleType\022\r\n\tP" + - "ERMANENT\020\001\022\r\n\tTEMPORARY\020\002B-\n)se.scalable" + - "solutions.akka.remote.protocolH\001" - }; - com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = - new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { - public com.google.protobuf.ExtensionRegistry assignDescriptors( - com.google.protobuf.Descriptors.FileDescriptor root) { - descriptor = root; - internal_static_RemoteActorRefProtocol_descriptor = - getDescriptor().getMessageTypes().get(0); - internal_static_RemoteActorRefProtocol_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_RemoteActorRefProtocol_descriptor, - new java.lang.String[] { "Uuid", "ActorClassname", "HomeAddress", "Timeout", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder.class); - internal_static_SerializedActorRefProtocol_descriptor = - getDescriptor().getMessageTypes().get(1); - internal_static_SerializedActorRefProtocol_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_SerializedActorRefProtocol_descriptor, - new java.lang.String[] { "Uuid", "Id", "ActorClassname", "OriginalAddress", "ActorInstance", "SerializerClassname", "IsTransactor", "Timeout", "ReceiveTimeout", "LifeCycle", "Supervisor", "HotswapStack", "Messages", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder.class); - internal_static_MessageProtocol_descriptor = - getDescriptor().getMessageTypes().get(2); - internal_static_MessageProtocol_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_MessageProtocol_descriptor, - new java.lang.String[] { "SerializationScheme", "Message", "MessageManifest", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder.class); - internal_static_ActorInfoProtocol_descriptor = - getDescriptor().getMessageTypes().get(3); - internal_static_ActorInfoProtocol_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_ActorInfoProtocol_descriptor, - new java.lang.String[] { "Uuid", "Target", "Timeout", "ActorType", "TypedActorInfo", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder.class); - internal_static_TypedActorInfoProtocol_descriptor = - getDescriptor().getMessageTypes().get(4); - internal_static_TypedActorInfoProtocol_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_TypedActorInfoProtocol_descriptor, - new java.lang.String[] { "Interface", "Method", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.Builder.class); - internal_static_RemoteRequestProtocol_descriptor = - getDescriptor().getMessageTypes().get(5); - internal_static_RemoteRequestProtocol_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_RemoteRequestProtocol_descriptor, - new java.lang.String[] { "Id", "Message", "ActorInfo", "IsOneWay", "SupervisorUuid", "Sender", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder.class); - internal_static_RemoteReplyProtocol_descriptor = - getDescriptor().getMessageTypes().get(6); - internal_static_RemoteReplyProtocol_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_RemoteReplyProtocol_descriptor, - new java.lang.String[] { "Id", "Message", "Exception", "SupervisorUuid", "IsActor", "IsSuccessful", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.Builder.class); - internal_static_LifeCycleProtocol_descriptor = - getDescriptor().getMessageTypes().get(7); - internal_static_LifeCycleProtocol_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_LifeCycleProtocol_descriptor, - new java.lang.String[] { "LifeCycle", "PreRestart", "PostRestart", "Init", "Shutdown", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder.class); - internal_static_AddressProtocol_descriptor = - getDescriptor().getMessageTypes().get(8); - internal_static_AddressProtocol_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_AddressProtocol_descriptor, - new java.lang.String[] { "Hostname", "Port", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder.class); - internal_static_ExceptionProtocol_descriptor = - getDescriptor().getMessageTypes().get(9); - internal_static_ExceptionProtocol_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_ExceptionProtocol_descriptor, - new java.lang.String[] { "Classname", "Message", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder.class); - return null; - } - }; - com.google.protobuf.Descriptors.FileDescriptor - .internalBuildGeneratedFileFrom(descriptorData, - new com.google.protobuf.Descriptors.FileDescriptor[] { - }, assigner); - } - - public static void internalForceInit() {} - - // @@protoc_insertion_point(outer_class_scope) -} diff --git a/akka-remote/src/test/resources/META-INF/aop.xml b/akka-remote/src/test/resources/META-INF/aop.xml index bdc167ca54..be133a51b8 100644 --- a/akka-remote/src/test/resources/META-INF/aop.xml +++ b/akka-remote/src/test/resources/META-INF/aop.xml @@ -2,6 +2,7 @@ + diff --git a/akka-remote/src/test/resources/logback-test.xml b/akka-remote/src/test/resources/logback-test.xml deleted file mode 100644 index 78eae40ec4..0000000000 --- a/akka-remote/src/test/resources/logback-test.xml +++ /dev/null @@ -1,21 +0,0 @@ - - - - - - - - - - - - - - [%4p] [%d{ISO8601}] [%t] %c{1}: %m%n - - - - - - - diff --git a/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala b/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala index 7ff46ab910..6670722b02 100644 --- a/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala +++ b/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala @@ -93,6 +93,7 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite { actor.stop } + @Test def shouldSendOneWayAndReceiveReply = { val actor = actorOf[SendOneWayAndReplyReceiverActor] @@ -103,7 +104,7 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite { sender.actor.asInstanceOf[SendOneWayAndReplySenderActor].sendTo = actor sender.start sender.actor.asInstanceOf[SendOneWayAndReplySenderActor].sendOff - assert(SendOneWayAndReplySenderActor.latch.await(1, TimeUnit.SECONDS)) + assert(SendOneWayAndReplySenderActor.latch.await(3, TimeUnit.SECONDS)) assert(sender.actor.asInstanceOf[SendOneWayAndReplySenderActor].state.isDefined === true) assert("World" === sender.actor.asInstanceOf[SendOneWayAndReplySenderActor].state.get.asInstanceOf[String]) actor.stop @@ -134,6 +135,6 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite { assert("Expected exception; to test fault-tolerance" === e.getMessage()) } actor.stop - } + } } diff --git a/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala b/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala index 780828c310..8b28b35f57 100644 --- a/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala +++ b/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala @@ -4,10 +4,7 @@ package se.scalablesolutions.akka.actor.remote -import org.scalatest.Spec -import org.scalatest.Assertions import org.scalatest.matchers.ShouldMatchers -import org.scalatest.BeforeAndAfterAll import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith @@ -19,6 +16,7 @@ import se.scalablesolutions.akka.actor._ import se.scalablesolutions.akka.remote.{RemoteServer, RemoteClient} import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, BlockingQueue} +import org.scalatest.{BeforeAndAfterEach, Spec, Assertions, BeforeAndAfterAll} object RemoteTypedActorSpec { val HOSTNAME = "localhost" @@ -40,7 +38,7 @@ object RemoteTypedActorLog { class RemoteTypedActorSpec extends Spec with ShouldMatchers with - BeforeAndAfterAll { + BeforeAndAfterEach with BeforeAndAfterAll { import RemoteTypedActorLog._ import RemoteTypedActorSpec._ @@ -82,6 +80,10 @@ class RemoteTypedActorSpec extends ActorRegistry.shutdownAll } + override def afterEach() { + server.typedActors.clear + } + describe("Remote Typed Actor ") { it("should receive one-way message") { diff --git a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala index 59cfe3778d..59f122c656 100644 --- a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala +++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala @@ -5,8 +5,8 @@ import org.scalatest.junit.JUnitSuite import org.junit.{Test, Before, After} import se.scalablesolutions.akka.remote.{RemoteServer, RemoteClient} -import se.scalablesolutions.akka.actor.{ActorRef, Actor} -import Actor._ +import se.scalablesolutions.akka.actor.Actor._ +import se.scalablesolutions.akka.actor.{ActorRegistry, ActorRef, Actor} object ServerInitiatedRemoteActorSpec { val HOSTNAME = "localhost" @@ -67,7 +67,7 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite { Thread.sleep(1000) } - // make sure the servers shutdown cleanly after the test has finished + // make sure the servers postStop cleanly after the test has finished @After def finished { try { @@ -79,6 +79,7 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite { } } + @Test def shouldSendWithBang { val actor = RemoteClient.actorFor( @@ -132,5 +133,50 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite { } actor.stop } + + @Test + def reflectiveAccessShouldNotCreateNewRemoteServerObject { + val server1 = new RemoteServer() + server1.start("localhost", 9990) + + var found = RemoteServer.serverFor("localhost", 9990) + assert(found.isDefined, "sever not found") + + val a = actor { case _ => } + + found = RemoteServer.serverFor("localhost", 9990) + assert(found.isDefined, "sever not found after creating an actor") + } + + + @Test + def shouldNotRecreateRegisteredActor { + server.register(actorOf[RemoteActorSpecActorUnidirectional]) + val actor = RemoteClient.actorFor("se.scalablesolutions.akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", HOSTNAME, PORT) + val numberOfActorsInRegistry = ActorRegistry.actors.length + actor ! "OneWay" + assert(RemoteActorSpecActorUnidirectional.latch.await(1, TimeUnit.SECONDS)) + assert(numberOfActorsInRegistry === ActorRegistry.actors.length) + actor.stop + } + + @Test + def shouldUseServiceNameAsIdForRemoteActorRef { + server.register(actorOf[RemoteActorSpecActorUnidirectional]) + server.register("my-service", actorOf[RemoteActorSpecActorUnidirectional]) + val actor1 = RemoteClient.actorFor("se.scalablesolutions.akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", HOSTNAME, PORT) + val actor2 = RemoteClient.actorFor("my-service", HOSTNAME, PORT) + val actor3 = RemoteClient.actorFor("my-service", HOSTNAME, PORT) + + actor1 ! "OneWay" + actor2 ! "OneWay" + actor3 ! "OneWay" + + assert(actor1.uuid != actor2.uuid) + assert(actor1.uuid != actor3.uuid) + assert(actor1.id != actor2.id) + assert(actor2.id == actor3.id) + } + } diff --git a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala new file mode 100644 index 0000000000..b800fbf2c3 --- /dev/null +++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala @@ -0,0 +1,112 @@ +/** + * Copyright (C) 2009-2010 Scalable Solutions AB + */ + +package se.scalablesolutions.akka.actor.remote + +import org.scalatest.Spec +import org.scalatest.matchers.ShouldMatchers +import org.scalatest.BeforeAndAfterAll +import org.scalatest.junit.JUnitRunner +import org.junit.runner.RunWith + +import java.util.concurrent.TimeUnit + +import se.scalablesolutions.akka.remote.{RemoteServer, RemoteClient} +import se.scalablesolutions.akka.actor._ +import RemoteTypedActorLog._ + +object ServerInitiatedRemoteTypedActorSpec { + val HOSTNAME = "localhost" + val PORT = 9990 + var server: RemoteServer = null +} + +@RunWith(classOf[JUnitRunner]) +class ServerInitiatedRemoteTypedActorSpec extends + Spec with + ShouldMatchers with + BeforeAndAfterAll { + import ServerInitiatedRemoteTypedActorSpec._ + + private val unit = TimeUnit.MILLISECONDS + + + override def beforeAll = { + server = new RemoteServer() + server.start(HOSTNAME, PORT) + + val typedActor = TypedActor.newInstance(classOf[RemoteTypedActorOne], classOf[RemoteTypedActorOneImpl], 1000) + server.registerTypedActor("typed-actor-service", typedActor) + + Thread.sleep(1000) + } + + // make sure the servers shutdown cleanly after the test has finished + override def afterAll = { + try { + server.shutdown + RemoteClient.shutdownAll + Thread.sleep(1000) + } catch { + case e => () + } + } + + describe("Server managed remote typed Actor ") { + + it("should receive one-way message") { + clearMessageLogs + val actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, HOSTNAME, PORT) + expect("oneway") { + actor.oneWay + oneWayLog.poll(5, TimeUnit.SECONDS) + } + } + + it("should respond to request-reply message") { + clearMessageLogs + val actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, HOSTNAME, PORT) + expect("pong") { + actor.requestReply("ping") + } + } + + it("should not recreate registered actors") { + val actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, HOSTNAME, PORT) + val numberOfActorsInRegistry = ActorRegistry.actors.length + expect("oneway") { + actor.oneWay + oneWayLog.poll(5, TimeUnit.SECONDS) + } + assert(numberOfActorsInRegistry === ActorRegistry.actors.length) + } + + it("should support multiple variants to get the actor from client side") { + var actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, HOSTNAME, PORT) + expect("oneway") { + actor.oneWay + oneWayLog.poll(5, TimeUnit.SECONDS) + } + actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", HOSTNAME, PORT) + expect("oneway") { + actor.oneWay + oneWayLog.poll(5, TimeUnit.SECONDS) + } + actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, HOSTNAME, PORT, this.getClass().getClassLoader) + expect("oneway") { + actor.oneWay + oneWayLog.poll(5, TimeUnit.SECONDS) + } + } + + 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) + server.unregisterTypedActor("my-test-service") + assert(server.typedActors().get("my-test-service") == null) + } + } +} + diff --git a/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala b/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala index 011c656f8d..e05e4b0394 100644 --- a/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala +++ b/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala @@ -47,7 +47,7 @@ class ProtobufActorMessageSerializationSpec extends JUnitSuite { Thread.sleep(1000) } - // make sure the servers shutdown cleanly after the test has finished + // make sure the servers postStop cleanly after the test has finished @After def finished() { server.shutdown diff --git a/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala b/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala index a9bcc35790..7e8babe168 100644 --- a/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala +++ b/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala @@ -7,6 +7,7 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith +import se.scalablesolutions.akka.serialization._ import se.scalablesolutions.akka.actor._ import ActorSerialization._ import Actor._ @@ -17,7 +18,6 @@ class SerializableTypeClassActorSpec extends ShouldMatchers with BeforeAndAfterAll { - import se.scalablesolutions.akka.serialization.Serializer object BinaryFormatMyActor { implicit object MyActorFormat extends Format[MyActor] { diff --git a/akka-remote/src/test/scala/serialization/Ticket436Spec.scala b/akka-remote/src/test/scala/serialization/Ticket436Spec.scala new file mode 100644 index 0000000000..042f3f07be --- /dev/null +++ b/akka-remote/src/test/scala/serialization/Ticket436Spec.scala @@ -0,0 +1,49 @@ +package se.scalablesolutions.akka.actor.serialization + + +import org.scalatest.Spec +import org.scalatest.matchers.ShouldMatchers +import org.scalatest.BeforeAndAfterAll +import org.scalatest.junit.JUnitRunner +import org.junit.runner.RunWith + +import se.scalablesolutions.akka.serialization.Serializer +import se.scalablesolutions.akka.serialization.Serializable.ScalaJSON +import scala.reflect._ +import scala.annotation.target._ +import sjson.json.JSONTypeHint + +@BeanInfo class MyJsonObject(val key: String, + @(JSONTypeHint @field)(value = classOf[Int]) + val map: Map[String, Int], + val standAloneInt: Int) extends ScalaJSON { + private def this() = this(null, null, -1) + override def toString(): String = try { + val mapValue: Int = map.getOrElse(key, -1) + println("Map value: %s".format(mapValue.asInstanceOf[AnyRef].getClass)) + "Key: %s, Map value: %d, Stand Alone Int: %d".format(key, mapValue, standAloneInt) + } catch { + case e: ClassCastException => e.getMessage + case _ => "Unknown error" + } +} + +@RunWith(classOf[JUnitRunner]) +class Ticket436Spec extends + Spec with + ShouldMatchers with + BeforeAndAfterAll { + + describe("Serialization of Maps containing Int") { + it("should be able to serialize and de-serialize preserving the data types of the Map") { + val key: String = "myKey" + val value: Int = 123 + val standAloneInt: Int = 35 + val message = new MyJsonObject(key, Map(key -> value), standAloneInt) + + val json = message.toJSON + val copy = Serializer.ScalaJSON.fromJSON[MyJsonObject](json) + copy.asInstanceOf[MyJsonObject].map.get("myKey").get.isInstanceOf[Int] should equal(true) + } + } +} diff --git a/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala b/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala index 6588c7cdc4..d230bbaffe 100644 --- a/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala +++ b/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala @@ -7,6 +7,7 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith +import se.scalablesolutions.akka.serialization._ import se.scalablesolutions.akka.actor._ import ActorSerialization._ import Actor._ @@ -17,8 +18,6 @@ class UntypedActorSerializationSpec extends ShouldMatchers with BeforeAndAfterAll { - import se.scalablesolutions.akka.serialization.Serializer - class MyUntypedActorFormat extends Format[MyUntypedActor] { def fromBinary(bytes: Array[Byte], act: MyUntypedActor) = { val p = Serializer.Protobuf.fromBinary(bytes, Some(classOf[ProtobufProtocol.Counter])).asInstanceOf[ProtobufProtocol.Counter] diff --git a/akka-remote/src/test/scala/ticket/Ticket434Spec.scala b/akka-remote/src/test/scala/ticket/Ticket434Spec.scala new file mode 100644 index 0000000000..b27c17cfe5 --- /dev/null +++ b/akka-remote/src/test/scala/ticket/Ticket434Spec.scala @@ -0,0 +1,46 @@ +/** + * Copyright (C) 2009-2010 Scalable Solutions AB + */ +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.remote.ServerInitiatedRemoteActorSpec.RemoteActorSpecActorUnidirectional +import java.util.concurrent.TimeUnit +import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer} +import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._ + + +class Ticket434Spec extends Spec with ShouldMatchers { + + describe("A server managed remote actor") { + it("should possible be use a custom service name containing ':'") { + val server = new RemoteServer().start("localhost", 9999) + server.register("my:service", actorOf[RemoteActorSpecActorUnidirectional]) + + val actor = RemoteClient.actorFor("my:service", 5000L, "localhost", 9999) + actor ! "OneWay" + + assert(RemoteActorSpecActorUnidirectional.latch.await(1, TimeUnit.SECONDS)) + actor.stop + + server.shutdown + RemoteClient.shutdownAll + } + } + + describe("The ActorInfoProtocol") { + it("should be possible to set the acor id and uuuid") { + val actorInfoBuilder = ActorInfoProtocol.newBuilder + .setUuid("unique-id") + .setId("some-id") + .setTarget("actorClassName") + .setTimeout(5000L) + .setActorType(ActorType.SCALA_ACTOR) + val actorInfo = actorInfoBuilder.build + assert(actorInfo.getUuid === "unique-id") + assert(actorInfo.getId === "some-id") + } + } +} diff --git a/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer1Impl.java b/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer1Impl.java index 71eda6c270..0e05d5036c 100644 --- a/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer1Impl.java +++ b/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer1Impl.java @@ -1,8 +1,5 @@ package sample.camel; -import org.apache.camel.Body; -import org.apache.camel.Header; - import se.scalablesolutions.akka.actor.TypedActor; /** @@ -10,7 +7,7 @@ import se.scalablesolutions.akka.actor.TypedActor; */ public class RemoteTypedConsumer1Impl extends TypedActor implements RemoteTypedConsumer1 { - public String foo(@Body String body, @Header("name") String header) { + public String foo(String body, String header) { return String.format("remote1: body=%s header=%s", body, header); } } diff --git a/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer2.java b/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer2.java new file mode 100644 index 0000000000..3b2494ee4e --- /dev/null +++ b/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer2.java @@ -0,0 +1,15 @@ +package sample.camel; + +import org.apache.camel.Body; +import org.apache.camel.Header; +import se.scalablesolutions.akka.camel.consume; + +/** + * @author Martin Krasser + */ +public interface RemoteTypedConsumer2 { + + @consume("jetty:http://localhost:6644/camel/remote-typed-actor-2") + public String foo(@Body String body, @Header("name") String header); + +} diff --git a/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer2Impl.java b/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer2Impl.java index 68e208d834..5c76485aa7 100644 --- a/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer2Impl.java +++ b/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer2Impl.java @@ -1,16 +1,13 @@ package sample.camel; -import org.apache.camel.Body; -import org.apache.camel.Header; -import se.scalablesolutions.akka.camel.consume; +import se.scalablesolutions.akka.actor.TypedActor; /** * @author Martin Krasser */ -public class RemoteTypedConsumer2Impl { +public class RemoteTypedConsumer2Impl extends TypedActor implements RemoteTypedConsumer2 { - @consume("jetty:http://localhost:6644/camel/remote-typed-actor-2") - public String foo(@Body String body, @Header("name") String header) { + public String foo(String body, String header) { return String.format("remote2: body=%s header=%s", body, header); } diff --git a/akka-samples/akka-sample-camel/src/main/scala/ClientApplication.scala b/akka-samples/akka-sample-camel/src/main/scala/ClientApplication.scala index 7f671b0e45..9fab1b9649 100644 --- a/akka-samples/akka-sample-camel/src/main/scala/ClientApplication.scala +++ b/akka-samples/akka-sample-camel/src/main/scala/ClientApplication.scala @@ -1,7 +1,7 @@ package sample.camel import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.actor.{TypedActor, Actor, ActorRef} +import se.scalablesolutions.akka.actor.TypedActor import se.scalablesolutions.akka.camel.Message import se.scalablesolutions.akka.remote.RemoteClient @@ -10,22 +10,20 @@ import se.scalablesolutions.akka.remote.RemoteClient */ object ClientApplication extends Application { - // - // TODO: completion of example - // - - val actor1 = actorOf[RemoteActor1] + val actor1 = actorOf[RemoteActor1].start val actor2 = RemoteClient.actorFor("remote2", "localhost", 7777) - val actobj1 = TypedActor.newRemoteInstance( - classOf[RemoteTypedConsumer1], classOf[RemoteTypedConsumer1Impl], "localhost", 7777) - //val actobj2 = TODO: create reference to server-managed typed actor (RemoteTypedConsumer2Impl) + val typedActor1 = TypedActor.newRemoteInstance( + classOf[RemoteTypedConsumer1], + classOf[RemoteTypedConsumer1Impl], "localhost", 7777) - actor1.start + val typedActor2 = RemoteClient.typedActorFor( + classOf[RemoteTypedConsumer2], "remote3", "localhost", 7777) println(actor1 !! Message("actor1")) // activates and publishes actor remotely println(actor2 !! Message("actor2")) // actor already activated and published remotely - println(actobj1.foo("x", "y")) // activates and publishes typed actor methods remotely - // ... + println(typedActor1.foo("x1", "y1")) // activates and publishes typed actor methods remotely + println(typedActor2.foo("x2", "y2")) // typed actor methods already activated and published remotely + } diff --git a/akka-samples/akka-sample-camel/src/main/scala/ServerApplication.scala b/akka-samples/akka-sample-camel/src/main/scala/ServerApplication.scala index 3eb40aaca9..b3ef404841 100644 --- a/akka-samples/akka-sample-camel/src/main/scala/ServerApplication.scala +++ b/akka-samples/akka-sample-camel/src/main/scala/ServerApplication.scala @@ -3,6 +3,7 @@ package sample.camel import se.scalablesolutions.akka.actor.Actor._ import se.scalablesolutions.akka.remote.RemoteNode import se.scalablesolutions.akka.camel.CamelServiceManager +import se.scalablesolutions.akka.actor.TypedActor /** * @author Martin Krasser @@ -10,11 +11,14 @@ import se.scalablesolutions.akka.camel.CamelServiceManager object ServerApplication extends Application { import CamelServiceManager._ - // - // TODO: completion of example - // - startCamelService + + val ua = actorOf[RemoteActor2].start + val ta = TypedActor.newInstance( + classOf[RemoteTypedConsumer2], + classOf[RemoteTypedConsumer2Impl], 2000) + RemoteNode.start("localhost", 7777) - RemoteNode.register("remote2", actorOf[RemoteActor2].start) + RemoteNode.register("remote2", ua) + RemoteNode.registerTypedActor("remote3", ta) } diff --git a/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala b/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala index e3fd76a344..6f70d8071a 100644 --- a/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala +++ b/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala @@ -185,7 +185,7 @@ trait ChatServer extends Actor { protected def sessionManagement: Receive protected def shutdownSessions(): Unit - override def shutdown = { + override def postStop = { log.info("Chat server is shutting down...") shutdownSessions self.unlink(storage) @@ -205,7 +205,7 @@ class ChatService extends SessionManagement with ChatManagement with RedisChatStorageFactory { - override def init = { + override def preStart = { RemoteNode.start("localhost", 9999) RemoteNode.register("chat:service", self) } diff --git a/akka-samples/akka-sample-lift/src/main/scala/akka/SimpleService.scala b/akka-samples/akka-sample-lift/src/main/scala/akka/SimpleService.scala deleted file mode 100644 index d5358a7d89..0000000000 --- a/akka-samples/akka-sample-lift/src/main/scala/akka/SimpleService.scala +++ /dev/null @@ -1,94 +0,0 @@ -package sample.lift - -import se.scalablesolutions.akka.actor._ -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.config.ScalaConfig._ -import se.scalablesolutions.akka.stm.TransactionalMap -import se.scalablesolutions.akka.persistence.cassandra.CassandraStorage -import scala.xml.Node -import java.lang.Integer -import javax.ws.rs.{GET, Path, Produces} -import java.nio.ByteBuffer -import net.liftweb.http._ -import net.liftweb.http.rest._ - -class SimpleServiceActor extends Transactor { - private val KEY = "COUNTER" - private var hasStartedTicking = false - private lazy val storage = TransactionalMap[String, Integer]() - - def receive = { - case "Tick" => if (hasStartedTicking) { - val counter = storage.get(KEY).get.asInstanceOf[Integer].intValue - storage.put(KEY, new Integer(counter + 1)) - self.reply(

    Tick: {counter + 1}

    ) - } else { - storage.put(KEY, new Integer(0)) - hasStartedTicking = true - self.reply(

    Tick: 0

    ) - } - } -} - -class PersistentServiceActor extends Transactor { - - private val KEY = "COUNTER" - private var hasStartedTicking = false - private lazy val storage = CassandraStorage.newMap - - def receive = { - case "Tick" => if (hasStartedTicking) { - val bytes = storage.get(KEY.getBytes).get - val counter = ByteBuffer.wrap(bytes).getInt - storage.put(KEY.getBytes, ByteBuffer.allocate(4).putInt(counter + 1).array) - self.reply(Tick:{counter + 1}) - } else { - storage.put(KEY.getBytes, ByteBuffer.allocate(4).putInt(0).array) - hasStartedTicking = true - self.reply(Tick: 0) - } - } -} - - -/** - * Try service out by invoking (multiple times): - *
    - * curl http://localhost:8080/liftcount
    - * 
    - * Or browse to the URL from a web browser. - */ - -object SimpleRestService extends RestHelper { - serve { - case Get("liftcount" :: _, req) => - //Fetch the first actor of type SimpleServiceActor - //Send it the "Tick" message and expect a Node back - val result = for( a <- ActorRegistry.actorFor[SimpleServiceActor]; - r <- (a !! "Tick").as[Node] ) yield r - - //Return either the resulting NodeSeq or a default one - (result getOrElse

    Error in counter

    ).asInstanceOf[Node] - } -} - - -/** - * Try service out by invoking (multiple times): - *
    - * curl http://localhost:8080/persistentliftcount
    - * 
    - * Or browse to the URL from a web browser. - */ - object PersistentRestService extends RestHelper { - serve { - case Get("persistentliftcount" :: _, req) => - //Fetch the first actor of type SimpleServiceActor - //Send it the "Tick" message and expect a Node back - val result = for( a <- ActorRegistry.actorFor[PersistentServiceActor]; - r <- (a !! "Tick").as[Node] ) yield r - - //Return either the resulting NodeSeq or a default one - (result getOrElse

    Error in counter

    ).asInstanceOf[Node] - } - } diff --git a/akka-samples/akka-sample-lift/src/main/scala/bootstrap/liftweb/Boot.scala b/akka-samples/akka-sample-lift/src/main/scala/bootstrap/liftweb/Boot.scala deleted file mode 100644 index 2e56a5857a..0000000000 --- a/akka-samples/akka-sample-lift/src/main/scala/bootstrap/liftweb/Boot.scala +++ /dev/null @@ -1,60 +0,0 @@ -package bootstrap.liftweb - -import _root_.net.liftweb.util._ -import _root_.net.liftweb.http._ -import _root_.net.liftweb.sitemap._ -import _root_.net.liftweb.sitemap.Loc._ -import _root_.net.liftweb.http.auth._ -import _root_.net.liftweb.common._ -import Helpers._ - -import se.scalablesolutions.akka.actor.{SupervisorFactory, Actor} -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.config.ScalaConfig._ -import se.scalablesolutions.akka.util.Logging - -import sample.lift._ - -/** - * A class that's instantiated early and run. It allows the application - * to modify lift's environment - */ -class Boot extends Logging { - def boot { - // where to search snippet - LiftRules.addToPackages("sample.lift") - - LiftRules.httpAuthProtectedResource.prepend { - case (Req("liftcount" :: Nil, _, _)) => Full(AuthRole("admin")) - } - - LiftRules.authentication = HttpBasicAuthentication("lift") { - case ("someuser", "1234", req) => { - log.info("You are now authenticated !") - userRoles(AuthRole("admin")) - true - } - } - LiftRules.statelessDispatchTable.append(SimpleRestService) - LiftRules.statelessDispatchTable.append(PersistentRestService) - - LiftRules.passNotFoundToChain = true - - val factory = SupervisorFactory( - SupervisorConfig( - RestartStrategy(OneForOne, 3, 100, List(classOf[Exception])), - Supervise( - actorOf[SimpleServiceActor], - LifeCycle(Permanent)) :: - Supervise( - actorOf[PersistentServiceActor], - LifeCycle(Permanent)) :: - Nil)) - factory.newInstance.start - - // Build SiteMap - // val entries = Menu(Loc("Home", List("index"), "Home")) :: Nil - // LiftRules.setSiteMap(SiteMap(entries:_*)) - } -} - diff --git a/akka-samples/akka-sample-lift/src/main/scala/comet/.keep b/akka-samples/akka-sample-lift/src/main/scala/comet/.keep deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/akka-samples/akka-sample-lift/src/main/scala/model/.keep b/akka-samples/akka-sample-lift/src/main/scala/model/.keep deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/akka-samples/akka-sample-lift/src/main/scala/snippet/.keep b/akka-samples/akka-sample-lift/src/main/scala/snippet/.keep deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/akka-samples/akka-sample-lift/src/main/scala/snippet/HelloWorld.scala b/akka-samples/akka-sample-lift/src/main/scala/snippet/HelloWorld.scala deleted file mode 100644 index aed272c0b2..0000000000 --- a/akka-samples/akka-sample-lift/src/main/scala/snippet/HelloWorld.scala +++ /dev/null @@ -1,6 +0,0 @@ -package sample.lift.snippet - -class HelloWorld { - def howdy = Welcome to lift-akka at {new _root_.java.util.Date} -} - diff --git a/akka-samples/akka-sample-lift/src/main/scala/view/.keep b/akka-samples/akka-sample-lift/src/main/scala/view/.keep deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/akka-samples/akka-sample-lift/src/main/webapp/WEB-INF/web.xml b/akka-samples/akka-sample-lift/src/main/webapp/WEB-INF/web.xml deleted file mode 100644 index 3a1b672cec..0000000000 --- a/akka-samples/akka-sample-lift/src/main/webapp/WEB-INF/web.xml +++ /dev/null @@ -1,22 +0,0 @@ - - - - - LiftFilter - Lift Filter - The Filter that intercepts lift calls - net.liftweb.http.LiftFilter - - - LiftFilter - /* - - - AkkaServlet - se.scalablesolutions.akka.comet.AkkaServlet - - - AkkaServlet - /* - - diff --git a/akka-samples/akka-sample-lift/src/main/webapp/index.html b/akka-samples/akka-sample-lift/src/main/webapp/index.html deleted file mode 100644 index aa25a1d91d..0000000000 --- a/akka-samples/akka-sample-lift/src/main/webapp/index.html +++ /dev/null @@ -1,15 +0,0 @@ - -

    Welcome to the Akka + Lift Sample

    -

    This page is served by Lift, and Lift alone. In order to demonstrate how AkkaServlet and
    - Lift can work in harmony we have supplied a sample JAX-RS service that is secured using
    - Lift's HTTP Basic Authentication.

    - -

    To access the Akka service, visit this url and enter the - following access credentials:

    - -

    user: someuser
    - password: 1234

    - -

    -
    - diff --git a/akka-samples/akka-sample-lift/src/main/webapp/templates-hidden/default.html b/akka-samples/akka-sample-lift/src/main/webapp/templates-hidden/default.html deleted file mode 100644 index 4a18c18f8a..0000000000 --- a/akka-samples/akka-sample-lift/src/main/webapp/templates-hidden/default.html +++ /dev/null @@ -1,17 +0,0 @@ - - - - - - - Akka with Lift Example - - - -
    - - - -
    - - diff --git a/akka-samples/akka-sample-lift/src/test/scala/LiftConsole.scala b/akka-samples/akka-sample-lift/src/test/scala/LiftConsole.scala deleted file mode 100644 index 43296bc1f4..0000000000 --- a/akka-samples/akka-sample-lift/src/test/scala/LiftConsole.scala +++ /dev/null @@ -1,16 +0,0 @@ -/*import _root_.bootstrap.liftweb.Boot -import _root_.scala.tools.nsc.MainGenericRunner - -object LiftConsole { - def main(args : Array[String]) { - // Instantiate your project's Boot file - val b = new Boot() - // Boot your project - b.boot - // Now run the MainGenericRunner to get your repl - MainGenericRunner.main(args) - // After the repl exits, then exit the scala script - exit(0) - } -} -*/ diff --git a/akka-samples/akka-sample-lift/src/test/scala/RunWebApp.scala b/akka-samples/akka-sample-lift/src/test/scala/RunWebApp.scala deleted file mode 100644 index fd8ea053c3..0000000000 --- a/akka-samples/akka-sample-lift/src/test/scala/RunWebApp.scala +++ /dev/null @@ -1,27 +0,0 @@ -import org.eclipse.jetty.webapp.WebAppContext -import org.eclipse.jetty.server.Server - -object RunWebApp extends Application { - val server = new Server(8080) - val context = new WebAppContext() - context.setServer(server) - context.setContextPath("/") - context.setWar("src/main/webapp") - - server.setHandler(context) - - try { - println(">>> STARTING EMBEDDED JETTY SERVER, PRESS ANY KEY TO STOP") - server.start() - while (System.in.available() == 0) { - Thread.sleep(5000) - } - server.stop() - server.join() - } catch { - case exc : Exception => { - exc.printStackTrace() - System.exit(100) - } - } -} diff --git a/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-0.10.xsd b/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-0.10.xsd index 2a42ec0900..e66090fe16 100644 --- a/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-0.10.xsd +++ b/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-0.10.xsd @@ -39,8 +39,6 @@ - - diff --git a/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-1.0-SNAPSHOT.xsd b/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-1.0-SNAPSHOT.xsd new file mode 100644 index 0000000000..84a382a78e --- /dev/null +++ b/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-1.0-SNAPSHOT.xsd @@ -0,0 +1,355 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Name of the remote host. + + + + + + + Port of the remote host. + + + + + + + Management type for remote actors: client managed or server managed. + + + + + + + Custom service name for server managed actor. + + + + + + + + + + + + + + + + + + Name of the interface implemented by implementation class. + + + + + + + Name of the implementation class. + + + + + + + The default timeout for '!!' invocations. + + + + + + + Set this to true if messages should have REQUIRES_NEW semantics. + + + + + + + Defines the lifecycle, can be either 'permanent' or 'temporary'. + + + + + + + Supported scopes are 'singleton' and 'prototype'. + + + + + + + + + + + + + + + + + + Name of the implementation class. + + + + + + + The default timeout for '!!' invocations. + + + + + + + Set this to true if messages should have REQUIRES_NEW semantics. + + + + + + + Defines the lifecycle, can be either 'permanent' or 'temporary'. + + + + + + + Supported scopes are 'singleton' and 'prototype'. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Name of the remote host. + + + + + + + Port of the remote host. + + + + + + + Custom service name or class name for the server managed actor. + + + + + + + Name of the interface the typed actor implements. + + + + + + + + + + + + + + + Failover scheme, can be one of 'AllForOne' or 'OneForOne'. + + + + + + + Maximal number of restarts. + + + + + + + Time range for maximal number of restart. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/akka-spring/src/main/scala/ActorBeanDefinitionParser.scala b/akka-spring/src/main/scala/ActorBeanDefinitionParser.scala new file mode 100644 index 0000000000..55aa82b8e4 --- /dev/null +++ b/akka-spring/src/main/scala/ActorBeanDefinitionParser.scala @@ -0,0 +1,72 @@ +/** + * Copyright (C) 2009-2010 Scalable Solutions AB + */ +package se.scalablesolutions.akka.spring + +import org.springframework.beans.factory.support.BeanDefinitionBuilder +import org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser +import org.springframework.beans.factory.xml.ParserContext +import AkkaSpringConfigurationTags._ +import org.w3c.dom.Element + + +/** + * Parser for custom namespace configuration. + * @author michaelkober + */ +class TypedActorBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorParser { + /* + * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder) + */ + override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) { + val typedActorConf = parseActor(element) + typedActorConf.typed = TYPED_ACTOR_TAG + typedActorConf.setAsProperties(builder) + } + + /* + * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element) + */ + override def getBeanClass(element: Element): Class[_] = classOf[ActorFactoryBean] +} + + +/** + * Parser for custom namespace configuration. + * @author michaelkober + */ +class UntypedActorBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorParser { + /* + * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder) + */ + override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) { + val untypedActorConf = parseActor(element) + untypedActorConf.typed = UNTYPED_ACTOR_TAG + untypedActorConf.setAsProperties(builder) + } + + /* + * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element) + */ + override def getBeanClass(element: Element): Class[_] = classOf[ActorFactoryBean] +} + + +/** + * Parser for custom namespace configuration. + * @author michaelkober + */ +class ActorForBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorForParser { + /* + * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder) + */ + override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) { + val actorForConf = parseActorFor(element) + actorForConf.setAsProperties(builder) + } + + /* + * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element) + */ + override def getBeanClass(element: Element): Class[_] = classOf[ActorForFactoryBean] +} diff --git a/akka-spring/src/main/scala/ActorFactoryBean.scala b/akka-spring/src/main/scala/ActorFactoryBean.scala index 11d5274a70..caa344825a 100644 --- a/akka-spring/src/main/scala/ActorFactoryBean.scala +++ b/akka-spring/src/main/scala/ActorFactoryBean.scala @@ -4,22 +4,19 @@ package se.scalablesolutions.akka.spring -import java.beans.PropertyDescriptor -import java.lang.reflect.Method -import javax.annotation.PreDestroy -import javax.annotation.PostConstruct - import org.springframework.beans.{BeanUtils,BeansException,BeanWrapper,BeanWrapperImpl} -import org.springframework.beans.factory.BeanFactory +import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer} +//import org.springframework.beans.factory.BeanFactory import org.springframework.beans.factory.config.AbstractFactoryBean import org.springframework.context.{ApplicationContext,ApplicationContextAware} -import org.springframework.util.ReflectionUtils +//import org.springframework.util.ReflectionUtils import org.springframework.util.StringUtils import se.scalablesolutions.akka.actor.{ActorRef, AspectInitRegistry, TypedActorConfiguration, TypedActor,Actor} import se.scalablesolutions.akka.dispatch.MessageDispatcher import se.scalablesolutions.akka.util.{Logging, Duration} import scala.reflect.BeanProperty +import java.net.InetSocketAddress /** * Exception to use when something goes wrong during bean creation. @@ -49,13 +46,15 @@ class ActorFactoryBean extends AbstractFactoryBean[AnyRef] with Logging with App @BeanProperty var transactional: Boolean = false @BeanProperty var host: String = "" @BeanProperty var port: Int = _ + @BeanProperty var serverManaged: Boolean = false + @BeanProperty var serviceName: String = "" @BeanProperty var lifecycle: String = "" @BeanProperty var dispatcher: DispatcherProperties = _ @BeanProperty var scope: String = VAL_SCOPE_SINGLETON @BeanProperty var property: PropertyEntries = _ @BeanProperty var applicationContext: ApplicationContext = _ - // Holds info about if deps has been set or not. Depends on + // Holds info about if deps have been set or not. Depends on // if interface is specified or not. We must set deps on // target instance if interface is specified var hasSetDependecies = false @@ -94,7 +93,16 @@ class ActorFactoryBean extends AbstractFactoryBean[AnyRef] with Logging with App if (implementation == null || implementation == "") throw new AkkaBeansException( "The 'implementation' part of the 'akka:typed-actor' element in the Spring config file can't be null or empty string") - TypedActor.newInstance(interface.toClass, implementation.toClass, createConfig) + val typedActor: AnyRef = TypedActor.newInstance(interface.toClass, implementation.toClass, createConfig) + if (isRemote && serverManaged) { + val server = RemoteServer.getOrCreateServer(new InetSocketAddress(host, port)) + if (serviceName.isEmpty) { + server.registerTypedActor(interface, typedActor) + } else { + server.registerTypedActor(serviceName, typedActor) + } + } + typedActor } /** @@ -111,7 +119,16 @@ class ActorFactoryBean extends AbstractFactoryBean[AnyRef] with Logging with App actorRef.makeTransactionRequired } if (isRemote) { - actorRef.makeRemote(host, port) + if (serverManaged) { + val server = RemoteServer.getOrCreateServer(new InetSocketAddress(host, port)) + if (serviceName.isEmpty) { + server.register(actorRef) + } else { + server.register(serviceName, actorRef) + } + } else { + actorRef.makeRemote(host, port) + } } if (hasDispatcher) { if (dispatcher.dispatcherType != THREAD_BASED){ @@ -159,7 +176,7 @@ class ActorFactoryBean extends AbstractFactoryBean[AnyRef] with Logging with App private[akka] def createConfig: TypedActorConfiguration = { val config = new TypedActorConfiguration().timeout(Duration(timeout, "millis")) if (transactional) config.makeTransactionRequired - if (isRemote) config.makeRemote(host, port) + if (isRemote && !serverManaged) config.makeRemote(host, port) if (hasDispatcher) { if (dispatcher.dispatcherType != THREAD_BASED) { config.dispatcher(dispatcherInstance()) @@ -191,3 +208,39 @@ class ActorFactoryBean extends AbstractFactoryBean[AnyRef] with Logging with App } } } + +/** + * Factory bean for remote client actor-for. + * + * @author michaelkober + */ +class ActorForFactoryBean extends AbstractFactoryBean[AnyRef] with Logging with ApplicationContextAware { + import StringReflect._ + import AkkaSpringConfigurationTags._ + + @BeanProperty var interface: String = "" + @BeanProperty var host: String = "" + @BeanProperty var port: Int = _ + @BeanProperty var serviceName: String = "" + //@BeanProperty var scope: String = VAL_SCOPE_SINGLETON + @BeanProperty var applicationContext: ApplicationContext = _ + + override def isSingleton = false + + /* + * @see org.springframework.beans.factory.FactoryBean#getObjectType() + */ + def getObjectType: Class[AnyRef] = classOf[AnyRef] + + /* + * @see org.springframework.beans.factory.config.AbstractFactoryBean#createInstance() + */ + def createInstance: AnyRef = { + if (interface.isEmpty) { + RemoteClient.actorFor(serviceName, host, port) + } else { + RemoteClient.typedActorFor(interface.toClass, serviceName, host, port) + } + } +} + diff --git a/akka-spring/src/main/scala/ActorParser.scala b/akka-spring/src/main/scala/ActorParser.scala index 69073bd52f..9858e9ad7e 100644 --- a/akka-spring/src/main/scala/ActorParser.scala +++ b/akka-spring/src/main/scala/ActorParser.scala @@ -6,6 +6,7 @@ package se.scalablesolutions.akka.spring import org.springframework.util.xml.DomUtils import org.w3c.dom.Element import scala.collection.JavaConversions._ +import se.scalablesolutions.akka.util.Logging import se.scalablesolutions.akka.actor.IllegalActorStateException @@ -27,11 +28,17 @@ trait ActorParser extends BeanParser with DispatcherParser { val objectProperties = new ActorProperties() val remoteElement = DomUtils.getChildElementByTagName(element, REMOTE_TAG); val dispatcherElement = DomUtils.getChildElementByTagName(element, DISPATCHER_TAG) - val propertyEntries = DomUtils.getChildElementsByTagName(element,PROPERTYENTRY_TAG) + val propertyEntries = DomUtils.getChildElementsByTagName(element, PROPERTYENTRY_TAG) if (remoteElement != null) { objectProperties.host = mandatory(remoteElement, HOST) objectProperties.port = mandatory(remoteElement, PORT).toInt + objectProperties.serverManaged = (remoteElement.getAttribute(MANAGED_BY) != null) && (remoteElement.getAttribute(MANAGED_BY).equals(SERVER_MANAGED)) + val serviceName = remoteElement.getAttribute(SERVICE_NAME) + if ((serviceName != null) && (!serviceName.isEmpty)) { + objectProperties.serviceName = serviceName + objectProperties.serverManaged = true + } } if (dispatcherElement != null) { @@ -43,7 +50,7 @@ trait ActorParser extends BeanParser with DispatcherParser { val entry = new PropertyEntry entry.name = element.getAttribute("name"); entry.value = element.getAttribute("value") - entry.ref = element.getAttribute("ref") + entry.ref = element.getAttribute("ref") objectProperties.propertyEntries.add(entry) } @@ -59,15 +66,13 @@ trait ActorParser extends BeanParser with DispatcherParser { objectProperties.target = mandatory(element, IMPLEMENTATION) objectProperties.transactional = if (element.getAttribute(TRANSACTIONAL).isEmpty) false else element.getAttribute(TRANSACTIONAL).toBoolean - if (!element.getAttribute(INTERFACE).isEmpty) { + if (element.hasAttribute(INTERFACE)) { objectProperties.interface = element.getAttribute(INTERFACE) } - - if (!element.getAttribute(LIFECYCLE).isEmpty) { + if (element.hasAttribute(LIFECYCLE)) { objectProperties.lifecycle = element.getAttribute(LIFECYCLE) } - - if (!element.getAttribute(SCOPE).isEmpty) { + if (element.hasAttribute(SCOPE)) { objectProperties.scope = element.getAttribute(SCOPE) } @@ -75,3 +80,158 @@ trait ActorParser extends BeanParser with DispatcherParser { } } + +/** + * Parser trait for custom namespace configuration for RemoteClient actor-for. + * @author michaelkober + */ +trait ActorForParser extends BeanParser { + import AkkaSpringConfigurationTags._ + + /** + * Parses the given element and returns a ActorForProperties. + * @param element dom element to parse + * @return configuration for the typed actor + */ + def parseActorFor(element: Element): ActorForProperties = { + val objectProperties = new ActorForProperties() + + objectProperties.host = mandatory(element, HOST) + objectProperties.port = mandatory(element, PORT).toInt + objectProperties.serviceName = mandatory(element, SERVICE_NAME) + if (element.hasAttribute(INTERFACE)) { + objectProperties.interface = element.getAttribute(INTERFACE) + } + objectProperties + } + +} + +/** + * Base trait with utility methods for bean parsing. + */ +trait BeanParser extends Logging { + + /** + * Get a mandatory element attribute. + * @param element the element with the mandatory attribute + * @param attribute name of the mandatory attribute + */ + def mandatory(element: Element, attribute: String): String = { + if ((element.getAttribute(attribute) == null) || (element.getAttribute(attribute).isEmpty)) { + throw new IllegalArgumentException("Mandatory attribute missing: " + attribute) + } else { + element.getAttribute(attribute) + } + } + + /** + * Get a mandatory child element. + * @param element the parent element + * @param childName name of the mandatory child element + */ + def mandatoryElement(element: Element, childName: String): Element = { + val childElement = DomUtils.getChildElementByTagName(element, childName); + if (childElement == null) { + throw new IllegalArgumentException("Mandatory element missing: ''") + } else { + childElement + } + } + +} + + +/** + * Parser trait for custom namespace for Akka dispatcher configuration. + * @author michaelkober + */ +trait DispatcherParser extends BeanParser { + import AkkaSpringConfigurationTags._ + + /** + * Parses the given element and returns a DispatcherProperties. + * @param element dom element to parse + * @return configuration for the dispatcher + */ + def parseDispatcher(element: Element): DispatcherProperties = { + val properties = new DispatcherProperties() + var dispatcherElement = element + if (hasRef(element)) { + val ref = element.getAttribute(REF) + dispatcherElement = element.getOwnerDocument.getElementById(ref) + if (dispatcherElement == null) { + throw new IllegalArgumentException("Referenced dispatcher not found: '" + ref + "'") + } + } + + properties.dispatcherType = mandatory(dispatcherElement, TYPE) + if (properties.dispatcherType == THREAD_BASED) { + val allowedParentNodes = "akka:typed-actor" :: "akka:untyped-actor" :: "typed-actor" :: "untyped-actor" :: Nil + if (!allowedParentNodes.contains(dispatcherElement.getParentNode.getNodeName)) { + throw new IllegalArgumentException("Thread based dispatcher must be nested in 'typed-actor' or 'untyped-actor' element!") + } + } + + if (properties.dispatcherType == HAWT) { // no name for HawtDispatcher + properties.name = dispatcherElement.getAttribute(NAME) + if (dispatcherElement.hasAttribute(AGGREGATE)) { + properties.aggregate = dispatcherElement.getAttribute(AGGREGATE).toBoolean + } + } else { + properties.name = mandatory(dispatcherElement, NAME) + } + + val threadPoolElement = DomUtils.getChildElementByTagName(dispatcherElement, THREAD_POOL_TAG); + if (threadPoolElement != null) { + if (properties.dispatcherType == THREAD_BASED) { + throw new IllegalArgumentException("Element 'thread-pool' not allowed for this dispatcher type.") + } + val threadPoolProperties = parseThreadPool(threadPoolElement) + properties.threadPool = threadPoolProperties + } + properties + } + + /** + * Parses the given element and returns a ThreadPoolProperties. + * @param element dom element to parse + * @return configuration for the thread pool + */ + def parseThreadPool(element: Element): ThreadPoolProperties = { + val properties = new ThreadPoolProperties() + properties.queue = element.getAttribute(QUEUE) + if (element.hasAttribute(CAPACITY)) { + properties.capacity = element.getAttribute(CAPACITY).toInt + } + if (element.hasAttribute(BOUND)) { + properties.bound = element.getAttribute(BOUND).toInt + } + if (element.hasAttribute(FAIRNESS)) { + properties.fairness = element.getAttribute(FAIRNESS).toBoolean + } + if (element.hasAttribute(CORE_POOL_SIZE)) { + properties.corePoolSize = element.getAttribute(CORE_POOL_SIZE).toInt + } + if (element.hasAttribute(MAX_POOL_SIZE)) { + properties.maxPoolSize = element.getAttribute(MAX_POOL_SIZE).toInt + } + if (element.hasAttribute(KEEP_ALIVE)) { + properties.keepAlive = element.getAttribute(KEEP_ALIVE).toLong + } + if (element.hasAttribute(REJECTION_POLICY)) { + properties.rejectionPolicy = element.getAttribute(REJECTION_POLICY) + } + if (element.hasAttribute(MAILBOX_CAPACITY)) { + properties.mailboxCapacity = element.getAttribute(MAILBOX_CAPACITY).toInt + } + properties + } + + def hasRef(element: Element): Boolean = { + val ref = element.getAttribute(REF) + (ref != null) && !ref.isEmpty + } + +} + diff --git a/akka-spring/src/main/scala/ActorProperties.scala b/akka-spring/src/main/scala/ActorProperties.scala index 15c7e61fe0..0f86942935 100644 --- a/akka-spring/src/main/scala/ActorProperties.scala +++ b/akka-spring/src/main/scala/ActorProperties.scala @@ -8,7 +8,7 @@ import org.springframework.beans.factory.support.BeanDefinitionBuilder import AkkaSpringConfigurationTags._ /** - * Data container for typed actor configuration data. + * Data container for actor configuration data. * @author michaelkober * @author Martin Krasser */ @@ -20,6 +20,8 @@ class ActorProperties { var transactional: Boolean = false var host: String = "" var port: Int = _ + var serverManaged: Boolean = false + var serviceName: String = "" var lifecycle: String = "" var scope:String = VAL_SCOPE_SINGLETON var dispatcher: DispatcherProperties = _ @@ -34,6 +36,8 @@ class ActorProperties { builder.addPropertyValue("typed", typed) builder.addPropertyValue(HOST, host) builder.addPropertyValue(PORT, port) + builder.addPropertyValue("serverManaged", serverManaged) + builder.addPropertyValue("serviceName", serviceName) builder.addPropertyValue(TIMEOUT, timeout) builder.addPropertyValue(IMPLEMENTATION, target) builder.addPropertyValue(INTERFACE, interface) @@ -45,3 +49,26 @@ class ActorProperties { } } + +/** + * Data container for actor configuration data. + * @author michaelkober + */ +class ActorForProperties { + var interface: String = "" + var host: String = "" + var port: Int = _ + var serviceName: String = "" + + /** + * Sets the properties to the given builder. + * @param builder bean definition builder + */ + def setAsProperties(builder: BeanDefinitionBuilder) { + builder.addPropertyValue(HOST, host) + builder.addPropertyValue(PORT, port) + builder.addPropertyValue("serviceName", serviceName) + builder.addPropertyValue(INTERFACE, interface) + } + +} diff --git a/akka-spring/src/main/scala/AkkaNamespaceHandler.scala b/akka-spring/src/main/scala/AkkaNamespaceHandler.scala index a478b7b262..b1c58baa20 100644 --- a/akka-spring/src/main/scala/AkkaNamespaceHandler.scala +++ b/akka-spring/src/main/scala/AkkaNamespaceHandler.scala @@ -12,10 +12,11 @@ import AkkaSpringConfigurationTags._ */ class AkkaNamespaceHandler extends NamespaceHandlerSupport { def init = { - registerBeanDefinitionParser(TYPED_ACTOR_TAG, new TypedActorBeanDefinitionParser()); - registerBeanDefinitionParser(UNTYPED_ACTOR_TAG, new UntypedActorBeanDefinitionParser()); - registerBeanDefinitionParser(SUPERVISION_TAG, new SupervisionBeanDefinitionParser()); - registerBeanDefinitionParser(DISPATCHER_TAG, new DispatcherBeanDefinitionParser()); - registerBeanDefinitionParser(CAMEL_SERVICE_TAG, new CamelServiceBeanDefinitionParser); + registerBeanDefinitionParser(TYPED_ACTOR_TAG, new TypedActorBeanDefinitionParser()) + registerBeanDefinitionParser(UNTYPED_ACTOR_TAG, new UntypedActorBeanDefinitionParser()) + registerBeanDefinitionParser(SUPERVISION_TAG, new SupervisionBeanDefinitionParser()) + registerBeanDefinitionParser(DISPATCHER_TAG, new DispatcherBeanDefinitionParser()) + registerBeanDefinitionParser(CAMEL_SERVICE_TAG, new CamelServiceBeanDefinitionParser) + registerBeanDefinitionParser(ACTOR_FOR_TAG, new ActorForBeanDefinitionParser()); } } diff --git a/akka-spring/src/main/scala/AkkaSpringConfigurationTags.scala b/akka-spring/src/main/scala/AkkaSpringConfigurationTags.scala index 518727bd4c..0e4de3576f 100644 --- a/akka-spring/src/main/scala/AkkaSpringConfigurationTags.scala +++ b/akka-spring/src/main/scala/AkkaSpringConfigurationTags.scala @@ -19,6 +19,7 @@ object AkkaSpringConfigurationTags { val DISPATCHER_TAG = "dispatcher" val PROPERTYENTRY_TAG = "property" val CAMEL_SERVICE_TAG = "camel-service" + val ACTOR_FOR_TAG = "actor-for" // actor sub tags val REMOTE_TAG = "remote" @@ -45,6 +46,8 @@ object AkkaSpringConfigurationTags { val TRANSACTIONAL = "transactional" val HOST = "host" val PORT = "port" + val MANAGED_BY = "managed-by" + val SERVICE_NAME = "service-name" val LIFECYCLE = "lifecycle" val SCOPE = "scope" @@ -68,6 +71,7 @@ object AkkaSpringConfigurationTags { val KEEP_ALIVE = "keep-alive" val BOUND ="bound" val REJECTION_POLICY ="rejection-policy" + val MAILBOX_CAPACITY ="mailbox-capacity" // --- VALUES // @@ -97,9 +101,11 @@ object AkkaSpringConfigurationTags { // dispatcher types val EXECUTOR_BASED_EVENT_DRIVEN = "executor-based-event-driven" val EXECUTOR_BASED_EVENT_DRIVEN_WORK_STEALING = "executor-based-event-driven-work-stealing" - val REACTOR_BASED_THREAD_POOL_EVENT_DRIVEN = "reactor-based-thread-pool-event-driven" - val REACTOR_BASED_SINGLE_THREAD_EVENT_DRIVEN = "reactor-based-single-thread-event-driven" val THREAD_BASED = "thread-based" val HAWT = "hawt" + // managed by types + val SERVER_MANAGED = "server" + val CLIENT_MANAGED = "client" + } diff --git a/akka-spring/src/main/scala/BeanParser.scala b/akka-spring/src/main/scala/BeanParser.scala deleted file mode 100644 index 1bbba9f09f..0000000000 --- a/akka-spring/src/main/scala/BeanParser.scala +++ /dev/null @@ -1,42 +0,0 @@ -/** - * Copyright (C) 2009-2010 Scalable Solutions AB - */ -package se.scalablesolutions.akka.spring - -import se.scalablesolutions.akka.util.Logging -import org.w3c.dom.Element -import org.springframework.util.xml.DomUtils - -/** - * Base trait with utility methods for bean parsing. - */ -trait BeanParser extends Logging { - - /** - * Get a mandatory element attribute. - * @param element the element with the mandatory attribute - * @param attribute name of the mandatory attribute - */ - def mandatory(element: Element, attribute: String): String = { - if ((element.getAttribute(attribute) == null) || (element.getAttribute(attribute).isEmpty)) { - throw new IllegalArgumentException("Mandatory attribute missing: " + attribute) - } else { - element.getAttribute(attribute) - } - } - - /** - * Get a mandatory child element. - * @param element the parent element - * @param childName name of the mandatory child element - */ - def mandatoryElement(element: Element, childName: String): Element = { - val childElement = DomUtils.getChildElementByTagName(element, childName); - if (childElement == null) { - throw new IllegalArgumentException("Mandatory element missing: ''") - } else { - childElement - } - } - -} diff --git a/akka-spring/src/main/scala/DispatcherFactoryBean.scala b/akka-spring/src/main/scala/DispatcherFactoryBean.scala index 06c9994c7f..4d13fa6814 100644 --- a/akka-spring/src/main/scala/DispatcherFactoryBean.scala +++ b/akka-spring/src/main/scala/DispatcherFactoryBean.scala @@ -26,8 +26,6 @@ object DispatcherFactoryBean { var dispatcher = properties.dispatcherType match { case EXECUTOR_BASED_EVENT_DRIVEN => Dispatchers.newExecutorBasedEventDrivenDispatcher(properties.name) case EXECUTOR_BASED_EVENT_DRIVEN_WORK_STEALING => Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher(properties.name) - case REACTOR_BASED_THREAD_POOL_EVENT_DRIVEN => Dispatchers.newReactorBasedThreadPoolEventDrivenDispatcher(properties.name) - case REACTOR_BASED_SINGLE_THREAD_EVENT_DRIVEN => Dispatchers.newReactorBasedSingleThreadEventDrivenDispatcher(properties.name) case THREAD_BASED => if (!actorRef.isDefined) { throw new IllegalArgumentException("Need an ActorRef to create a thread based dispatcher.") } else { @@ -58,6 +56,9 @@ object DispatcherFactoryBean { if (properties.threadPool.keepAlive > -1) { threadPoolBuilder.setKeepAliveTimeInMillis(properties.threadPool.keepAlive) } + if (properties.threadPool.mailboxCapacity > -1) { + threadPoolBuilder.setMailboxCapacity(properties.threadPool.mailboxCapacity) + } if ((properties.threadPool.rejectionPolicy != null) && (!properties.threadPool.rejectionPolicy.isEmpty)) { val policy: RejectedExecutionHandler = properties.threadPool.rejectionPolicy match { case "abort-policy" => new AbortPolicy() diff --git a/akka-spring/src/main/scala/DispatcherParser.scala b/akka-spring/src/main/scala/DispatcherParser.scala deleted file mode 100644 index c4257230f7..0000000000 --- a/akka-spring/src/main/scala/DispatcherParser.scala +++ /dev/null @@ -1,98 +0,0 @@ -/** - * Copyright (C) 2009-2010 Scalable Solutions AB - */ -package se.scalablesolutions.akka.spring - -import org.w3c.dom.Element -import org.springframework.util.xml.DomUtils - -/** - * Parser trait for custom namespace for Akka dispatcher configuration. - * @author michaelkober - */ -trait DispatcherParser extends BeanParser { - import AkkaSpringConfigurationTags._ - - /** - * Parses the given element and returns a DispatcherProperties. - * @param element dom element to parse - * @return configuration for the dispatcher - */ - def parseDispatcher(element: Element): DispatcherProperties = { - val properties = new DispatcherProperties() - var dispatcherElement = element - if (hasRef(element)) { - val ref = element.getAttribute(REF) - dispatcherElement = element.getOwnerDocument.getElementById(ref) - if (dispatcherElement == null) { - throw new IllegalArgumentException("Referenced dispatcher not found: '" + ref + "'") - } - } - - properties.dispatcherType = mandatory(dispatcherElement, TYPE) - if (properties.dispatcherType == THREAD_BASED) { - val allowedParentNodes = "akka:typed-actor" :: "akka:untyped-actor" :: "typed-actor" :: "untyped-actor" :: Nil - if (!allowedParentNodes.contains(dispatcherElement.getParentNode.getNodeName)) { - throw new IllegalArgumentException("Thread based dispatcher must be nested in 'typed-actor' or 'untyped-actor' element!") - } - } - - if (properties.dispatcherType == HAWT) { // no name for HawtDispatcher - properties.name = dispatcherElement.getAttribute(NAME) - if (dispatcherElement.hasAttribute(AGGREGATE)) { - properties.aggregate = dispatcherElement.getAttribute(AGGREGATE).toBoolean - } - } else { - properties.name = mandatory(dispatcherElement, NAME) - } - - val threadPoolElement = DomUtils.getChildElementByTagName(dispatcherElement, THREAD_POOL_TAG); - if (threadPoolElement != null) { - if (properties.dispatcherType == REACTOR_BASED_SINGLE_THREAD_EVENT_DRIVEN || - properties.dispatcherType == THREAD_BASED) { - throw new IllegalArgumentException("Element 'thread-pool' not allowed for this dispatcher type.") - } - val threadPoolProperties = parseThreadPool(threadPoolElement) - properties.threadPool = threadPoolProperties - } - properties - } - - /** - * Parses the given element and returns a ThreadPoolProperties. - * @param element dom element to parse - * @return configuration for the thread pool - */ - def parseThreadPool(element: Element): ThreadPoolProperties = { - val properties = new ThreadPoolProperties() - properties.queue = element.getAttribute(QUEUE) - if (element.hasAttribute(CAPACITY)) { - properties.capacity = element.getAttribute(CAPACITY).toInt - } - if (element.hasAttribute(BOUND)) { - properties.bound = element.getAttribute(BOUND).toInt - } - if (element.hasAttribute(FAIRNESS)) { - properties.fairness = element.getAttribute(FAIRNESS).toBoolean - } - if (element.hasAttribute(CORE_POOL_SIZE)) { - properties.corePoolSize = element.getAttribute(CORE_POOL_SIZE).toInt - } - if (element.hasAttribute(MAX_POOL_SIZE)) { - properties.maxPoolSize = element.getAttribute(MAX_POOL_SIZE).toInt - } - if (element.hasAttribute(KEEP_ALIVE)) { - properties.keepAlive = element.getAttribute(KEEP_ALIVE).toLong - } - if (element.hasAttribute(REJECTION_POLICY)) { - properties.rejectionPolicy = element.getAttribute(REJECTION_POLICY) - } - properties - } - - def hasRef(element: Element): Boolean = { - val ref = element.getAttribute(REF) - (ref != null) && !ref.isEmpty - } - -} diff --git a/akka-spring/src/main/scala/DispatcherProperties.scala b/akka-spring/src/main/scala/DispatcherProperties.scala index 183b3825bb..89d97670ca 100644 --- a/akka-spring/src/main/scala/DispatcherProperties.scala +++ b/akka-spring/src/main/scala/DispatcherProperties.scala @@ -45,6 +45,7 @@ class ThreadPoolProperties { var maxPoolSize = -1 var keepAlive = -1L var rejectionPolicy = "" + var mailboxCapacity = -1 override def toString : String = { "ThreadPoolProperties[queue=" + queue + @@ -54,6 +55,7 @@ class ThreadPoolProperties { ", corePoolSize=" + corePoolSize + ", maxPoolSize=" + maxPoolSize + ", keepAlive=" + keepAlive + - ", policy=" + rejectionPolicy + "]" + ", policy=" + rejectionPolicy + + ", mailboxCapacity=" + mailboxCapacity + "]" } } diff --git a/akka-spring/src/main/scala/PropertyEntries.scala b/akka-spring/src/main/scala/PropertyEntries.scala index bf1898a805..9a7dc098de 100644 --- a/akka-spring/src/main/scala/PropertyEntries.scala +++ b/akka-spring/src/main/scala/PropertyEntries.scala @@ -18,3 +18,19 @@ class PropertyEntries { entryList.append(entry) } } + +/** + * Represents a property element + * @author Johan Rask + */ +class PropertyEntry { + var name: String = _ + var value: String = null + var ref: String = null + + + override def toString(): String = { + format("name = %s,value = %s, ref = %s", name, value, ref) + } +} + diff --git a/akka-spring/src/main/scala/PropertyEntry.scala b/akka-spring/src/main/scala/PropertyEntry.scala deleted file mode 100644 index 9fe6357fc0..0000000000 --- a/akka-spring/src/main/scala/PropertyEntry.scala +++ /dev/null @@ -1,19 +0,0 @@ -/** - * Copyright (C) 2009-2010 Scalable Solutions AB - */ -package se.scalablesolutions.akka.spring - -/** - * Represents a property element - * @author Johan Rask - */ -class PropertyEntry { - var name: String = _ - var value: String = null - var ref: String = null - - - override def toString(): String = { - format("name = %s,value = %s, ref = %s", name, value, ref) - } -} diff --git a/akka-spring/src/main/scala/TypedActorBeanDefinitionParser.scala b/akka-spring/src/main/scala/TypedActorBeanDefinitionParser.scala deleted file mode 100644 index e8e0cef7d4..0000000000 --- a/akka-spring/src/main/scala/TypedActorBeanDefinitionParser.scala +++ /dev/null @@ -1,31 +0,0 @@ -/** - * Copyright (C) 2009-2010 Scalable Solutions AB - */ -package se.scalablesolutions.akka.spring - -import org.springframework.beans.factory.support.BeanDefinitionBuilder -import org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser -import org.springframework.beans.factory.xml.ParserContext -import AkkaSpringConfigurationTags._ -import org.w3c.dom.Element - - -/** - * Parser for custom namespace configuration. - * @author michaelkober - */ -class TypedActorBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorParser { - /* - * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder) - */ - override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) { - val typedActorConf = parseActor(element) - typedActorConf.typed = TYPED_ACTOR_TAG - typedActorConf.setAsProperties(builder) - } - - /* - * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element) - */ - override def getBeanClass(element: Element): Class[_] = classOf[ActorFactoryBean] -} diff --git a/akka-spring/src/main/scala/UntypedActorBeanDefinitionParser.scala b/akka-spring/src/main/scala/UntypedActorBeanDefinitionParser.scala deleted file mode 100644 index 752e18559f..0000000000 --- a/akka-spring/src/main/scala/UntypedActorBeanDefinitionParser.scala +++ /dev/null @@ -1,31 +0,0 @@ -/** - * Copyright (C) 2009-2010 Scalable Solutions AB - */ -package se.scalablesolutions.akka.spring - -import org.springframework.beans.factory.support.BeanDefinitionBuilder -import org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser -import org.springframework.beans.factory.xml.ParserContext -import AkkaSpringConfigurationTags._ -import org.w3c.dom.Element - - -/** - * Parser for custom namespace configuration. - * @author michaelkober - */ -class UntypedActorBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorParser { - /* - * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder) - */ - override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) { - val untypedActorConf = parseActor(element) - untypedActorConf.typed = UNTYPED_ACTOR_TAG - untypedActorConf.setAsProperties(builder) - } - - /* - * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element) - */ - override def getBeanClass(element: Element): Class[_] = classOf[ActorFactoryBean] -} diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/Pojo.java b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/Pojo.java index 24c0fea352..6046f2bb5d 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/Pojo.java +++ b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/Pojo.java @@ -14,7 +14,7 @@ public class Pojo extends TypedActor implements PojoInf, ApplicationContextAware private String stringFromRef; private boolean gotApplicationContext = false; - private boolean initInvoked = false; + private boolean preStartInvoked = false; public boolean gotApplicationContext() { return gotApplicationContext; @@ -41,11 +41,11 @@ public class Pojo extends TypedActor implements PojoInf, ApplicationContextAware } @Override - public void init() { - initInvoked = true; + public void preStart() { + preStartInvoked = true; } - public boolean isInitInvoked() { - return initInvoked; + public boolean isPreStartInvoked() { + return preStartInvoked; } } diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/PojoInf.java b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/PojoInf.java index 9ebf80e89b..0a313ceb18 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/PojoInf.java +++ b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/PojoInf.java @@ -8,6 +8,6 @@ public interface PojoInf { public String getStringFromVal(); public String getStringFromRef(); public boolean gotApplicationContext(); - public boolean isInitInvoked(); + public boolean isPreStartInvoked(); } diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleBean.java b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleBean.java index f9d3381436..29e80d1c65 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleBean.java +++ b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleBean.java @@ -19,7 +19,7 @@ public class SampleBean extends TypedActor implements SampleBeanIntf { } @Override - public void shutdown() { + public void postStop() { down = true; } } diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IMyPojo.java b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IMyPojo.java index f2c5e24884..5a2a272e6c 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IMyPojo.java +++ b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IMyPojo.java @@ -8,14 +8,12 @@ package se.scalablesolutions.akka.spring.foo; * To change this template use File | Settings | File Templates. */ public interface IMyPojo { + public void oneWay(String message); + public String getFoo(); - public String getBar(); - - public void preRestart(); - - public void postRestart(); - public String longRunning(); + + } diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/MyPojo.java b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/MyPojo.java index fe3e9ba767..8f610eef63 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/MyPojo.java +++ b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/MyPojo.java @@ -1,42 +1,34 @@ package se.scalablesolutions.akka.spring.foo; -import se.scalablesolutions.akka.actor.*; +import se.scalablesolutions.akka.actor.TypedActor; -public class MyPojo extends TypedActor implements IMyPojo{ +import java.util.concurrent.CountDownLatch; - private String foo; - private String bar; +public class MyPojo extends TypedActor implements IMyPojo { + + public static CountDownLatch latch = new CountDownLatch(1); + public static String lastOneWayMessage = null; + private String foo = "foo"; - public MyPojo() { - this.foo = "foo"; - this.bar = "bar"; - } + public MyPojo() { + } + public String getFoo() { + return foo; + } - public String getFoo() { - return foo; - } + public void oneWay(String message) { + lastOneWayMessage = message; + latch.countDown(); + } - - public String getBar() { - return bar; - } - - public void preRestart() { - System.out.println("pre restart"); - } - - public void postRestart() { - System.out.println("post restart"); - } - - public String longRunning() { - try { - Thread.sleep(6000); - } catch (InterruptedException e) { - } - return "this took long"; + public String longRunning() { + try { + Thread.sleep(6000); + } catch (InterruptedException e) { } + return "this took long"; + } } diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/PingActor.java b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/PingActor.java index e447b26a28..3063a1b529 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/PingActor.java +++ b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/PingActor.java @@ -6,6 +6,8 @@ import se.scalablesolutions.akka.actor.ActorRef; import org.springframework.context.ApplicationContext; import org.springframework.context.ApplicationContextAware; +import java.util.concurrent.CountDownLatch; + /** * test class @@ -14,6 +16,9 @@ public class PingActor extends UntypedActor implements ApplicationContextAware { private String stringFromVal; private String stringFromRef; + public static String lastMessage = null; + public static CountDownLatch latch = new CountDownLatch(1); + private boolean gotApplicationContext = false; @@ -42,7 +47,6 @@ public class PingActor extends UntypedActor implements ApplicationContextAware { stringFromRef = s; } - private String longRunning() { try { Thread.sleep(6000); @@ -53,12 +57,12 @@ public class PingActor extends UntypedActor implements ApplicationContextAware { public void onReceive(Object message) throws Exception { if (message instanceof String) { - System.out.println("Ping received String message: " + message); + lastMessage = (String) message; if (message.equals("longRunning")) { - System.out.println("### starting pong"); ActorRef pongActor = UntypedActor.actorOf(PongActor.class).start(); pongActor.sendRequestReply("longRunning", getContext()); } + latch.countDown(); } else { throw new IllegalArgumentException("Unknown message: " + message); } diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/StatefulPojo.java b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/StatefulPojo.java index 3b4e05453b..ce85267edc 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/StatefulPojo.java +++ b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/StatefulPojo.java @@ -5,6 +5,7 @@ import se.scalablesolutions.akka.stm.TransactionalMap; import se.scalablesolutions.akka.stm.TransactionalVector; import se.scalablesolutions.akka.stm.Ref; import se.scalablesolutions.akka.actor.*; +import se.scalablesolutions.akka.stm.local.Atomic; public class StatefulPojo extends TypedActor { private TransactionalMap mapState; @@ -13,12 +14,16 @@ public class StatefulPojo extends TypedActor { private boolean isInitialized = false; @Override - public void initTransactionalState() { - if (!isInitialized) { - mapState = new TransactionalMap(); - vectorState = new TransactionalVector(); - refState = new Ref(); - isInitialized = true; + public void preStart() { + if(!isInitialized) { + isInitialized = new Atomic() { + public Boolean atomically() { + mapState = new TransactionalMap(); + vectorState = new TransactionalVector(); + refState = new Ref(); + return true; + } + }.execute(); } } diff --git a/akka-spring/src/test/resources/dispatcher-config.xml b/akka-spring/src/test/resources/dispatcher-config.xml index 9f0dfa3802..728917c6c8 100644 --- a/akka-spring/src/test/resources/dispatcher-config.xml +++ b/akka-spring/src/test/resources/dispatcher-config.xml @@ -42,6 +42,13 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> bound="10" /> + + + + + - - - - - - - diff --git a/akka-spring/src/test/resources/server-managed-config.xml b/akka-spring/src/test/resources/server-managed-config.xml new file mode 100644 index 0000000000..128b16c8b6 --- /dev/null +++ b/akka-spring/src/test/resources/server-managed-config.xml @@ -0,0 +1,57 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/akka-spring/src/test/resources/typed-actor-config.xml b/akka-spring/src/test/resources/typed-actor-config.xml index faca749469..989884e4fa 100644 --- a/akka-spring/src/test/resources/typed-actor-config.xml +++ b/akka-spring/src/test/resources/typed-actor-config.xml @@ -37,7 +37,7 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> implementation="se.scalablesolutions.akka.spring.foo.MyPojo" timeout="2000" transactional="true"> - + - + val props = parser.parseDispatcher(dom(xml).getDocumentElement); assert(props != null) - assert(props.dispatcherType == "reactor-based-thread-pool-event-driven") + assert(props.dispatcherType == "executor-based-event-driven") assert(props.name == "myDispatcher") assert(props.threadPool.corePoolSize == 2) assert(props.threadPool.maxPoolSize == 10) @@ -86,16 +86,6 @@ class DispatcherBeanDefinitionParserTest extends Spec with ShouldMatchers { evaluating {parser.parseDispatcher(dom(xml).getDocumentElement)} should produce[IllegalArgumentException] } - it("should throw IllegalArgumentException when configuring a single thread dispatcher with a thread pool") { - val xml = - - - evaluating {parser.parseDispatcher(dom(xml).getDocumentElement)} should produce[IllegalArgumentException] - } - - it("should throw IllegalArgumentException when configuring a thread based dispatcher without TypedActor or UntypedActor") { val xml = evaluating {parser.parseDispatcher(dom(xml).getDocumentElement)} should produce[IllegalArgumentException] diff --git a/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala b/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala index 7b2c740c73..db62acde3f 100644 --- a/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala +++ b/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala @@ -6,8 +6,6 @@ package se.scalablesolutions.akka.spring import foo.{IMyPojo, MyPojo, PingActor} import se.scalablesolutions.akka.dispatch._ -import se.scalablesolutions.akka.actor.ActorRef - import org.scalatest.FeatureSpec import org.scalatest.matchers.ShouldMatchers import org.scalatest.junit.JUnitRunner @@ -18,6 +16,10 @@ import org.springframework.context.ApplicationContext import org.springframework.context.support.ClassPathXmlApplicationContext import org.springframework.core.io.{ClassPathResource, Resource} import java.util.concurrent._ +import se.scalablesolutions.akka.actor.{UntypedActor, Actor, ActorRef} + + + /** * Tests for spring configuration of typed actors. @@ -41,27 +43,39 @@ class DispatcherSpringFeatureTest extends FeatureSpec with ShouldMatchers { assert(executor.getQueue().remainingCapacity() === 100) } + scenario("get a dispatcher via ref from context") { val context = new ClassPathXmlApplicationContext("/dispatcher-config.xml") val pojo = context.getBean("typed-actor-with-dispatcher-ref").asInstanceOf[IMyPojo] assert(pojo != null) } - scenario("get a executor-event-driven-dispatcher with bounded-linked-blocking-queue with unbounded capacity from context") { + scenario("get a executor-event-driven-dispatcher with blocking-queue with unbounded capacity from context") { val context = new ClassPathXmlApplicationContext("/dispatcher-config.xml") val dispatcher = context.getBean("executor-event-driven-dispatcher-2").asInstanceOf[ExecutorBasedEventDrivenDispatcher] val executor = getThreadPoolExecutorAndAssert(dispatcher) - assert(executor.getQueue().isInstanceOf[LinkedBlockingQueue[Runnable]]) + assert(executor.getQueue().isInstanceOf[BlockingQueue[Runnable]]) assert(executor.getQueue().remainingCapacity() === Integer.MAX_VALUE) assert(dispatcher.name === EVENT_DRIVEN_PREFIX + "dispatcher-2") } + scenario("get a executor-event-driven-dispatcher with bounded-blocking-queue and with bounded mailbox capacity") { + val context = new ClassPathXmlApplicationContext("/dispatcher-config.xml") + val dispatcher = context.getBean("executor-event-driven-dispatcher-mc").asInstanceOf[ExecutorBasedEventDrivenDispatcher] + assert(dispatcher.name === EVENT_DRIVEN_PREFIX + "dispatcher-mc") + val actorRef = UntypedActor.actorOf(classOf[PingActor]) + actorRef.dispatcher = dispatcher + actorRef.start + assert(actorRef.mailbox.isInstanceOf[BlockingQueue[MessageInvocation]]) + assert((actorRef.mailbox.asInstanceOf[BlockingQueue[MessageInvocation]]).remainingCapacity === 1000) + } + scenario("get a executor-event-driven-dispatcher with unbounded-linked-blocking-queue with bounded capacity from context") { val context = new ClassPathXmlApplicationContext("/dispatcher-config.xml") val dispatcher = context.getBean("executor-event-driven-dispatcher-4").asInstanceOf[ExecutorBasedEventDrivenDispatcher] assert(dispatcher.name === EVENT_DRIVEN_PREFIX + "dispatcher-4") val executor = getThreadPoolExecutorAndAssert(dispatcher) - assert(executor.getQueue().isInstanceOf[LinkedBlockingQueue[Runnable]]) + assert(executor.getQueue().isInstanceOf[BlockingQueue[Runnable]]) assert(executor.getQueue().remainingCapacity() === 55) } @@ -70,7 +84,7 @@ class DispatcherSpringFeatureTest extends FeatureSpec with ShouldMatchers { val dispatcher = context.getBean("executor-event-driven-dispatcher-5").asInstanceOf[ExecutorBasedEventDrivenDispatcher] assert(dispatcher.name === EVENT_DRIVEN_PREFIX + "dispatcher-5") val executor = getThreadPoolExecutorAndAssert(dispatcher) - assert(executor.getQueue().isInstanceOf[LinkedBlockingQueue[Runnable]]) + assert(executor.getQueue().isInstanceOf[BlockingQueue[Runnable]]) assert(executor.getQueue().remainingCapacity() === Integer.MAX_VALUE) } @@ -82,19 +96,6 @@ class DispatcherSpringFeatureTest extends FeatureSpec with ShouldMatchers { assert(executor.getQueue().isInstanceOf[SynchronousQueue[Runnable]]) } - scenario("get a reactor-based-thread-pool-event-driven-dispatcher with synchronous-queue from context") { - val context = new ClassPathXmlApplicationContext("/dispatcher-config.xml") - val dispatcher = context.getBean("reactor-based-thread-pool-event-driven-dispatcher").asInstanceOf[ReactorBasedThreadPoolEventDrivenDispatcher] - val executor = getThreadPoolExecutorAndAssert(dispatcher) - assert(executor.getQueue().isInstanceOf[SynchronousQueue[Runnable]]) - } - - scenario("get a reactor-based-single-thread-event-driven-dispatcher with synchronous-queue from context") { - val context = new ClassPathXmlApplicationContext("/dispatcher-config.xml") - val dispatcher = context.getBean("reactor-based-single-thread-event-driven-dispatcher").asInstanceOf[ReactorBasedSingleThreadEventDrivenDispatcher] - assert(dispatcher != null) - } - scenario("get a executor-based-event-driven-work-stealing-dispatcher from context") { val context = new ClassPathXmlApplicationContext("/dispatcher-config.xml") val dispatcher = context.getBean("executor-based-event-driven-work-stealing-dispatcher").asInstanceOf[ExecutorBasedEventDrivenWorkStealingDispatcher] diff --git a/akka-spring/src/test/scala/TypedActorBeanDefinitionParserTest.scala b/akka-spring/src/test/scala/TypedActorBeanDefinitionParserTest.scala index 27a42f3d6c..bd0b018e75 100644 --- a/akka-spring/src/test/scala/TypedActorBeanDefinitionParserTest.scala +++ b/akka-spring/src/test/scala/TypedActorBeanDefinitionParserTest.scala @@ -19,7 +19,7 @@ import org.w3c.dom.Element class TypedActorBeanDefinitionParserTest extends Spec with ShouldMatchers { private class Parser extends ActorParser - describe("An TypedActorParser") { + describe("A TypedActorParser") { val parser = new Parser() it("should parse the typed actor configuration") { val xml = + + + val props = parser.parseActor(dom(xml).getDocumentElement); + assert(props != null) + assert(props.host === "com.some.host") + assert(props.port === 9999) + assert(props.serviceName === "my-service") + assert(props.serverManaged) } } } diff --git a/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala b/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala index 8767b2e75a..3cdcd17cb0 100644 --- a/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala +++ b/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala @@ -4,10 +4,8 @@ package se.scalablesolutions.akka.spring -import foo.{IMyPojo, MyPojo} +import foo.{PingActor, IMyPojo, MyPojo} import se.scalablesolutions.akka.dispatch.FutureTimeoutException -import se.scalablesolutions.akka.remote.RemoteNode -import org.scalatest.FeatureSpec import org.scalatest.matchers.ShouldMatchers import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith @@ -16,13 +14,52 @@ import org.springframework.beans.factory.xml.XmlBeanDefinitionReader import org.springframework.context.ApplicationContext import org.springframework.context.support.ClassPathXmlApplicationContext import org.springframework.core.io.{ClassPathResource, Resource} +import org.scalatest.{BeforeAndAfterAll, FeatureSpec} +import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer, RemoteNode} +import java.util.concurrent.CountDownLatch +import se.scalablesolutions.akka.actor.{TypedActor, RemoteTypedActorOne, Actor} +import se.scalablesolutions.akka.actor.remote.RemoteTypedActorOneImpl /** * Tests for spring configuration of typed actors. * @author michaelkober */ @RunWith(classOf[JUnitRunner]) -class TypedActorSpringFeatureTest extends FeatureSpec with ShouldMatchers { +class TypedActorSpringFeatureTest extends FeatureSpec with ShouldMatchers with BeforeAndAfterAll { + + var server1: RemoteServer = null + var server2: RemoteServer = null + + override def beforeAll = { + val actor = Actor.actorOf[PingActor] // FIXME: remove this line when ticket 425 is fixed + server1 = new RemoteServer() + server1.start("localhost", 9990) + server2 = new RemoteServer() + server2.start("localhost", 9992) + + val typedActor = TypedActor.newInstance(classOf[RemoteTypedActorOne], classOf[RemoteTypedActorOneImpl], 1000) + server1.registerTypedActor("typed-actor-service", typedActor) + } + + // make sure the servers shutdown cleanly after the test has finished + override def afterAll = { + try { + server1.shutdown + server2.shutdown + RemoteClient.shutdownAll + Thread.sleep(1000) + } catch { + case e => () + } + } + + def getTypedActorFromContext(config: String, id: String) : IMyPojo = { + MyPojo.latch = new CountDownLatch(1) + val context = new ClassPathXmlApplicationContext(config) + val myPojo: IMyPojo = context.getBean(id).asInstanceOf[IMyPojo] + myPojo + } + feature("parse Spring application context") { scenario("akka:typed-actor and akka:supervision and akka:dispatcher can be used as top level elements") { @@ -37,41 +74,79 @@ class TypedActorSpringFeatureTest extends FeatureSpec with ShouldMatchers { } scenario("get a typed actor") { - val context = new ClassPathXmlApplicationContext("/typed-actor-config.xml") - val myPojo = context.getBean("simple-typed-actor").asInstanceOf[IMyPojo] - var msg = myPojo.getFoo() - msg += myPojo.getBar() - assert(msg === "foobar") + val myPojo = getTypedActorFromContext("/typed-actor-config.xml", "simple-typed-actor") + assert(myPojo.getFoo() === "foo") + myPojo.oneWay("hello 1") + MyPojo.latch.await + assert(MyPojo.lastOneWayMessage === "hello 1") } scenario("FutureTimeoutException when timed out") { - val context = new ClassPathXmlApplicationContext("/typed-actor-config.xml") - val myPojo = context.getBean("simple-typed-actor").asInstanceOf[IMyPojo] + val myPojo = getTypedActorFromContext("/typed-actor-config.xml", "simple-typed-actor") evaluating {myPojo.longRunning()} should produce[FutureTimeoutException] - } scenario("typed-actor with timeout") { - val context = new ClassPathXmlApplicationContext("/typed-actor-config.xml") - val myPojo = context.getBean("simple-typed-actor-long-timeout").asInstanceOf[IMyPojo] + val myPojo = getTypedActorFromContext("/typed-actor-config.xml", "simple-typed-actor-long-timeout") assert(myPojo.longRunning() === "this took long"); } scenario("transactional typed-actor") { - val context = new ClassPathXmlApplicationContext("/typed-actor-config.xml") - val myPojo = context.getBean("transactional-typed-actor").asInstanceOf[IMyPojo] - var msg = myPojo.getFoo() - msg += myPojo.getBar() - assert(msg === "foobar") + val myPojo = getTypedActorFromContext("/typed-actor-config.xml", "transactional-typed-actor") + assert(myPojo.getFoo() === "foo") + myPojo.oneWay("hello 2") + MyPojo.latch.await + assert(MyPojo.lastOneWayMessage === "hello 2") } scenario("get a remote typed-actor") { - RemoteNode.start - Thread.sleep(1000) - val context = new ClassPathXmlApplicationContext("/typed-actor-config.xml") - val myPojo = context.getBean("remote-typed-actor").asInstanceOf[IMyPojo] - assert(myPojo.getFoo === "foo") + val myPojo = getTypedActorFromContext("/typed-actor-config.xml", "remote-typed-actor") + assert(myPojo.getFoo() === "foo") + myPojo.oneWay("hello 3") + MyPojo.latch.await + assert(MyPojo.lastOneWayMessage === "hello 3") } + + scenario("get a client-managed-remote-typed-actor") { + val myPojo = getTypedActorFromContext("/server-managed-config.xml", "client-managed-remote-typed-actor") + assert(myPojo.getFoo() === "foo") + myPojo.oneWay("hello client-managed-remote-typed-actor") + MyPojo.latch.await + assert(MyPojo.lastOneWayMessage === "hello client-managed-remote-typed-actor") + } + + scenario("get a server-managed-remote-typed-actor") { + val serverPojo = getTypedActorFromContext("/server-managed-config.xml", "server-managed-remote-typed-actor") + // + val myPojoProxy = RemoteClient.typedActorFor(classOf[IMyPojo], classOf[IMyPojo].getName, 5000L, "localhost", 9990) + assert(myPojoProxy.getFoo() === "foo") + myPojoProxy.oneWay("hello server-managed-remote-typed-actor") + MyPojo.latch.await + assert(MyPojo.lastOneWayMessage === "hello server-managed-remote-typed-actor") + } + + scenario("get a server-managed-remote-typed-actor-custom-id") { + val serverPojo = getTypedActorFromContext("/server-managed-config.xml", "server-managed-remote-typed-actor-custom-id") + // + val myPojoProxy = RemoteClient.typedActorFor(classOf[IMyPojo], "mypojo-service", 5000L, "localhost", 9990) + assert(myPojoProxy.getFoo() === "foo") + myPojoProxy.oneWay("hello server-managed-remote-typed-actor 2") + MyPojo.latch.await + assert(MyPojo.lastOneWayMessage === "hello server-managed-remote-typed-actor 2") + } + + scenario("get a client proxy for server-managed-remote-typed-actor") { + MyPojo.latch = new CountDownLatch(1) + val context = new ClassPathXmlApplicationContext("/server-managed-config.xml") + val myPojo: IMyPojo = context.getBean("server-managed-remote-typed-actor-custom-id").asInstanceOf[IMyPojo] + // get client proxy from spring context + val myPojoProxy = context.getBean("typed-client-1").asInstanceOf[IMyPojo] + assert(myPojoProxy.getFoo() === "foo") + myPojoProxy.oneWay("hello") + MyPojo.latch.await + } + + } } diff --git a/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala b/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala index cf7d8d9805..0397d30bf0 100644 --- a/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala +++ b/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala @@ -6,74 +6,146 @@ package se.scalablesolutions.akka.spring import foo.PingActor import se.scalablesolutions.akka.dispatch.ExecutorBasedEventDrivenWorkStealingDispatcher -import se.scalablesolutions.akka.remote.RemoteNode -import se.scalablesolutions.akka.actor.ActorRef -import org.scalatest.FeatureSpec import org.scalatest.matchers.ShouldMatchers import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import org.springframework.context.ApplicationContext import org.springframework.context.support.ClassPathXmlApplicationContext +import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer} +import org.scalatest.{BeforeAndAfterAll, FeatureSpec} +import java.util.concurrent.CountDownLatch +import se.scalablesolutions.akka.actor.{RemoteActorRef, ActorRegistry, Actor, ActorRef} /** * Tests for spring configuration of typed actors. * @author michaelkober */ @RunWith(classOf[JUnitRunner]) -class UntypedActorSpringFeatureTest extends FeatureSpec with ShouldMatchers { +class UntypedActorSpringFeatureTest extends FeatureSpec with ShouldMatchers with BeforeAndAfterAll { + + var server1: RemoteServer = null + var server2: RemoteServer = null + + + override def beforeAll = { + val actor = Actor.actorOf[PingActor] // FIXME: remove this line when ticket 425 is fixed + server1 = new RemoteServer() + server1.start("localhost", 9990) + server2 = new RemoteServer() + server2.start("localhost", 9992) + } + + // make sure the servers shutdown cleanly after the test has finished + override def afterAll = { + try { + server1.shutdown + server2.shutdown + RemoteClient.shutdownAll + Thread.sleep(1000) + } catch { + case e => () + } + } + + + def getPingActorFromContext(config: String, id: String) : ActorRef = { + PingActor.latch = new CountDownLatch(1) + val context = new ClassPathXmlApplicationContext(config) + val pingActor = context.getBean(id).asInstanceOf[ActorRef] + assert(pingActor.getActorClassName() === "se.scalablesolutions.akka.spring.foo.PingActor") + pingActor.start() + } + + feature("parse Spring application context") { scenario("get a untyped actor") { - val context = new ClassPathXmlApplicationContext("/untyped-actor-config.xml") - val myactor = context.getBean("simple-untyped-actor").asInstanceOf[ActorRef] - assert(myactor.getActorClassName() === "se.scalablesolutions.akka.spring.foo.PingActor") - myactor.start() + val myactor = getPingActorFromContext("/untyped-actor-config.xml", "simple-untyped-actor") myactor.sendOneWay("Hello") + PingActor.latch.await + assert(PingActor.lastMessage === "Hello") assert(myactor.isDefinedAt("some string message")) } scenario("untyped-actor with timeout") { - val context = new ClassPathXmlApplicationContext("/untyped-actor-config.xml") - val myactor = context.getBean("simple-untyped-actor-long-timeout").asInstanceOf[ActorRef] - assert(myactor.getActorClassName() === "se.scalablesolutions.akka.spring.foo.PingActor") - myactor.start() - myactor.sendOneWay("Hello") + val myactor = getPingActorFromContext("/untyped-actor-config.xml", "simple-untyped-actor-long-timeout") assert(myactor.getTimeout() === 10000) + myactor.sendOneWay("Hello 2") + PingActor.latch.await + assert(PingActor.lastMessage === "Hello 2") } scenario("transactional untyped-actor") { - val context = new ClassPathXmlApplicationContext("/untyped-actor-config.xml") - val myactor = context.getBean("transactional-untyped-actor").asInstanceOf[ActorRef] - assert(myactor.getActorClassName() === "se.scalablesolutions.akka.spring.foo.PingActor") - myactor.start() - myactor.sendOneWay("Hello") - assert(myactor.isDefinedAt("some string message")) + val myactor = getPingActorFromContext("/untyped-actor-config.xml", "transactional-untyped-actor") + myactor.sendOneWay("Hello 3") + PingActor.latch.await + assert(PingActor.lastMessage === "Hello 3") } scenario("get a remote typed-actor") { - RemoteNode.start - Thread.sleep(1000) - val context = new ClassPathXmlApplicationContext("/untyped-actor-config.xml") - val myactor = context.getBean("remote-untyped-actor").asInstanceOf[ActorRef] - assert(myactor.getActorClassName() === "se.scalablesolutions.akka.spring.foo.PingActor") - myactor.start() - myactor.sendOneWay("Hello") - assert(myactor.isDefinedAt("some string message")) + val myactor = getPingActorFromContext("/untyped-actor-config.xml", "remote-untyped-actor") + myactor.sendOneWay("Hello 4") assert(myactor.getRemoteAddress().isDefined) assert(myactor.getRemoteAddress().get.getHostName() === "localhost") - assert(myactor.getRemoteAddress().get.getPort() === 9999) + assert(myactor.getRemoteAddress().get.getPort() === 9992) + PingActor.latch.await + assert(PingActor.lastMessage === "Hello 4") } scenario("untyped-actor with custom dispatcher") { - val context = new ClassPathXmlApplicationContext("/untyped-actor-config.xml") - val myactor = context.getBean("untyped-actor-with-dispatcher").asInstanceOf[ActorRef] - assert(myactor.getActorClassName() === "se.scalablesolutions.akka.spring.foo.PingActor") - myactor.start() - myactor.sendOneWay("Hello") + val myactor = getPingActorFromContext("/untyped-actor-config.xml", "untyped-actor-with-dispatcher") assert(myactor.getTimeout() === 1000) assert(myactor.getDispatcher.isInstanceOf[ExecutorBasedEventDrivenWorkStealingDispatcher]) + myactor.sendOneWay("Hello 5") + PingActor.latch.await + assert(PingActor.lastMessage === "Hello 5") } + + scenario("create client managed remote untyped-actor") { + val myactor = getPingActorFromContext("/server-managed-config.xml", "client-managed-remote-untyped-actor") + myactor.sendOneWay("Hello client managed remote untyped-actor") + PingActor.latch.await + assert(PingActor.lastMessage === "Hello client managed remote untyped-actor") + assert(myactor.getRemoteAddress().isDefined) + assert(myactor.getRemoteAddress().get.getHostName() === "localhost") + assert(myactor.getRemoteAddress().get.getPort() === 9990) + } + + scenario("create server managed remote untyped-actor") { + val myactor = getPingActorFromContext("/server-managed-config.xml", "server-managed-remote-untyped-actor") + val nrOfActors = ActorRegistry.actors.length + val actorRef = RemoteClient.actorFor("se.scalablesolutions.akka.spring.foo.PingActor", "localhost", 9990) + actorRef.sendOneWay("Hello server managed remote untyped-actor") + PingActor.latch.await + assert(PingActor.lastMessage === "Hello server managed remote untyped-actor") + assert(ActorRegistry.actors.length === nrOfActors) + } + + scenario("create server managed remote untyped-actor with custom service id") { + val myactor = getPingActorFromContext("/server-managed-config.xml", "server-managed-remote-untyped-actor-custom-id") + val nrOfActors = ActorRegistry.actors.length + val actorRef = RemoteClient.actorFor("ping-service", "localhost", 9990) + actorRef.sendOneWay("Hello server managed remote untyped-actor") + PingActor.latch.await + assert(PingActor.lastMessage === "Hello server managed remote untyped-actor") + assert(ActorRegistry.actors.length === nrOfActors) + } + + scenario("get client actor for server managed remote untyped-actor") { + PingActor.latch = new CountDownLatch(1) + val context = new ClassPathXmlApplicationContext("/server-managed-config.xml") + val pingActor = context.getBean("server-managed-remote-untyped-actor-custom-id").asInstanceOf[ActorRef] + assert(pingActor.getActorClassName() === "se.scalablesolutions.akka.spring.foo.PingActor") + pingActor.start() + val nrOfActors = ActorRegistry.actors.length + // get client actor ref from spring context + val actorRef = context.getBean("client-1").asInstanceOf[ActorRef] + assert(actorRef.isInstanceOf[RemoteActorRef]) + actorRef.sendOneWay("Hello") + PingActor.latch.await + assert(ActorRegistry.actors.length === nrOfActors) + } + } } diff --git a/akka-typed-actor/src/main/scala/actor/TypedActor.scala b/akka-typed-actor/src/main/scala/actor/TypedActor.scala index b27f5b4b4d..c3457cb43b 100644 --- a/akka-typed-actor/src/main/scala/actor/TypedActor.scala +++ b/akka-typed-actor/src/main/scala/actor/TypedActor.scala @@ -16,9 +16,8 @@ import org.codehaus.aspectwerkz.proxy.Proxy import org.codehaus.aspectwerkz.annotation.{Aspect, Around} import java.net.InetSocketAddress -import java.lang.reflect.{InvocationTargetException, Method, Field} - import scala.reflect.BeanProperty +import java.lang.reflect.{Method, Field, InvocationHandler, Proxy => JProxy} /** * TypedActor is a type-safe actor made out of a POJO with interface. @@ -41,12 +40,12 @@ import scala.reflect.BeanProperty * } * * @Override - * public void init() { + * public void preStart() { * ... // optional initialization on start * } * * @Override - * public void shutdown() { + * public void postStop() { * ... // optional cleanup on stop * } * @@ -79,11 +78,11 @@ import scala.reflect.BeanProperty * * def square(x: Int): Future[Integer] = future(x * x) * - * override def init = { + * override def preStart = { * ... // optional initialization on start * } * - * override def shutdown = { + * override def postStop = { * ... // optional cleanup on stop * } * @@ -390,7 +389,8 @@ object TypedActor extends Logging { typedActor.initialize(proxy) if (config._messageDispatcher.isDefined) actorRef.dispatcher = config._messageDispatcher.get if (config._threadBasedDispatcher.isDefined) actorRef.dispatcher = Dispatchers.newThreadBasedDispatcher(actorRef) - AspectInitRegistry.register(proxy, AspectInit(intfClass, typedActor, actorRef, None, config.timeout)) + if (config._host.isDefined) actorRef.makeRemote(config._host.get) + AspectInitRegistry.register(proxy, AspectInit(intfClass, typedActor, actorRef, config._host, config.timeout)) actorRef.start proxy.asInstanceOf[T] } @@ -408,24 +408,47 @@ object TypedActor extends Logging { proxy.asInstanceOf[T] } -/* - // NOTE: currently not used - but keep it around - private[akka] def newInstance[T <: TypedActor](targetClass: Class[T], - remoteAddress: Option[InetSocketAddress], timeout: Long): T = { - val proxy = { - val instance = Proxy.newInstance(targetClass, true, false) - if (instance.isInstanceOf[TypedActor]) instance.asInstanceOf[TypedActor] - else throw new IllegalActorStateException("Actor [" + targetClass.getName + "] is not a sub class of 'TypedActor'") + /** + * Create a proxy for a RemoteActorRef representing a server managed remote typed actor. + * + */ + private[akka] def createProxyForRemoteActorRef[T](intfClass: Class[T], actorRef: ActorRef): T = { + + class MyInvocationHandler extends InvocationHandler { + def invoke(proxy: AnyRef, method: Method, args: Array[AnyRef]): AnyRef = { + // do nothing, this is just a dummy + null + } } - val context = injectTypedActorContext(proxy) - actorRef.actor.asInstanceOf[Dispatcher].initialize(targetClass, proxy, proxy, context) - actorRef.timeout = timeout - if (remoteAddress.isDefined) actorRef.makeRemote(remoteAddress.get) - AspectInitRegistry.register(proxy, AspectInit(targetClass, proxy, actorRef, remoteAddress, timeout)) - actorRef.start - proxy.asInstanceOf[T] + val handler = new MyInvocationHandler() + + val interfaces = Array(intfClass, classOf[ServerManagedTypedActor]).asInstanceOf[Array[java.lang.Class[_]]] + val jProxy = JProxy.newProxyInstance(intfClass.getClassLoader(), interfaces, handler) + val awProxy = Proxy.newInstance(interfaces, Array(jProxy, jProxy), true, false) + + AspectInitRegistry.register(awProxy, AspectInit(intfClass, null, actorRef, None, 5000L)) + awProxy.asInstanceOf[T] } -*/ + + + /* + // NOTE: currently not used - but keep it around + private[akka] def newInstance[T <: TypedActor](targetClass: Class[T], + remoteAddress: Option[InetSocketAddress], timeout: Long): T = { + val proxy = { + val instance = Proxy.newInstance(targetClass, true, false) + if (instance.isInstanceOf[TypedActor]) instance.asInstanceOf[TypedActor] + else throw new IllegalActorStateException("Actor [" + targetClass.getName + "] is not a sub class of 'TypedActor'") + } + val context = injectTypedActorContext(proxy) + actorRef.actor.asInstanceOf[Dispatcher].initialize(targetClass, proxy, proxy, context) + actorRef.timeout = timeout + if (remoteAddress.isDefined) actorRef.makeRemote(remoteAddress.get) + AspectInitRegistry.register(proxy, AspectInit(targetClass, proxy, actorRef, remoteAddress, timeout)) + actorRef.start + proxy.asInstanceOf[T] + } + */ /** * Stops the current Typed Actor. @@ -519,11 +542,7 @@ object TypedActor extends Logging { val typedActor = if (instance.isInstanceOf[TypedActor]) instance.asInstanceOf[TypedActor] else throw new IllegalArgumentException("Actor [" + targetClass.getName + "] is not a sub class of 'TypedActor'") - typedActor.init - import se.scalablesolutions.akka.stm.local.atomic - atomic { - typedActor.initTransactionalState - } + typedActor.preStart typedActor } @@ -546,6 +565,30 @@ object TypedActor extends Logging { private[akka] def isJoinPoint(message: Any): Boolean = message.isInstanceOf[JoinPoint] } + +/** + * AspectWerkz Aspect that is turning POJO into proxy to a server managed remote TypedActor. + *

    + * Is deployed on a 'perInstance' basis with the pointcut 'execution(* *.*(..))', + * e.g. all methods on the instance. + * + * @author Jonas Bonér + */ +@Aspect("perInstance") +private[akka] sealed class ServerManagedTypedActorAspect extends ActorAspect { + + @Around("execution(* *.*(..)) && this(se.scalablesolutions.akka.actor.ServerManagedTypedActor)") + def invoke(joinPoint: JoinPoint): AnyRef = { + if (!isInitialized) initialize(joinPoint) + remoteDispatch(joinPoint) + } + + override def initialize(joinPoint: JoinPoint): Unit = { + super.initialize(joinPoint) + remoteAddress = actorRef.remoteAddress + } +} + /** * AspectWerkz Aspect that is turning POJO into TypedActor. *

    @@ -555,18 +598,9 @@ object TypedActor extends Logging { * @author Jonas Bonér */ @Aspect("perInstance") -private[akka] sealed class TypedActorAspect { - @volatile private var isInitialized = false - @volatile private var isStopped = false - private var interfaceClass: Class[_] = _ - private var typedActor: TypedActor = _ - private var actorRef: ActorRef = _ - private var remoteAddress: Option[InetSocketAddress] = _ - private var timeout: Long = _ - private var uuid: String = _ - @volatile private var instance: TypedActor = _ +private[akka] sealed class TypedActorAspect extends ActorAspect { - @Around("execution(* *.*(..))") + @Around("execution(* *.*(..)) && !this(se.scalablesolutions.akka.actor.ServerManagedTypedActor)") def invoke(joinPoint: JoinPoint): AnyRef = { if (!isInitialized) initialize(joinPoint) dispatch(joinPoint) @@ -576,12 +610,26 @@ private[akka] sealed class TypedActorAspect { if (remoteAddress.isDefined) remoteDispatch(joinPoint) else localDispatch(joinPoint) } +} - private def localDispatch(joinPoint: JoinPoint): AnyRef = { - val methodRtti = joinPoint.getRtti.asInstanceOf[MethodRtti] - val isOneWay = TypedActor.isOneWay(methodRtti) +/** + * Base class for TypedActorAspect and ServerManagedTypedActorAspect to reduce code duplication. + */ +private[akka] abstract class ActorAspect { + @volatile protected var isInitialized = false + @volatile protected var isStopped = false + protected var interfaceClass: Class[_] = _ + protected var typedActor: TypedActor = _ + protected var actorRef: ActorRef = _ + protected var timeout: Long = _ + protected var uuid: String = _ + protected var remoteAddress: Option[InetSocketAddress] = _ + + protected def localDispatch(joinPoint: JoinPoint): AnyRef = { + val methodRtti = joinPoint.getRtti.asInstanceOf[MethodRtti] + val isOneWay = TypedActor.isOneWay(methodRtti) val senderActorRef = Some(SenderContextInfo.senderActorRef.value) - val senderProxy = Some(SenderContextInfo.senderProxy.value) + val senderProxy = Some(SenderContextInfo.senderProxy.value) typedActor.context._sender = senderProxy if (!actorRef.isRunning && !isStopped) { @@ -602,7 +650,7 @@ private[akka] sealed class TypedActorAspect { } } - private def remoteDispatch(joinPoint: JoinPoint): AnyRef = { + protected def remoteDispatch(joinPoint: JoinPoint): AnyRef = { val methodRtti = joinPoint.getRtti.asInstanceOf[MethodRtti] val isOneWay = TypedActor.isOneWay(methodRtti) @@ -641,7 +689,7 @@ private[akka] sealed class TypedActorAspect { (escapedArgs, isEscaped) } - private def initialize(joinPoint: JoinPoint): Unit = { + protected def initialize(joinPoint: JoinPoint): Unit = { val init = AspectInitRegistry.initFor(joinPoint.getThis) interfaceClass = init.interfaceClass typedActor = init.targetInstance @@ -653,6 +701,7 @@ private[akka] sealed class TypedActorAspect { } } + /** * Internal helper class to help pass the contextual information between threads. * @@ -674,7 +723,7 @@ private[akka] object AspectInitRegistry extends ListenerManagement { def register(proxy: AnyRef, init: AspectInit) = { val res = initializations.put(proxy, init) - foreachListener(_ ! AspectInitRegistered(proxy, init)) + notifyListeners(AspectInitRegistered(proxy, init)) res } @@ -683,7 +732,7 @@ private[akka] object AspectInitRegistry extends ListenerManagement { */ def unregister(proxy: AnyRef): AspectInit = { val init = initializations.remove(proxy) - foreachListener(_ ! AspectInitUnregistered(proxy, init)) + notifyListeners(AspectInitUnregistered(proxy, init)) init.actorRef.stop init } @@ -704,5 +753,11 @@ private[akka] sealed case class AspectInit( val timeout: Long) { def this(interfaceClass: Class[_], targetInstance: TypedActor, actorRef: ActorRef, timeout: Long) = this(interfaceClass, targetInstance, actorRef, None, timeout) + } + +/** + * Marker interface for server manager typed actors. + */ +private[akka] sealed trait ServerManagedTypedActor extends TypedActor diff --git a/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala b/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala index 339c4d297d..5ca249a3ec 100644 --- a/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala +++ b/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala @@ -122,7 +122,6 @@ private[akka] class TypedActorGuiceConfigurator extends TypedActorConfiguratorBa remoteAddress.foreach { address => actorRef.makeRemote(remoteAddress.get) - RemoteServerModule.registerTypedActor(address, implementationClass.getName, proxy) } AspectInitRegistry.register( diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActorImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActorImpl.java index 1b95517c22..cb002b0a9e 100644 --- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActorImpl.java +++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActorImpl.java @@ -10,7 +10,7 @@ public class NestedTransactionalTypedActorImpl extends TypedTransactor implement private boolean isInitialized = false; @Override - public void init() { + public void preStart() { if (!isInitialized) { mapState = new TransactionalMap(); vectorState = new TransactionalVector(); diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojoImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojoImpl.java index 12985c72ce..1e567014d9 100644 --- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojoImpl.java +++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojoImpl.java @@ -38,7 +38,7 @@ public class SamplePojoImpl extends TypedActor implements SamplePojo { } @Override - public void shutdown() { + public void postStop() { _down = true; latch.countDown(); } diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActorImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActorImpl.java index 9b32f5d329..45bda4a675 100644 --- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActorImpl.java +++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActorImpl.java @@ -2,6 +2,8 @@ package se.scalablesolutions.akka.actor; import se.scalablesolutions.akka.actor.*; import se.scalablesolutions.akka.stm.*; +import se.scalablesolutions.akka.stm.local.*; +import se.scalablesolutions.akka.stm.local.Atomic; public class TransactionalTypedActorImpl extends TypedTransactor implements TransactionalTypedActor { private TransactionalMap mapState; @@ -10,12 +12,16 @@ public class TransactionalTypedActorImpl extends TypedTransactor implements Tran private boolean isInitialized = false; @Override - public void initTransactionalState() { + public void preStart() { if (!isInitialized) { - mapState = new TransactionalMap(); - vectorState = new TransactionalVector(); - refState = new Ref(); - isInitialized = true; + isInitialized = new Atomic() { + public Boolean atomically() { + mapState = new TransactionalMap(); + vectorState = new TransactionalVector(); + refState = new Ref(); + return true; + } + }.execute(); } } diff --git a/akka-typed-actor/src/test/resources/META-INF/aop.xml b/akka-typed-actor/src/test/resources/META-INF/aop.xml index bdc167ca54..be133a51b8 100644 --- a/akka-typed-actor/src/test/resources/META-INF/aop.xml +++ b/akka-typed-actor/src/test/resources/META-INF/aop.xml @@ -2,6 +2,7 @@ + diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala index 10fc40493b..052f4cc7de 100644 --- a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala +++ b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala @@ -87,7 +87,7 @@ class TypedActorLifecycleSpec extends Spec with ShouldMatchers with BeforeAndAft SamplePojoImpl.reset val pojo = TypedActor.newInstance(classOf[SimpleJavaPojo], classOf[SimpleJavaPojoImpl]) val supervisor = TypedActor.newInstance(classOf[SimpleJavaPojo], classOf[SimpleJavaPojoImpl]) - link(supervisor, pojo, new OneForOneStrategy(3, 2000), Array(classOf[Throwable])) + link(supervisor, pojo, OneForOneStrategy(3, 2000), Array(classOf[Throwable])) pojo.throwException Thread.sleep(500) SimpleJavaPojoImpl._pre should be(true) @@ -95,7 +95,7 @@ class TypedActorLifecycleSpec extends Spec with ShouldMatchers with BeforeAndAft } /* - it("should shutdown non-supervised, annotated typed actor on TypedActor.stop") { + it("should postStop non-supervised, annotated typed actor on TypedActor.stop") { val obj = TypedActor.newInstance(classOf[SamplePojoAnnotated]) assert(AspectInitRegistry.initFor(obj) ne null) assert("hello akka" === obj.greet("akka")) @@ -112,7 +112,7 @@ class TypedActorLifecycleSpec extends Spec with ShouldMatchers with BeforeAndAft } } - it("should shutdown non-supervised, annotated typed actor on ActorRegistry.shutdownAll") { + it("should postStop non-supervised, annotated typed actor on ActorRegistry.shutdownAll") { val obj = TypedActor.newInstance(classOf[SamplePojoAnnotated]) assert(AspectInitRegistry.initFor(obj) ne null) assert("hello akka" === obj.greet("akka")) @@ -147,7 +147,7 @@ class TypedActorLifecycleSpec extends Spec with ShouldMatchers with BeforeAndAft } } - it("should shutdown supervised, annotated typed actor on failure") { + it("should postStop supervised, annotated typed actor on failure") { val obj = conf2.getInstance[SamplePojoAnnotated](classOf[SamplePojoAnnotated]) val cdl = obj.newCountdownLatch(1) assert(AspectInitRegistry.initFor(obj) ne null) diff --git a/config/akka-reference.conf b/config/akka-reference.conf index 9f8ec799f0..10a9c84118 100644 --- a/config/akka-reference.conf +++ b/config/akka-reference.conf @@ -25,6 +25,7 @@ akka { # - TypedActor: methods with non-void return type serialize-messages = off # Does a deep clone of (non-primitive) messages to ensure immutability throughput = 5 # Default throughput for all ExecutorBasedEventDrivenDispatcher, set to 1 for complete fairness + throughput-deadline-ms = -1 # Default throughput deadline for all ExecutorBasedEventDrivenDispatcher, set to 0 or negative for no deadline default-dispatcher { type = "GlobalExecutorBasedEventDriven" # Must be one of the following, all "Global*" are non-configurable @@ -44,20 +45,34 @@ akka { allow-core-timeout = on # Allow core threads to time out rejection-policy = "caller-runs" # abort, caller-runs, discard-oldest, discard throughput = 5 # Throughput for ExecutorBasedEventDrivenDispatcher, set to 1 for complete fairness + throughput-deadline-ms = -1 # Throughput deadline for ExecutorBasedEventDrivenDispatcher, set to 0 or negative for no deadline aggregate = off # Aggregate on/off for HawtDispatchers mailbox-capacity = -1 # If negative (or zero) then an unbounded mailbox is used (default) # If positive then a bounded mailbox is used and the capacity is set using the property # NOTE: setting a mailbox to 'blocking' can be a bit dangerous, # could lead to deadlock, use with care + # + # The following are only used for ExecutorBasedEventDriven + # and only if mailbox-capacity > 0 + mailbox-push-timeout-ms = 10000 # Specifies the timeout (in milliseconds) to add a new message to a mailbox that is full } } stm { - fair = on # Should global transactions be fair or non-fair (non fair yield better performance) - jta-aware = off # Option 'on' means that if there JTA Transaction Manager available then the STM will - # begin (or join), commit or rollback the JTA transaction. Default is 'off'. - timeout = 5 # Default timeout for blocking transactions and transaction set (in unit defined by - # the time-unit property) + fair = on # Should global transactions be fair or non-fair (non fair yield better performance) + max-retries = 1000 + timeout = 5 # Default timeout for blocking transactions and transaction set (in unit defined by + # the time-unit property) + write-skew = true + blocking-allowed = false + interruptible = false + speculative = true + quick-release = true + propagation = "requires" + trace-level = "none" + hooks = true + jta-aware = off # Option 'on' means that if there JTA Transaction Manager available then the STM will + # begin (or join), commit or rollback the JTA transaction. Default is 'off'. } jta { diff --git a/config/microkernel-server.xml b/config/microkernel-server.xml index ecb4bee120..d7b8087428 100644 --- a/config/microkernel-server.xml +++ b/config/microkernel-server.xml @@ -21,10 +21,7 @@ - - - 10 - 200 + diff --git a/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.jar b/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.jar deleted file mode 100644 index 5d2a6a3632..0000000000 Binary files a/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.jar and /dev/null differ diff --git a/embedded-repo/com/redis/redisclient/1.2/redisclient-1.2.jar b/embedded-repo/com/redis/redisclient/1.2/redisclient-1.2.jar deleted file mode 100644 index 91ff84b97c..0000000000 Binary files a/embedded-repo/com/redis/redisclient/1.2/redisclient-1.2.jar and /dev/null differ diff --git a/embedded-repo/com/redis/redisclient/2.8.0-2.0/redisclient-2.8.0-2.0.jar b/embedded-repo/com/redis/redisclient/2.8.0-2.0/redisclient-2.8.0-2.0.jar new file mode 100644 index 0000000000..66c18b6fbf Binary files /dev/null and b/embedded-repo/com/redis/redisclient/2.8.0-2.0/redisclient-2.8.0-2.0.jar differ diff --git a/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom b/embedded-repo/com/redis/redisclient/2.8.0-2.0/redisclient-2.8.0-2.0.pom old mode 100755 new mode 100644 similarity index 90% rename from embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom rename to embedded-repo/com/redis/redisclient/2.8.0-2.0/redisclient-2.8.0-2.0.pom index 16dd81402a..12558da1c4 --- a/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom +++ b/embedded-repo/com/redis/redisclient/2.8.0-2.0/redisclient-2.8.0-2.0.pom @@ -3,6 +3,6 @@ 4.0.0 com.redis redisclient - 1.1 + 2.8.0-2.0 jar - + \ No newline at end of file diff --git a/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.2/redisclient-2.8.0.Beta1-1.2.jar b/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.2/redisclient-2.8.0.Beta1-1.2.jar deleted file mode 100644 index 3f1593380b..0000000000 Binary files a/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.2/redisclient-2.8.0.Beta1-1.2.jar and /dev/null differ diff --git a/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.2/redisclient-2.8.0.Beta1-1.2.pom b/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.2/redisclient-2.8.0.Beta1-1.2.pom deleted file mode 100755 index 68f3763187..0000000000 --- a/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.2/redisclient-2.8.0.Beta1-1.2.pom +++ /dev/null @@ -1,8 +0,0 @@ - - - 4.0.0 - com.redis - redisclient - 2.8.0.Beta1-1.2 - jar - diff --git a/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.3-SNAPSHOT/redisclient-2.8.0.Beta1-1.3-SNAPSHOT.jar b/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.3-SNAPSHOT/redisclient-2.8.0.Beta1-1.3-SNAPSHOT.jar deleted file mode 100644 index 0daede37f0..0000000000 Binary files a/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.3-SNAPSHOT/redisclient-2.8.0.Beta1-1.3-SNAPSHOT.jar and /dev/null differ diff --git a/embedded-repo/com/redis/redisclient/2.8.0.RC2-1.4-SNAPSHOT/redisclient-2.8.0.RC2-1.4-SNAPSHOT.jar b/embedded-repo/com/redis/redisclient/2.8.0.RC2-1.4-SNAPSHOT/redisclient-2.8.0.RC2-1.4-SNAPSHOT.jar deleted file mode 100644 index 261b5cc1be..0000000000 Binary files a/embedded-repo/com/redis/redisclient/2.8.0.RC2-1.4-SNAPSHOT/redisclient-2.8.0.RC2-1.4-SNAPSHOT.jar and /dev/null differ diff --git a/embedded-repo/com/redis/redisclient/2.8.0.RC3-1.4-SNAPSHOT/redisclient-2.8.0.RC3-1.4-SNAPSHOT.jar b/embedded-repo/com/redis/redisclient/2.8.0.RC3-1.4-SNAPSHOT/redisclient-2.8.0.RC3-1.4-SNAPSHOT.jar deleted file mode 100644 index d939a49d7c..0000000000 Binary files a/embedded-repo/com/redis/redisclient/2.8.0.RC3-1.4-SNAPSHOT/redisclient-2.8.0.RC3-1.4-SNAPSHOT.jar and /dev/null differ diff --git a/embedded-repo/com/redis/redisclient/2.8.0.RC3-1.4/redisclient-2.8.0.RC3-1.4.jar b/embedded-repo/com/redis/redisclient/2.8.0.RC3-1.4/redisclient-2.8.0.RC3-1.4.jar deleted file mode 100644 index 351ff49c9d..0000000000 Binary files a/embedded-repo/com/redis/redisclient/2.8.0.RC3-1.4/redisclient-2.8.0.RC3-1.4.jar and /dev/null differ diff --git a/embedded-repo/sjson/json/sjson/0.4/sjson-0.4.jar b/embedded-repo/sjson/json/sjson/0.4/sjson-0.4.jar deleted file mode 100644 index dcab322c81..0000000000 Binary files a/embedded-repo/sjson/json/sjson/0.4/sjson-0.4.jar and /dev/null differ diff --git a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.Beta1/sjson-0.5-SNAPSHOT-2.8.Beta1.jar b/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.Beta1/sjson-0.5-SNAPSHOT-2.8.Beta1.jar deleted file mode 100644 index 4543815742..0000000000 Binary files a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.Beta1/sjson-0.5-SNAPSHOT-2.8.Beta1.jar and /dev/null differ diff --git a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.Beta1/sjson-0.5-SNAPSHOT-2.8.Beta1.pom b/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.Beta1/sjson-0.5-SNAPSHOT-2.8.Beta1.pom deleted file mode 100755 index 9c6db30374..0000000000 --- a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.Beta1/sjson-0.5-SNAPSHOT-2.8.Beta1.pom +++ /dev/null @@ -1,8 +0,0 @@ - - - 4.0.0 - sjson.json - sjson - 0.5-SNAPSHOT-2.8.Beta1 - jar - diff --git a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.RC2/sjson-0.5-SNAPSHOT-2.8.RC2.jar b/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.RC2/sjson-0.5-SNAPSHOT-2.8.RC2.jar deleted file mode 100644 index 11d149aeee..0000000000 Binary files a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.RC2/sjson-0.5-SNAPSHOT-2.8.RC2.jar and /dev/null differ diff --git a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.RC2/sjson-0.5-SNAPSHOT-2.8.RC2.pom b/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.RC2/sjson-0.5-SNAPSHOT-2.8.RC2.pom deleted file mode 100644 index fa80e90e01..0000000000 --- a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.RC2/sjson-0.5-SNAPSHOT-2.8.RC2.pom +++ /dev/null @@ -1,9 +0,0 @@ - - - 4.0.0 - sjson.json - sjson - 0.5-SNAPSHOT-2.8.RC2 - POM was created from install:install-file - diff --git a/embedded-repo/sjson/json/sjson/0.6-SNAPSHOT-2.8.RC3/sjson-0.6-SNAPSHOT-2.8.RC3.jar b/embedded-repo/sjson/json/sjson/0.6-SNAPSHOT-2.8.RC3/sjson-0.6-SNAPSHOT-2.8.RC3.jar deleted file mode 100644 index 5cf971df15..0000000000 Binary files a/embedded-repo/sjson/json/sjson/0.6-SNAPSHOT-2.8.RC3/sjson-0.6-SNAPSHOT-2.8.RC3.jar and /dev/null differ diff --git a/embedded-repo/sjson/json/sjson/0.6-SNAPSHOT-2.8.RC3/sjson-0.6-SNAPSHOT-2.8.RC3.pom b/embedded-repo/sjson/json/sjson/0.6-SNAPSHOT-2.8.RC3/sjson-0.6-SNAPSHOT-2.8.RC3.pom deleted file mode 100644 index 708571fc10..0000000000 --- a/embedded-repo/sjson/json/sjson/0.6-SNAPSHOT-2.8.RC3/sjson-0.6-SNAPSHOT-2.8.RC3.pom +++ /dev/null @@ -1,9 +0,0 @@ - - - 4.0.0 - sjson.json - sjson - 0.6-SNAPSHOT-2.8.RC3 - POM was created from install:install-file - diff --git a/embedded-repo/sjson/json/sjson/0.7-2.8.0/sjson-0.7-2.8.0.pom b/embedded-repo/sjson/json/sjson/0.7-2.8.0/sjson-0.7-2.8.0.pom deleted file mode 100644 index dfc9017dcf..0000000000 --- a/embedded-repo/sjson/json/sjson/0.7-2.8.0/sjson-0.7-2.8.0.pom +++ /dev/null @@ -1,9 +0,0 @@ - - - 4.0.0 - sjson.json - sjson - 0.7-2.8.0 - POM was created from install:install-file - diff --git a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.0/sjson-0.7-SNAPSHOT-2.8.0.jar b/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.0/sjson-0.7-SNAPSHOT-2.8.0.jar deleted file mode 100644 index f0d321a9e3..0000000000 Binary files a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.0/sjson-0.7-SNAPSHOT-2.8.0.jar and /dev/null differ diff --git a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.0/sjson-0.7-SNAPSHOT-2.8.0.pom b/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.0/sjson-0.7-SNAPSHOT-2.8.0.pom deleted file mode 100644 index e8081164fc..0000000000 --- a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.0/sjson-0.7-SNAPSHOT-2.8.0.pom +++ /dev/null @@ -1,9 +0,0 @@ - - - 4.0.0 - sjson.json - sjson - 0.7-SNAPSHOT-2.8.0 - POM was created from install:install-file - diff --git a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.RC7/sjson-0.7-SNAPSHOT-2.8.RC7.jar b/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.RC7/sjson-0.7-SNAPSHOT-2.8.RC7.jar deleted file mode 100644 index 6c77d359b4..0000000000 Binary files a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.RC7/sjson-0.7-SNAPSHOT-2.8.RC7.jar and /dev/null differ diff --git a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.RC7/sjson-0.7-SNAPSHOT-2.8.RC7.pom b/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.RC7/sjson-0.7-SNAPSHOT-2.8.RC7.pom deleted file mode 100644 index 7c02578f5f..0000000000 --- a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.RC7/sjson-0.7-SNAPSHOT-2.8.RC7.pom +++ /dev/null @@ -1,9 +0,0 @@ - - - 4.0.0 - sjson.json - sjson - 0.7-SNAPSHOT-2.8.RC7 - POM was created from install:install-file - diff --git a/embedded-repo/sjson/json/sjson/0.8-2.8.0/sjson-0.8-2.8.0.jar b/embedded-repo/sjson/json/sjson/0.8-2.8.0/sjson-0.8-2.8.0.jar new file mode 100644 index 0000000000..7dc4a9f66d Binary files /dev/null and b/embedded-repo/sjson/json/sjson/0.8-2.8.0/sjson-0.8-2.8.0.jar differ diff --git a/embedded-repo/sjson/json/sjson/0.4/sjson-0.4.pom b/embedded-repo/sjson/json/sjson/0.8-2.8.0/sjson-0.8-2.8.0.pom old mode 100755 new mode 100644 similarity index 92% rename from embedded-repo/sjson/json/sjson/0.4/sjson-0.4.pom rename to embedded-repo/sjson/json/sjson/0.8-2.8.0/sjson-0.8-2.8.0.pom index 784adff55e..51167926f1 --- a/embedded-repo/sjson/json/sjson/0.4/sjson-0.4.pom +++ b/embedded-repo/sjson/json/sjson/0.8-2.8.0/sjson-0.8-2.8.0.pom @@ -3,6 +3,6 @@ 4.0.0 sjson.json sjson - 0.4 + 0.8-2.8.0 jar diff --git a/embedded-repo/sjson/json/sjson/0.7-2.8.0/sjson-0.7-2.8.0.jar b/embedded-repo/sjson/json/sjson/0.8-SNAPSHOT-2.8.0/sjson-0.8-SNAPSHOT-2.8.0.jar similarity index 67% rename from embedded-repo/sjson/json/sjson/0.7-2.8.0/sjson-0.7-2.8.0.jar rename to embedded-repo/sjson/json/sjson/0.8-SNAPSHOT-2.8.0/sjson-0.8-SNAPSHOT-2.8.0.jar index e8700f11d3..1542632a82 100644 Binary files a/embedded-repo/sjson/json/sjson/0.7-2.8.0/sjson-0.7-2.8.0.jar and b/embedded-repo/sjson/json/sjson/0.8-SNAPSHOT-2.8.0/sjson-0.8-SNAPSHOT-2.8.0.jar differ diff --git a/project/build/AkkaProject.scala b/project/build/AkkaProject.scala index cf7378b0d2..ad1339da56 100644 --- a/project/build/AkkaProject.scala +++ b/project/build/AkkaProject.scala @@ -41,6 +41,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { object Repositories { lazy val AkkaRepo = MavenRepository("Akka Repository", "http://scalablesolutions.se/akka/repository") + lazy val CasbahRepo = MavenRepository("Casbah Repo", "http://repo.bumnetworks.com/releases") + lazy val CasbahSnapshotRepo = MavenRepository("Casbah Snapshots", "http://repo.bumnetworks.com/snapshots") lazy val CodehausRepo = MavenRepository("Codehaus Repo", "http://repository.codehaus.org") lazy val EmbeddedRepo = MavenRepository("Embedded Repo", (info.projectPath / "embedded-repo").asURL.toString) lazy val FusesourceSnapshotRepo = MavenRepository("Fusesource Snapshots", "http://repo.fusesource.com/nexus/content/repositories/snapshots") @@ -70,12 +72,13 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { lazy val jerseyContrModuleConfig = ModuleConfiguration("com.sun.jersey.contribs", JavaNetRepo) lazy val jerseyModuleConfig = ModuleConfiguration("com.sun.jersey", JavaNetRepo) lazy val jgroupsModuleConfig = ModuleConfiguration("jgroups", JBossRepo) - lazy val liftModuleConfig = ModuleConfiguration("net.liftweb", ScalaToolsReleases) lazy val multiverseModuleConfig = ModuleConfiguration("org.multiverse", CodehausRepo) lazy val nettyModuleConfig = ModuleConfiguration("org.jboss.netty", JBossRepo) lazy val scalaTestModuleConfig = ModuleConfiguration("org.scalatest", ScalaToolsSnapshots) lazy val logbackModuleConfig = ModuleConfiguration("ch.qos.logback",sbt.DefaultMavenRepository) lazy val atomikosModuleConfig = ModuleConfiguration("com.atomikos",sbt.DefaultMavenRepository) + lazy val casbahModuleConfig = ModuleConfiguration("com.novus", CasbahRepo) + lazy val timeModuleConfig = ModuleConfiguration("org.scala-tools", "time", CasbahSnapshotRepo) lazy val embeddedRepo = EmbeddedRepo // This is the only exception, because the embedded repo is fast! // ------------------------------------------------------------------------------------------------------------------- @@ -89,14 +92,13 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { lazy val HAWT_DISPATCH_VERSION = "1.0" lazy val JACKSON_VERSION = "1.2.1" lazy val JERSEY_VERSION = "1.2" - lazy val LIFT_VERSION = "2.1-M1" - lazy val MULTIVERSE_VERSION = "0.6" + 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" lazy val SLF4J_VERSION = "1.6.0" lazy val SPRING_VERSION = "3.0.3.RELEASE" lazy val ASPECTWERKZ_VERSION = "2.2.1" - lazy val JETTY_VERSION = "7.1.6.v20100715" + lazy val JETTY_VERSION = "7.1.4.v20100610" // ------------------------------------------------------------------------------------------------------------------- // Dependencies @@ -136,9 +138,10 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { lazy val dispatch_http = "net.databinder" % "dispatch-http_2.8.0" % DISPATCH_VERSION % "compile" lazy val dispatch_json = "net.databinder" % "dispatch-json_2.8.0" % DISPATCH_VERSION % "compile" - lazy val jetty = "org.eclipse.jetty" % "jetty-server" % JETTY_VERSION % "compile" - lazy val jetty_util = "org.eclipse.jetty" % "jetty-util" % JETTY_VERSION % "compile" - lazy val jetty_xml = "org.eclipse.jetty" % "jetty-xml" % JETTY_VERSION % "compile" + lazy val jetty = "org.eclipse.jetty" % "jetty-server" % JETTY_VERSION % "compile" + lazy val jetty_util = "org.eclipse.jetty" % "jetty-util" % JETTY_VERSION % "compile" + 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" @@ -167,11 +170,10 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { lazy val jta_1_1 = "org.apache.geronimo.specs" % "geronimo-jta_1.1_spec" % "1.1.1" % "compile" intransitive - lazy val lift_util = "net.liftweb" % "lift-util_2.8.0" % LIFT_VERSION % "compile" - lazy val lift_webkit = "net.liftweb" % "lift-webkit_2.8.0" % LIFT_VERSION % "compile" - lazy val mongo = "org.mongodb" % "mongo-java-driver" % "2.0" % "compile" + lazy val casbah = "com.novus" % "casbah_2.8.0" % "1.0.8.5" % "compile" + lazy val multiverse = "org.multiverse" % "multiverse-alpha" % MULTIVERSE_VERSION % "compile" intransitive lazy val netty = "org.jboss.netty" % "netty" % "3.2.2.Final" % "compile" @@ -182,13 +184,11 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { lazy val rabbit = "com.rabbitmq" % "amqp-client" % "1.8.1" % "compile" - lazy val redis = "com.redis" % "redisclient" % "2.8.0-1.4" % "compile" + lazy val redis = "com.redis" % "redisclient" % "2.8.0-2.0" % "compile" lazy val sbinary = "sbinary" % "sbinary" % "2.8.0-0.3.1" % "compile" - lazy val servlet = "javax.servlet" % "servlet-api" % "2.5" % "compile" - - lazy val sjson = "sjson.json" % "sjson" % "0.7-2.8.0" % "compile" + lazy val sjson = "sjson.json" % "sjson" % "0.8-2.8.0" % "compile" lazy val slf4j = "org.slf4j" % "slf4j-api" % SLF4J_VERSION % "compile" @@ -428,13 +428,13 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { val jetty = Dependencies.jetty val jetty_util = Dependencies.jetty_util val jetty_xml = Dependencies.jetty_xml + val jetty_servlet = Dependencies.jetty_servlet val jackson_core_asl = Dependencies.jackson_core_asl val jersey = Dependencies.jersey val jersey_contrib = Dependencies.jersey_contrib val jersey_json = Dependencies.jersey_json val jersey_server = Dependencies.jersey_server val jsr311 = Dependencies.jsr311 - val servlet = Dependencies.servlet val stax_api = Dependencies.stax_api // testing @@ -492,6 +492,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { class AkkaMongoProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) { val mongo = Dependencies.mongo + val casbah = Dependencies.casbah override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil } @@ -567,7 +568,6 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { // Provided by other bundles "!se.scalablesolutions.akka.*", - "!net.liftweb.*", "!com.google.inject.*", "!javax.transaction.*", "!javax.ws.rs.*", @@ -589,12 +589,6 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { // Scala bundle val scala_bundle = "com.weiglewilczek.scala-lang-osgi" % "scala-library" % buildScalaVersion % "compile" intransitive - // Lift bundles -// val lift_util = Dependencies.lift_util.intransitive -// val lift_actor = "net.liftweb" % "lift-actor" % LIFT_VERSION % "compile" intransitive -// val lift_common = "net.liftweb" % "lift-common" % LIFT_VERSION % "compile" intransitive -// val lift_json = "net.liftweb" % "lift-json" % LIFT_VERSION % "compile" intransitive - // Camel bundles val camel_core = Dependencies.camel_core.intransitive val fusesource_commonman = "org.fusesource.commonman" % "commons-management" % "1.0" intransitive @@ -659,21 +653,6 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { class AkkaSamplePubSubProject(info: ProjectInfo) extends AkkaDefaultProject(info, deployPath) class AkkaSampleFSMProject(info: ProjectInfo) extends AkkaDefaultProject(info, deployPath) - class AkkaSampleLiftProject(info: ProjectInfo) extends DefaultWebProject(info) with DeployProject { - //val commons_logging = Dependencies.commons_logging - val lift_util = Dependencies.lift_util - val lift_webkit = Dependencies.lift_webkit - val servlet = Dependencies.servlet - - // testing - val testJetty = Dependencies.testJetty - val testJettyWebApp = Dependencies.testJettyWebApp - val junit = Dependencies.junit - - def deployPath = AkkaParentProject.this.deployPath - override def jarPath = warPath - } - class AkkaSampleRestJavaProject(info: ProjectInfo) extends AkkaDefaultProject(info, deployPath) class AkkaSampleRemoteProject(info: ProjectInfo) extends AkkaDefaultProject(info, deployPath) @@ -721,8 +700,6 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { new AkkaSamplePubSubProject(_), akka_kernel) lazy val akka_sample_fsm = project("akka-sample-fsm", "akka-sample-fsm", new AkkaSampleFSMProject(_), akka_kernel) - lazy val akka_sample_lift = project("akka-sample-lift", "akka-sample-lift", - new AkkaSampleLiftProject(_), akka_kernel) lazy val akka_sample_rest_java = project("akka-sample-rest-java", "akka-sample-rest-java", new AkkaSampleRestJavaProject(_), akka_kernel) lazy val akka_sample_rest_scala = project("akka-sample-rest-scala", "akka-sample-rest-scala",