diff --git a/akka-actor/src/main/scala/actor/Actor.scala b/akka-actor/src/main/scala/actor/Actor.scala index 8087ecb889..943854fc61 100644 --- a/akka-actor/src/main/scala/actor/Actor.scala +++ b/akka-actor/src/main/scala/actor/Actor.scala @@ -2,20 +2,20 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor -import se.scalablesolutions.akka.dispatch._ -import se.scalablesolutions.akka.config.Config._ -import se.scalablesolutions.akka.config.Supervision._ -import se.scalablesolutions.akka.util.Helpers.{narrow, narrowSilently} -import se.scalablesolutions.akka.AkkaException +import akka.dispatch._ +import akka.config.Config._ +import akka.config.Supervision._ +import akka.util.Helpers.{narrow, narrowSilently} +import akka.AkkaException import java.util.concurrent.TimeUnit import java.net.InetSocketAddress import scala.reflect.BeanProperty -import se.scalablesolutions.akka.util. {ReflectiveAccess, Logging, Duration} -import se.scalablesolutions.akka.japi.Procedure +import akka.util. {ReflectiveAccess, Logging, Duration} +import akka.japi.Procedure /** * Implements the Transactor abstraction. E.g. a transactional actor. diff --git a/akka-actor/src/main/scala/actor/ActorRef.scala b/akka-actor/src/main/scala/actor/ActorRef.scala index b37dbc9670..b65fca33c1 100644 --- a/akka-actor/src/main/scala/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/actor/ActorRef.scala @@ -2,16 +2,16 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor -import se.scalablesolutions.akka.dispatch._ -import se.scalablesolutions.akka.config.Config._ -import se.scalablesolutions.akka.config.Supervision._ -import se.scalablesolutions.akka.stm.global._ -import se.scalablesolutions.akka.stm.TransactionManagement._ -import se.scalablesolutions.akka.stm.{ TransactionManagement, TransactionSetAbortedException } -import se.scalablesolutions.akka.AkkaException -import se.scalablesolutions.akka.util._ +import akka.dispatch._ +import akka.config.Config._ +import akka.config.Supervision._ +import akka.stm.global._ +import akka.stm.TransactionManagement._ +import akka.stm.{ TransactionManagement, TransactionSetAbortedException } +import akka.AkkaException +import akka.util._ import ReflectiveAccess._ import org.multiverse.api.ThreadLocalTransaction._ @@ -29,7 +29,7 @@ import scala.collection.immutable.Stack import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import annotation.tailrec -private[akka] object ActorRefInternals { +private[akka] object ActorRefInternals extends Logging { /** LifeCycles for ActorRefs */ @@ -77,7 +77,9 @@ private[akka] object ActorRefInternals { * * @author Jonas Bonér */ -trait ActorRef extends ActorRefShared with TransactionManagement with Logging with java.lang.Comparable[ActorRef] { scalaRef: ScalaActorRef => +trait ActorRef extends ActorRefShared with TransactionManagement with java.lang.Comparable[ActorRef] { scalaRef: ScalaActorRef => + //Reuse same logger + import Actor.log // Only mutable for RemoteServer in order to maintain identity across nodes @volatile @@ -156,7 +158,7 @@ trait ActorRef extends ActorRefShared with TransactionManagement with Logging wi * This means that all actors will share the same event-driven executor based dispatcher. *

* You can override it so it fits the specific use-case that the actor is used for. - * See the se.scalablesolutions.akka.dispatch.Dispatchers class for the different + * See the akka.dispatch.Dispatchers class for the different * dispatchers available. *

* The default is also that all actors that are created and spawned from within this actor diff --git a/akka-actor/src/main/scala/actor/ActorRegistry.scala b/akka-actor/src/main/scala/actor/ActorRegistry.scala index 41bff91132..f213021efd 100644 --- a/akka-actor/src/main/scala/actor/ActorRegistry.scala +++ b/akka-actor/src/main/scala/actor/ActorRegistry.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor import scala.collection.mutable.{ListBuffer, Map} import scala.reflect.Manifest @@ -11,8 +11,8 @@ import java.util.concurrent.{ConcurrentSkipListSet, ConcurrentHashMap} import java.util.{Set => JSet} import annotation.tailrec -import se.scalablesolutions.akka.util.ReflectiveAccess._ -import se.scalablesolutions.akka.util.{ReadWriteGuard, Address, ListenerManagement} +import akka.util.ReflectiveAccess._ +import akka.util.{ReadWriteGuard, Address, ListenerManagement} import java.net.InetSocketAddress /** diff --git a/akka-actor/src/main/scala/actor/Agent.scala b/akka-actor/src/main/scala/actor/Agent.scala index 00dceba21c..c9b91b3ca8 100644 --- a/akka-actor/src/main/scala/actor/Agent.scala +++ b/akka-actor/src/main/scala/actor/Agent.scala @@ -2,14 +2,14 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor -import se.scalablesolutions.akka.stm.Ref -import se.scalablesolutions.akka.AkkaException -import se.scalablesolutions.akka.japi.{ Function => JFunc, Procedure => JProc } +import akka.stm.Ref +import akka.AkkaException +import akka.japi.{ Function => JFunc, Procedure => JProc } import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.CountDownLatch -import se.scalablesolutions.akka.config.RemoteAddress +import akka.config.RemoteAddress class AgentException private[akka](message: String) extends AkkaException(message) diff --git a/akka-actor/src/main/scala/actor/BootableActorLoaderService.scala b/akka-actor/src/main/scala/actor/BootableActorLoaderService.scala index 278a75418e..6c58203de2 100644 --- a/akka-actor/src/main/scala/actor/BootableActorLoaderService.scala +++ b/akka-actor/src/main/scala/actor/BootableActorLoaderService.scala @@ -2,15 +2,15 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor import java.io.File import java.net.{URL, URLClassLoader} import java.util.jar.JarFile import java.util.Enumeration -import se.scalablesolutions.akka.util.{Bootable, Logging} -import se.scalablesolutions.akka.config.Config._ +import akka.util.{Bootable, Logging} +import akka.config.Config._ class AkkaDeployClassLoader(urls : List[URL], parent : ClassLoader) extends URLClassLoader(urls.toArray.asInstanceOf[Array[URL]],parent) { diff --git a/akka-actor/src/main/scala/actor/FSM.scala b/akka-actor/src/main/scala/actor/FSM.scala index eac861d358..df88db5ade 100644 --- a/akka-actor/src/main/scala/actor/FSM.scala +++ b/akka-actor/src/main/scala/actor/FSM.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor import scala.collection.mutable import java.util.concurrent.{ScheduledFuture, TimeUnit} @@ -13,11 +13,15 @@ trait FSM[S, D] { type StateFunction = scala.PartialFunction[Event, State] /** DSL */ - protected final def inState(stateName: S)(stateFunction: StateFunction) = { + protected final def notifying(transitionHandler: PartialFunction[Transition, Unit]) = { + transitionEvent = transitionHandler + } + + protected final def when(stateName: S)(stateFunction: StateFunction) = { register(stateName, stateFunction) } - protected final def setInitialState(stateName: S, stateData: D, timeout: Option[Long] = None) = { + protected final def startWith(stateName: S, stateData: D, timeout: Option[Long] = None) = { setState(State(stateName, stateData, timeout)) } @@ -74,6 +78,10 @@ trait FSM[S, D] { case reason => log.info("Stopping because of reason: %s", reason) } + private var transitionEvent: PartialFunction[Transition, Unit] = { + case Transition(from, to) => log.debug("Transitioning from state %s to %s", from, to) + } + override final protected def receive: Receive = { case Stop(reason, stateData) => terminateEvent.apply(reason) @@ -93,6 +101,9 @@ trait FSM[S, D] { if (!transitions.contains(nextState.stateName)) { stop(Failure("Next state %s does not exist".format(nextState.stateName))) } else { + if (currentState != null && currentState.stateName != nextState.stateName) { + transitionEvent.apply(Transition(currentState.stateName, nextState.stateName)) + } currentState = nextState currentState.timeout.foreach { t => @@ -128,6 +139,8 @@ trait FSM[S, D] { case class Failure(cause: Any) extends Reason case object StateTimeout - + + case class Transition(from: S, to: S) + private case class Stop(reason: Reason, stateData: D) } diff --git a/akka-actor/src/main/scala/actor/Implicits.scala b/akka-actor/src/main/scala/actor/Implicits.scala index 9992cd36a1..668d2d8876 100644 --- a/akka-actor/src/main/scala/actor/Implicits.scala +++ b/akka-actor/src/main/scala/actor/Implicits.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka +package akka import actor.{ScalaActorRef, ActorRef} diff --git a/akka-actor/src/main/scala/actor/Scheduler.scala b/akka-actor/src/main/scala/actor/Scheduler.scala index 6775479aa7..ae7f2193ee 100644 --- a/akka-actor/src/main/scala/actor/Scheduler.scala +++ b/akka-actor/src/main/scala/actor/Scheduler.scala @@ -13,14 +13,14 @@ * Rework of David Pollak's ActorPing class in the Lift Project * which is licensed under the Apache 2 License. */ -package se.scalablesolutions.akka.actor +package akka.actor import scala.collection.JavaConversions import java.util.concurrent._ -import se.scalablesolutions.akka.util.Logging -import se.scalablesolutions.akka.AkkaException +import akka.util.Logging +import akka.AkkaException object Scheduler extends Logging { import Actor._ diff --git a/akka-actor/src/main/scala/actor/Supervisor.scala b/akka-actor/src/main/scala/actor/Supervisor.scala index 0fd2e3ec89..cd1c85b019 100644 --- a/akka-actor/src/main/scala/actor/Supervisor.scala +++ b/akka-actor/src/main/scala/actor/Supervisor.scala @@ -2,17 +2,17 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor -import se.scalablesolutions.akka.config.Supervision._ -import se.scalablesolutions.akka.AkkaException -import se.scalablesolutions.akka.util._ +import akka.config.Supervision._ +import akka.AkkaException +import akka.util._ import ReflectiveAccess._ import Actor._ import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} import java.net.InetSocketAddress -import se.scalablesolutions.akka.config.Supervision._ +import akka.config.Supervision._ class SupervisorException private[akka](message: String) extends AkkaException(message) diff --git a/akka-actor/src/main/scala/actor/UntypedActor.scala b/akka-actor/src/main/scala/actor/UntypedActor.scala index 91b93b7a88..a3866fba92 100644 --- a/akka-actor/src/main/scala/actor/UntypedActor.scala +++ b/akka-actor/src/main/scala/actor/UntypedActor.scala @@ -2,16 +2,16 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor -import se.scalablesolutions.akka.dispatch._ -import se.scalablesolutions.akka.stm.global._ -import se.scalablesolutions.akka.config.Supervision._ +import akka.dispatch._ +import akka.stm.global._ +import akka.config.Supervision._ import java.net.InetSocketAddress import scala.reflect.BeanProperty -import se.scalablesolutions.akka.japi.Procedure +import akka.japi.Procedure /** * Subclass this abstract class to create a MDB-style untyped actor. diff --git a/akka-actor/src/main/scala/config/Config.scala b/akka-actor/src/main/scala/config/Config.scala index 16ed6d76c4..b88a222279 100644 --- a/akka-actor/src/main/scala/config/Config.scala +++ b/akka-actor/src/main/scala/config/Config.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.config +package akka.config -import se.scalablesolutions.akka.AkkaException -import se.scalablesolutions.akka.util.Logging -import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException} -import se.scalablesolutions.akka.dispatch.CompletableFuture +import akka.AkkaException +import akka.util.Logging +import akka.actor.{ActorRef, IllegalActorStateException} +import akka.dispatch.CompletableFuture import net.lag.configgy.{Config => CConfig, Configgy, ParseException} diff --git a/akka-actor/src/main/scala/config/Configuration.scala b/akka-actor/src/main/scala/config/Configuration.scala index e257c739a9..7e0cb406d4 100644 --- a/akka-actor/src/main/scala/config/Configuration.scala +++ b/akka-actor/src/main/scala/config/Configuration.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.config +package akka.config /* -import se.scalablesolutions.akka.kernel.{TypedActor, TypedActorProxy} +import akka.kernel.{TypedActor, TypedActorProxy} import com.google.inject.{AbstractModule} import java.util.{List => JList, ArrayList} import scala.reflect.BeanProperty @@ -18,43 +18,43 @@ import scala.reflect.BeanProperty sealed abstract class Configuration class RestartStrategy(@BeanProperty val scheme: FailOverScheme, @BeanProperty val maxNrOfRetries: Int, @BeanProperty val withinTimeRange: Int) extends Configuration { - def transform = se.scalablesolutions.akka.kernel.RestartStrategy(scheme.transform, maxNrOfRetries, withinTimeRange) + def transform = akka.kernel.RestartStrategy(scheme.transform, maxNrOfRetries, withinTimeRange) } class LifeCycle(@BeanProperty val scope: Scope, @BeanProperty val shutdownTime: Int) extends Configuration { - def transform = se.scalablesolutions.akka.kernel.LifeCycle(scope.transform, shutdownTime) + def transform = akka.kernel.LifeCycle(scope.transform, shutdownTime) } abstract class Scope extends Configuration { - def transform: se.scalablesolutions.akka.kernel.Scope + def transform: akka.kernel.Scope } class Permanent extends Scope { - override def transform = se.scalablesolutions.akka.kernel.Permanent + override def transform = akka.kernel.Permanent } class Transient extends Scope { - override def transform = se.scalablesolutions.akka.kernel.Transient + override def transform = akka.kernel.Transient } class Temporary extends Scope { - override def transform = se.scalablesolutions.akka.kernel.Temporary + override def transform = akka.kernel.Temporary } abstract class FailOverScheme extends Configuration { - def transform: se.scalablesolutions.akka.kernel.FailOverScheme + def transform: akka.kernel.FailOverScheme } class AllForOne extends FailOverScheme { - override def transform = se.scalablesolutions.akka.kernel.AllForOne + override def transform = akka.kernel.AllForOne } class OneForOne extends FailOverScheme { - override def transform = se.scalablesolutions.akka.kernel.OneForOne + override def transform = akka.kernel.OneForOne } abstract class Server extends Configuration //class kernelConfig(@BeanProperty val restartStrategy: RestartStrategy, @BeanProperty val servers: JList[Server]) extends Server { -// def transform = se.scalablesolutions.akka.kernel.kernelConfig(restartStrategy.transform, servers.toArray.toList.asInstanceOf[List[Server]].map(_.transform)) +// def transform = akka.kernel.kernelConfig(restartStrategy.transform, servers.toArray.toList.asInstanceOf[List[Server]].map(_.transform)) //} class Component(@BeanProperty val intf: Class[_], @BeanProperty val target: Class[_], @BeanProperty val lifeCycle: LifeCycle, @BeanProperty val timeout: Int) extends Server { - def newWorker(proxy: TypedActorProxy) = se.scalablesolutions.akka.kernel.Supervise(proxy.server, lifeCycle.transform) + def newWorker(proxy: TypedActorProxy) = akka.kernel.Supervise(proxy.server, lifeCycle.transform) } */ diff --git a/akka-actor/src/main/scala/config/Configurator.scala b/akka-actor/src/main/scala/config/Configurator.scala index 8560649018..f63c96b065 100644 --- a/akka-actor/src/main/scala/config/Configurator.scala +++ b/akka-actor/src/main/scala/config/Configurator.scala @@ -2,9 +2,9 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.config +package akka.config -import se.scalablesolutions.akka.config.Supervision. {SuperviseTypedActor, FaultHandlingStrategy} +import akka.config.Supervision. {SuperviseTypedActor, FaultHandlingStrategy} private[akka] trait TypedActorConfiguratorBase { def getExternalDependency[T](clazz: Class[T]): T diff --git a/akka-actor/src/main/scala/config/SupervisionConfig.scala b/akka-actor/src/main/scala/config/SupervisionConfig.scala index 2e72271830..b4ae9ddf51 100644 --- a/akka-actor/src/main/scala/config/SupervisionConfig.scala +++ b/akka-actor/src/main/scala/config/SupervisionConfig.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.config +package akka.config -import se.scalablesolutions.akka.actor.{ActorRef} -import se.scalablesolutions.akka.dispatch.MessageDispatcher +import akka.actor.{ActorRef} +import akka.dispatch.MessageDispatcher case class RemoteAddress(val hostname: String, val port: Int) diff --git a/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala b/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala index 56face4b6b..3d0dadad11 100644 --- a/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala +++ b/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala @@ -2,16 +2,16 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.dataflow +package akka.dataflow import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.{ConcurrentLinkedQueue, LinkedBlockingQueue} -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.japi.{ Function, SideEffect } +import akka.actor.{Actor, ActorRef} +import akka.actor.Actor._ +import akka.dispatch.CompletableFuture +import akka.AkkaException +import akka.japi.{ Function, SideEffect } /** * Implements Oz-style dataflow (single assignment) variables. diff --git a/akka-actor/src/main/scala/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/dispatch/Dispatchers.scala index cd06fd8857..925f856d3f 100644 --- a/akka-actor/src/main/scala/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/dispatch/Dispatchers.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.dispatch +package akka.dispatch -import se.scalablesolutions.akka.actor.{Actor, ActorRef} -import se.scalablesolutions.akka.config.Config._ -import se.scalablesolutions.akka.util.{Duration, Logging} -import se.scalablesolutions.akka.actor.newUuid +import akka.actor.{Actor, ActorRef} +import akka.config.Config._ +import akka.util.{Duration, Logging} +import akka.actor.newUuid import net.lag.configgy.ConfigMap @@ -67,7 +67,7 @@ object Dispatchers extends Logging { *

* Can be beneficial to use the HawtDispatcher.pin(self) to "pin" an actor to a specific thread. *

- * See the ScalaDoc for the {@link se.scalablesolutions.akka.dispatch.HawtDispatcher} for details. + * See the ScalaDoc for the {@link akka.dispatch.HawtDispatcher} for details. */ def newHawtDispatcher(aggregate: Boolean) = new HawtDispatcher(aggregate) diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala index 3622984e69..9ea0acb94c 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala @@ -2,13 +2,13 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.dispatch +package akka.dispatch -import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException} -import se.scalablesolutions.akka.util.ReflectiveAccess.EnterpriseModule +import akka.actor.{ActorRef, IllegalActorStateException} +import akka.util.ReflectiveAccess.EnterpriseModule import java.util.Queue -import se.scalablesolutions.akka.util.Switch +import akka.util.Switch import java.util.concurrent.atomic.AtomicReference import java.util.concurrent. {ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue, LinkedBlockingQueue} @@ -57,7 +57,7 @@ import java.util.concurrent. {ExecutorService, RejectedExecutionException, Concu *

* * But the preferred way of creating dispatchers is to use - * the {@link se.scalablesolutions.akka.dispatch.Dispatchers} factory object. + * the {@link akka.dispatch.Dispatchers} factory object. * * @author Jonas Bonér * @param throughput positive integer indicates the dispatcher will only process so much messages at a time from the diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala index 9e95384cf5..306d59b865 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.dispatch +package akka.dispatch import jsr166x.{Deque, ConcurrentLinkedDeque, LinkedBlockingDeque} -import se.scalablesolutions.akka.actor.{Actor, ActorRef, IllegalActorStateException} -import se.scalablesolutions.akka.util.Switch +import akka.actor.{Actor, ActorRef, IllegalActorStateException} +import akka.util.Switch import java.util.concurrent. {ExecutorService, CopyOnWriteArrayList} import java.util.concurrent.atomic.AtomicReference @@ -24,10 +24,10 @@ import java.util.concurrent.atomic.AtomicReference * TODO: it would be nice to be able to redistribute work even when no new messages are being dispatched, without impacting dispatching performance ?! *

* The preferred way of creating dispatchers is to use - * the {@link se.scalablesolutions.akka.dispatch.Dispatchers} factory object. + * the {@link akka.dispatch.Dispatchers} factory object. * - * @see se.scalablesolutions.akka.dispatch.ExecutorBasedEventDrivenWorkStealingDispatcher - * @see se.scalablesolutions.akka.dispatch.Dispatchers + * @see akka.dispatch.ExecutorBasedEventDrivenWorkStealingDispatcher + * @see akka.dispatch.Dispatchers * * @author Jan Van Besien */ diff --git a/akka-actor/src/main/scala/dispatch/Future.scala b/akka-actor/src/main/scala/dispatch/Future.scala index 57ea36d5d0..873c5e9f55 100644 --- a/akka-actor/src/main/scala/dispatch/Future.scala +++ b/akka-actor/src/main/scala/dispatch/Future.scala @@ -2,13 +2,13 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.dispatch +package akka.dispatch -import se.scalablesolutions.akka.AkkaException -import se.scalablesolutions.akka.actor.Actor.spawn +import akka.AkkaException +import akka.actor.Actor.spawn import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.TimeUnit -import se.scalablesolutions.akka.routing.Dispatcher +import akka.routing.Dispatcher class FutureTimeoutException(message: String) extends AkkaException(message) diff --git a/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala b/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala index 8f001084b7..211dca50be 100644 --- a/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala @@ -2,9 +2,9 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.dispatch +package akka.dispatch -import se.scalablesolutions.akka.actor.ActorRef +import akka.actor.ActorRef import org.fusesource.hawtdispatch.DispatchQueue import org.fusesource.hawtdispatch.ScalaDispatch._ @@ -13,7 +13,7 @@ import org.fusesource.hawtdispatch.ListEventAggregator import java.util.concurrent.atomic.{AtomicInteger, AtomicBoolean} import java.util.concurrent.CountDownLatch -import se.scalablesolutions.akka.util.Switch +import akka.util.Switch /** * Holds helper methods for working with actors that are using a HawtDispatcher as it's dispatcher. diff --git a/akka-actor/src/main/scala/dispatch/MailboxHandling.scala b/akka-actor/src/main/scala/dispatch/MailboxHandling.scala index d363e76382..102bc19615 100644 --- a/akka-actor/src/main/scala/dispatch/MailboxHandling.scala +++ b/akka-actor/src/main/scala/dispatch/MailboxHandling.scala @@ -2,16 +2,16 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.dispatch +package akka.dispatch -import se.scalablesolutions.akka.actor.{Actor, ActorType, ActorRef, ActorInitializationException} -import se.scalablesolutions.akka.util.ReflectiveAccess.EnterpriseModule -import se.scalablesolutions.akka.AkkaException +import akka.actor.{Actor, ActorType, ActorRef, ActorInitializationException} +import akka.util.ReflectiveAccess.EnterpriseModule +import akka.AkkaException import java.util.{Queue, List} import java.util.concurrent._ import concurrent.forkjoin.LinkedTransferQueue -import se.scalablesolutions.akka.util._ +import akka.util._ class MessageQueueAppendFailedException(message: String) extends AkkaException(message) diff --git a/akka-actor/src/main/scala/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/dispatch/MessageHandling.scala index 9465d79897..9978971377 100644 --- a/akka-actor/src/main/scala/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/dispatch/MessageHandling.scala @@ -2,14 +2,14 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.dispatch +package akka.dispatch import org.multiverse.commitbarriers.CountDownCommitBarrier import java.util.concurrent._ import atomic. {AtomicInteger, AtomicBoolean, AtomicReference, AtomicLong} -import se.scalablesolutions.akka.util. {Switch, ReentrantGuard, Logging, HashCode} -import se.scalablesolutions.akka.actor._ +import akka.util. {Switch, ReentrantGuard, Logging, HashCode} +import akka.actor._ /** * @author Jonas Bonér diff --git a/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala b/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala index 8ec4421c3e..ab7541e65b 100644 --- a/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala @@ -2,15 +2,15 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.dispatch +package akka.dispatch -import se.scalablesolutions.akka.actor.{Actor, ActorRef} -import se.scalablesolutions.akka.config.Config.config -import se.scalablesolutions.akka.util.Duration +import akka.actor.{Actor, ActorRef} +import akka.config.Config.config +import akka.util.Duration import java.util.Queue import java.util.concurrent.{ConcurrentLinkedQueue, BlockingQueue, TimeUnit, LinkedBlockingQueue} -import se.scalablesolutions.akka.actor +import akka.actor import java.util.concurrent.atomic.AtomicReference /** diff --git a/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala index 0e1a1ceafb..2f89cab81e 100644 --- a/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala @@ -2,14 +2,14 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.dispatch +package akka.dispatch import java.util.Collection import java.util.concurrent._ import atomic.{AtomicLong, AtomicInteger} import ThreadPoolExecutor.CallerRunsPolicy -import se.scalablesolutions.akka.util. {Duration, Logging} +import akka.util. {Duration, Logging} object ThreadPoolConfig { type Bounds = Int diff --git a/akka-actor/src/main/scala/japi/JavaAPI.scala b/akka-actor/src/main/scala/japi/JavaAPI.scala index 7e79fe8184..9f79fb7fc3 100644 --- a/akka-actor/src/main/scala/japi/JavaAPI.scala +++ b/akka-actor/src/main/scala/japi/JavaAPI.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.japi +package akka.japi /** * A Function interface. Used to create first-class-functions is Java (sort of). diff --git a/akka-actor/src/main/scala/routing/Iterators.scala b/akka-actor/src/main/scala/routing/Iterators.scala index 7f21589cfa..9b0d6b9742 100644 --- a/akka-actor/src/main/scala/routing/Iterators.scala +++ b/akka-actor/src/main/scala/routing/Iterators.scala @@ -2,9 +2,9 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.routing +package akka.routing -import se.scalablesolutions.akka.actor.ActorRef +import akka.actor.ActorRef import scala.collection.JavaConversions._ /** diff --git a/akka-actor/src/main/scala/routing/Listeners.scala b/akka-actor/src/main/scala/routing/Listeners.scala index 6531aee422..c80ff21353 100644 --- a/akka-actor/src/main/scala/routing/Listeners.scala +++ b/akka-actor/src/main/scala/routing/Listeners.scala @@ -2,9 +2,9 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.routing +package akka.routing -import se.scalablesolutions.akka.actor.{Actor, ActorRef} +import akka.actor.{Actor, ActorRef} import java.util.concurrent.ConcurrentSkipListSet import scala.collection.JavaConversions._ diff --git a/akka-actor/src/main/scala/routing/Routers.scala b/akka-actor/src/main/scala/routing/Routers.scala index d916e6c584..428141251b 100644 --- a/akka-actor/src/main/scala/routing/Routers.scala +++ b/akka-actor/src/main/scala/routing/Routers.scala @@ -2,9 +2,9 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.routing +package akka.routing -import se.scalablesolutions.akka.actor.{UntypedActor, Actor, ActorRef} +import akka.actor.{UntypedActor, Actor, ActorRef} /** * A Dispatcher is a trait whose purpose is to route incoming messages to actors. diff --git a/akka-actor/src/main/scala/routing/Routing.scala b/akka-actor/src/main/scala/routing/Routing.scala index a5242c72b9..4bbb51ddab 100644 --- a/akka-actor/src/main/scala/routing/Routing.scala +++ b/akka-actor/src/main/scala/routing/Routing.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.routing +package akka.routing -import se.scalablesolutions.akka.actor.{Actor, ActorRef} -import se.scalablesolutions.akka.actor.Actor._ +import akka.actor.{Actor, ActorRef} +import akka.actor.Actor._ object Routing { diff --git a/akka-actor/src/main/scala/stm/Ref.scala b/akka-actor/src/main/scala/stm/Ref.scala index fc81dbafa9..d76fd67797 100644 --- a/akka-actor/src/main/scala/stm/Ref.scala +++ b/akka-actor/src/main/scala/stm/Ref.scala @@ -2,9 +2,9 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.stm +package akka.stm -import se.scalablesolutions.akka.actor.{newUuid, Uuid} +import akka.actor.{newUuid, Uuid} import org.multiverse.transactional.refs.BasicRef diff --git a/akka-actor/src/main/scala/stm/Transaction.scala b/akka-actor/src/main/scala/stm/Transaction.scala index 9ea32d7ca6..c5531ea614 100644 --- a/akka-actor/src/main/scala/stm/Transaction.scala +++ b/akka-actor/src/main/scala/stm/Transaction.scala @@ -2,18 +2,18 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.stm +package akka.stm import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.HashMap -import se.scalablesolutions.akka.util.ReflectiveAccess.JtaModule +import akka.util.ReflectiveAccess.JtaModule -import se.scalablesolutions.akka.util.Logging -import se.scalablesolutions.akka.config.Config._ -import se.scalablesolutions.akka.AkkaException +import akka.util.Logging +import akka.config.Config._ +import akka.AkkaException import org.multiverse.api.{Transaction => MultiverseTransaction} import org.multiverse.api.lifecycle.{TransactionLifecycleListener, TransactionLifecycleEvent} @@ -28,10 +28,10 @@ class StmConfigurationException(message: String) extends AkkaException(message) object Transaction { val idFactory = new AtomicLong(-1L) - @deprecated("Use the se.scalablesolutions.akka.stm.local package object instead.") + @deprecated("Use the akka.stm.local package object instead.") object Local extends LocalStm - @deprecated("Use the se.scalablesolutions.akka.stm.global package object instead.") + @deprecated("Use the akka.stm.global package object instead.") object Global extends GlobalStm object Util extends StmUtil diff --git a/akka-actor/src/main/scala/stm/TransactionFactory.scala b/akka-actor/src/main/scala/stm/TransactionFactory.scala index 178032a9c0..befa097822 100644 --- a/akka-actor/src/main/scala/stm/TransactionFactory.scala +++ b/akka-actor/src/main/scala/stm/TransactionFactory.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.stm +package akka.stm import java.lang.{Boolean => JBoolean} -import se.scalablesolutions.akka.config.Config._ -import se.scalablesolutions.akka.util.Duration +import akka.config.Config._ +import akka.util.Duration import org.multiverse.api.GlobalStmInstance.getGlobalStmInstance import org.multiverse.stms.alpha.AlphaStm diff --git a/akka-actor/src/main/scala/stm/TransactionFactoryBuilder.scala b/akka-actor/src/main/scala/stm/TransactionFactoryBuilder.scala index 253ead0153..dc44461631 100644 --- a/akka-actor/src/main/scala/stm/TransactionFactoryBuilder.scala +++ b/akka-actor/src/main/scala/stm/TransactionFactoryBuilder.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.stm +package akka.stm import java.lang.{Boolean => JBoolean} -import se.scalablesolutions.akka.util.Duration +import akka.util.Duration import org.multiverse.api.TraceLevel import org.multiverse.api.{PropagationLevel => Propagation} diff --git a/akka-actor/src/main/scala/stm/TransactionManagement.scala b/akka-actor/src/main/scala/stm/TransactionManagement.scala index 4e3d995c11..c6f29c866b 100644 --- a/akka-actor/src/main/scala/stm/TransactionManagement.scala +++ b/akka-actor/src/main/scala/stm/TransactionManagement.scala @@ -2,9 +2,9 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.stm +package akka.stm -import se.scalablesolutions.akka.AkkaException +import akka.AkkaException import org.multiverse.api.{StmUtils => MultiverseStmUtils} import org.multiverse.api.ThreadLocalTransaction._ @@ -18,7 +18,7 @@ class TransactionSetAbortedException(msg: String) extends AkkaException(msg) * Internal helper methods and properties for transaction management. */ object TransactionManagement extends TransactionManagement { - import se.scalablesolutions.akka.config.Config._ + import akka.config.Config._ // FIXME move to stm.global.fair? val FAIR_TRANSACTIONS = config.getBool("akka.stm.fair", true) @@ -126,16 +126,16 @@ trait StmUtil { } trait StmCommon { - type TransactionConfig = se.scalablesolutions.akka.stm.TransactionConfig - val TransactionConfig = se.scalablesolutions.akka.stm.TransactionConfig + type TransactionConfig = akka.stm.TransactionConfig + val TransactionConfig = akka.stm.TransactionConfig - type TransactionFactory = se.scalablesolutions.akka.stm.TransactionFactory - val TransactionFactory = se.scalablesolutions.akka.stm.TransactionFactory + type TransactionFactory = akka.stm.TransactionFactory + val TransactionFactory = akka.stm.TransactionFactory - val Propagation = se.scalablesolutions.akka.stm.Transaction.Propagation + val Propagation = akka.stm.Transaction.Propagation - val TraceLevel = se.scalablesolutions.akka.stm.Transaction.TraceLevel + val TraceLevel = akka.stm.Transaction.TraceLevel - type Ref[T] = se.scalablesolutions.akka.stm.Ref[T] - val Ref = se.scalablesolutions.akka.stm.Ref + type Ref[T] = akka.stm.Ref[T] + val Ref = akka.stm.Ref } diff --git a/akka-actor/src/main/scala/stm/TransactionalMap.scala b/akka-actor/src/main/scala/stm/TransactionalMap.scala index 457774f755..db42caaf9f 100644 --- a/akka-actor/src/main/scala/stm/TransactionalMap.scala +++ b/akka-actor/src/main/scala/stm/TransactionalMap.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.stm +package akka.stm import scala.collection.immutable.HashMap -import se.scalablesolutions.akka.actor.{newUuid} +import akka.actor.{newUuid} import org.multiverse.api.ThreadLocalTransaction.getThreadLocalTransaction diff --git a/akka-actor/src/main/scala/stm/TransactionalVector.scala b/akka-actor/src/main/scala/stm/TransactionalVector.scala index 3004e97db8..963568af6c 100644 --- a/akka-actor/src/main/scala/stm/TransactionalVector.scala +++ b/akka-actor/src/main/scala/stm/TransactionalVector.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.stm +package akka.stm import scala.collection.immutable.Vector -import se.scalablesolutions.akka.actor.newUuid +import akka.actor.newUuid import org.multiverse.api.ThreadLocalTransaction.getThreadLocalTransaction diff --git a/akka-actor/src/main/scala/stm/global/Atomic.scala b/akka-actor/src/main/scala/stm/global/Atomic.scala index d5a92fe047..298c58d63d 100644 --- a/akka-actor/src/main/scala/stm/global/Atomic.scala +++ b/akka-actor/src/main/scala/stm/global/Atomic.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.stm.global +package akka.stm.global /** * Java-friendly atomic blocks. @@ -10,8 +10,8 @@ package se.scalablesolutions.akka.stm.global * Example usage (in Java): *

*

- * import se.scalablesolutions.akka.stm.*;
- * import se.scalablesolutions.akka.stm.global.Atomic;
+ * import akka.stm.*;
+ * import akka.stm.global.Atomic;
  *
  * final Ref ref = new Ref(0);
  *
diff --git a/akka-actor/src/main/scala/stm/global/GlobalStm.scala b/akka-actor/src/main/scala/stm/global/GlobalStm.scala
index f2dfce8a96..4ff66c7761 100644
--- a/akka-actor/src/main/scala/stm/global/GlobalStm.scala
+++ b/akka-actor/src/main/scala/stm/global/GlobalStm.scala
@@ -2,9 +2,9 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.stm
+package akka.stm
 
-import se.scalablesolutions.akka.util.Logging
+import akka.util.Logging
 
 import org.multiverse.api.{Transaction => MultiverseTransaction}
 import org.multiverse.templates.TransactionalCallable
@@ -18,7 +18,7 @@ object GlobalStm extends Logging
  * Example of atomic transaction management using the atomic block:
  * 

*

- * import se.scalablesolutions.akka.stm.global._
+ * import akka.stm.global._
  *
  * atomic  {
  *   // do something within a transaction
diff --git a/akka-actor/src/main/scala/stm/global/package.scala b/akka-actor/src/main/scala/stm/global/package.scala
index 9b8a1b289e..cf4b24bf8a 100644
--- a/akka-actor/src/main/scala/stm/global/package.scala
+++ b/akka-actor/src/main/scala/stm/global/package.scala
@@ -2,7 +2,7 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.stm
+package akka.stm
 
 /**
  * For easily importing global STM.
diff --git a/akka-actor/src/main/scala/stm/local/Atomic.scala b/akka-actor/src/main/scala/stm/local/Atomic.scala
index c06f99ba99..c4929fe57c 100644
--- a/akka-actor/src/main/scala/stm/local/Atomic.scala
+++ b/akka-actor/src/main/scala/stm/local/Atomic.scala
@@ -2,7 +2,7 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.stm.local
+package akka.stm.local
 
 /**
  * Java-friendly atomic blocks.
@@ -10,8 +10,8 @@ package se.scalablesolutions.akka.stm.local
  * Example usage (in Java):
  * 

*

- * import se.scalablesolutions.akka.stm.*;
- * import se.scalablesolutions.akka.stm.local.Atomic;
+ * import akka.stm.*;
+ * import akka.stm.local.Atomic;
  *
  * final Ref ref = new Ref(0);
  *
diff --git a/akka-actor/src/main/scala/stm/local/LocalStm.scala b/akka-actor/src/main/scala/stm/local/LocalStm.scala
index f0e60206f6..5048a745aa 100644
--- a/akka-actor/src/main/scala/stm/local/LocalStm.scala
+++ b/akka-actor/src/main/scala/stm/local/LocalStm.scala
@@ -2,9 +2,9 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.stm
+package akka.stm
 
-import se.scalablesolutions.akka.util.Logging
+import akka.util.Logging
 
 import org.multiverse.api.{Transaction => MultiverseTransaction}
 import org.multiverse.templates.TransactionalCallable
@@ -19,7 +19,7 @@ object LocalStm extends Logging
  * Example of atomic transaction management using the atomic block.
  * 

*

- * import se.scalablesolutions.akka.stm.local._
+ * import akka.stm.local._
  *
  * atomic  {
  *   // do something within a transaction
diff --git a/akka-actor/src/main/scala/stm/local/package.scala b/akka-actor/src/main/scala/stm/local/package.scala
index 406d4880f6..646e63e8fe 100644
--- a/akka-actor/src/main/scala/stm/local/package.scala
+++ b/akka-actor/src/main/scala/stm/local/package.scala
@@ -2,7 +2,7 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.stm
+package akka.stm
 
 /**
  * For easily importing local STM.
diff --git a/akka-actor/src/main/scala/stm/transactional.scala b/akka-actor/src/main/scala/stm/transactional.scala
index e00c7ef8e9..074b1308d8 100644
--- a/akka-actor/src/main/scala/stm/transactional.scala
+++ b/akka-actor/src/main/scala/stm/transactional.scala
@@ -2,18 +2,18 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.stm
+package akka.stm
 
 /**
  * For importing the transactional datastructures, including the primitive refs
  * and transactional data structures from Multiverse.
  */
 package object transactional {
-  type TransactionalMap[K,V] = se.scalablesolutions.akka.stm.TransactionalMap[K,V]
-  val TransactionalMap =  se.scalablesolutions.akka.stm.TransactionalMap
+  type TransactionalMap[K,V] = akka.stm.TransactionalMap[K,V]
+  val TransactionalMap =  akka.stm.TransactionalMap
 
-  type TransactionalVector[T] = se.scalablesolutions.akka.stm.TransactionalVector[T]
-  val TransactionalVector = se.scalablesolutions.akka.stm.TransactionalVector
+  type TransactionalVector[T] = akka.stm.TransactionalVector[T]
+  val TransactionalVector = akka.stm.TransactionalVector
 
   type BooleanRef = org.multiverse.transactional.refs.BooleanRef
   type ByteRef    = org.multiverse.transactional.refs.ByteRef
diff --git a/akka-actor/src/main/scala/util/Address.scala b/akka-actor/src/main/scala/util/Address.scala
index 34c3f51bd4..ee83896312 100644
--- a/akka-actor/src/main/scala/util/Address.scala
+++ b/akka-actor/src/main/scala/util/Address.scala
@@ -1,7 +1,7 @@
 /**
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
-package se.scalablesolutions.akka.util
+package akka.util
 
 object Address {
   def apply(hostname: String, port: Int) = new Address(hostname, port)
diff --git a/akka-actor/src/main/scala/util/AkkaException.scala b/akka-actor/src/main/scala/util/AkkaException.scala
index 0eb06549b4..3ab80a2512 100644
--- a/akka-actor/src/main/scala/util/AkkaException.scala
+++ b/akka-actor/src/main/scala/util/AkkaException.scala
@@ -2,10 +2,10 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka
+package akka
 
-import se.scalablesolutions.akka.util.Logging
-import se.scalablesolutions.akka.actor.newUuid
+import akka.util.Logging
+import akka.actor.newUuid
 
 import java.io.{StringWriter, PrintWriter}
 import java.net.{InetAddress, UnknownHostException}
diff --git a/akka-actor/src/main/scala/util/Bootable.scala b/akka-actor/src/main/scala/util/Bootable.scala
index 4a8e086914..ef1f750e93 100644
--- a/akka-actor/src/main/scala/util/Bootable.scala
+++ b/akka-actor/src/main/scala/util/Bootable.scala
@@ -2,7 +2,7 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.util
+package akka.util
 
 trait Bootable {
   def onLoad {}
diff --git a/akka-actor/src/main/scala/util/Crypt.scala b/akka-actor/src/main/scala/util/Crypt.scala
index fad9e4c281..65f62f3f27 100644
--- a/akka-actor/src/main/scala/util/Crypt.scala
+++ b/akka-actor/src/main/scala/util/Crypt.scala
@@ -2,7 +2,7 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.util
+package akka.util
 
 import java.security.{MessageDigest, SecureRandom}
 
diff --git a/akka-actor/src/main/scala/util/Duration.scala b/akka-actor/src/main/scala/util/Duration.scala
index 0dee2fc139..4b8c6e7f9a 100644
--- a/akka-actor/src/main/scala/util/Duration.scala
+++ b/akka-actor/src/main/scala/util/Duration.scala
@@ -2,7 +2,7 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.util
+package akka.util
 
 import java.util.concurrent.TimeUnit
 
@@ -24,7 +24,7 @@ object Duration {
  * 

* Examples of usage from Java: *

- * import se.scalablesolutions.akka.util.Duration;
+ * import akka.util.Duration;
  * import java.util.concurrent.TimeUnit;
  *
  * Duration duration = new Duration(100, TimeUnit.MILLISECONDS);
@@ -36,7 +36,7 @@ object Duration {
  * 

* Examples of usage from Scala: *

- * import se.scalablesolutions.akka.util.Duration
+ * import akka.util.Duration
  * import java.util.concurrent.TimeUnit
  *
  * val duration = Duration(100, TimeUnit.MILLISECONDS)
@@ -48,7 +48,7 @@ object Duration {
  * 

* Implicits are also provided for Int and Long. Example usage: *

- * import se.scalablesolutions.akka.util.duration._
+ * import akka.util.duration._
  *
  * val duration = 100.millis
  * 
diff --git a/akka-actor/src/main/scala/util/HashCode.scala b/akka-actor/src/main/scala/util/HashCode.scala index 83fea18489..e74f85d069 100644 --- a/akka-actor/src/main/scala/util/HashCode.scala +++ b/akka-actor/src/main/scala/util/HashCode.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.util +package akka.util import java.lang.reflect.{Array => JArray} import java.lang.{Float => JFloat, Double => JDouble} diff --git a/akka-actor/src/main/scala/util/Helpers.scala b/akka-actor/src/main/scala/util/Helpers.scala index bc468fff3c..e903f50556 100644 --- a/akka-actor/src/main/scala/util/Helpers.scala +++ b/akka-actor/src/main/scala/util/Helpers.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.util +package akka.util /** * @author Jonas Bonér @@ -97,4 +97,4 @@ object Helpers extends Logging { object ResultOrError { def apply[R](result: R) = new ResultOrError(result) } -} \ No newline at end of file +} diff --git a/akka-actor/src/main/scala/util/ListenerManagement.scala b/akka-actor/src/main/scala/util/ListenerManagement.scala index 10104e119d..9166680873 100644 --- a/akka-actor/src/main/scala/util/ListenerManagement.scala +++ b/akka-actor/src/main/scala/util/ListenerManagement.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.util +package akka.util import java.util.concurrent.ConcurrentSkipListSet -import se.scalablesolutions.akka.actor.ActorRef +import akka.actor.ActorRef /** * A manager for listener actors. Intended for mixin by observables. diff --git a/akka-actor/src/main/scala/util/LockUtil.scala b/akka-actor/src/main/scala/util/LockUtil.scala index d0a318aa02..1a90377ff1 100644 --- a/akka-actor/src/main/scala/util/LockUtil.scala +++ b/akka-actor/src/main/scala/util/LockUtil.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.util +package akka.util import java.util.concurrent.locks.{ReentrantReadWriteLock, ReentrantLock} import java.util.concurrent.atomic. {AtomicBoolean} diff --git a/akka-actor/src/main/scala/util/Logging.scala b/akka-actor/src/main/scala/util/Logging.scala index 8d2e64be58..5891475483 100644 --- a/akka-actor/src/main/scala/util/Logging.scala +++ b/akka-actor/src/main/scala/util/Logging.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.util +package akka.util import org.slf4j.{Logger => SLFLogger,LoggerFactory => SLFLoggerFactory} diff --git a/akka-actor/src/main/scala/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/util/ReflectiveAccess.scala index a7420da192..41283cbbae 100644 --- a/akka-actor/src/main/scala/util/ReflectiveAccess.scala +++ b/akka-actor/src/main/scala/util/ReflectiveAccess.scala @@ -2,13 +2,13 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.util +package akka.util -import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException, ActorType, Uuid} -import se.scalablesolutions.akka.dispatch.{Future, CompletableFuture, MessageInvocation} -import se.scalablesolutions.akka.config.{Config, ModuleNotAvailableException} -import se.scalablesolutions.akka.stm.Transaction -import se.scalablesolutions.akka.AkkaException +import akka.actor.{ActorRef, IllegalActorStateException, ActorType, Uuid} +import akka.dispatch.{Future, CompletableFuture, MessageInvocation} +import akka.config.{Config, ModuleNotAvailableException} +import akka.stm.Transaction +import akka.AkkaException import java.net.InetSocketAddress @@ -65,7 +65,7 @@ object ReflectiveAccess extends Logging { "Can't load the remoting module, make sure that akka-remote.jar is on the classpath") val remoteClientObjectInstance: Option[RemoteClientObject] = - getObjectFor("se.scalablesolutions.akka.remote.RemoteClient$") + getObjectFor("akka.remote.RemoteClient$") def register(address: InetSocketAddress, uuid: Uuid) = { ensureRemotingEnabled @@ -123,10 +123,10 @@ object ReflectiveAccess extends Logging { } val remoteServerObjectInstance: Option[RemoteServerObject] = - getObjectFor("se.scalablesolutions.akka.remote.RemoteServer$") + getObjectFor("akka.remote.RemoteServer$") val remoteNodeObjectInstance: Option[RemoteNodeObject] = - getObjectFor("se.scalablesolutions.akka.remote.RemoteNode$") + getObjectFor("akka.remote.RemoteNode$") def registerActor(address: InetSocketAddress, uuid: Uuid, actorRef: ActorRef) = { ensureRemotingEnabled @@ -165,7 +165,7 @@ object ReflectiveAccess extends Logging { "Can't load the typed actor module, make sure that akka-typed-actor.jar is on the classpath") val typedActorObjectInstance: Option[TypedActorObject] = - getObjectFor("se.scalablesolutions.akka.actor.TypedActor$") + getObjectFor("akka.actor.TypedActor$") def resolveFutureIfMessageIsJoinPoint(message: Any, future: Future[_]): Boolean = { ensureTypedActorEnabled @@ -194,7 +194,7 @@ object ReflectiveAccess extends Logging { "Can't load the typed actor module, make sure that akka-jta.jar is on the classpath") val transactionContainerObjectInstance: Option[TransactionContainerObject] = - getObjectFor("se.scalablesolutions.akka.jta.TransactionContainer$") + getObjectFor("akka.jta.TransactionContainer$") def createTransactionContainer: TransactionContainer = { ensureJtaEnabled @@ -217,21 +217,21 @@ object ReflectiveAccess extends Logging { lazy val isEnterpriseEnabled = clusterObjectInstance.isDefined val clusterObjectInstance: Option[AnyRef] = - getObjectFor("se.scalablesolutions.akka.cluster.Cluster$") + getObjectFor("akka.cluster.Cluster$") val serializerClass: Option[Class[_]] = - getClassFor("se.scalablesolutions.akka.serialization.Serializer") + getClassFor("akka.serialization.Serializer") def ensureEnterpriseEnabled = if (!isEnterpriseEnabled) throw new ModuleNotAvailableException( "Feature is only available in Akka Enterprise edition") - def createFileBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("se.scalablesolutions.akka.actor.mailbox.FileBasedMailbox", actorRef) + def createFileBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("akka.actor.mailbox.FileBasedMailbox", actorRef) - def createZooKeeperBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("se.scalablesolutions.akka.actor.mailbox.ZooKeeperBasedMailbox", actorRef) + def createZooKeeperBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("akka.actor.mailbox.ZooKeeperBasedMailbox", actorRef) - def createBeanstalkBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("se.scalablesolutions.akka.actor.mailbox.BeanstalkBasedMailbox", actorRef) + def createBeanstalkBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("akka.actor.mailbox.BeanstalkBasedMailbox", actorRef) - def createRedisBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("se.scalablesolutions.akka.actor.mailbox.RedisBasedMailbox", actorRef) + def createRedisBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("akka.actor.mailbox.RedisBasedMailbox", actorRef) private def createMailbox(mailboxClassname: String, actorRef: ActorRef): Mailbox = { ensureEnterpriseEnabled diff --git a/akka-actor/src/test/java/se/scalablesolutions/akka/config/SupervisionConfig.java b/akka-actor/src/test/java/akka/config/SupervisionConfig.java similarity index 81% rename from akka-actor/src/test/java/se/scalablesolutions/akka/config/SupervisionConfig.java rename to akka-actor/src/test/java/akka/config/SupervisionConfig.java index 967e4d7db1..908bc416d4 100644 --- a/akka-actor/src/test/java/se/scalablesolutions/akka/config/SupervisionConfig.java +++ b/akka-actor/src/test/java/akka/config/SupervisionConfig.java @@ -1,12 +1,12 @@ -package se.scalablesolutions.akka.config; +package akka.config; -import se.scalablesolutions.akka.actor.*; +import akka.actor.*; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import static se.scalablesolutions.akka.config.Supervision.*; +import static akka.config.Supervision.*; public class SupervisionConfig { /*Just some sample code to demonstrate the declarative supervision configuration for Java */ diff --git a/akka-actor/src/test/java/se/scalablesolutions/akka/japi/JavaAPITestBase.java b/akka-actor/src/test/java/akka/japi/JavaAPITestBase.java similarity index 96% rename from akka-actor/src/test/java/se/scalablesolutions/akka/japi/JavaAPITestBase.java rename to akka-actor/src/test/java/akka/japi/JavaAPITestBase.java index af00530593..e3a160f776 100644 --- a/akka-actor/src/test/java/se/scalablesolutions/akka/japi/JavaAPITestBase.java +++ b/akka-actor/src/test/java/akka/japi/JavaAPITestBase.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.japi; +package akka.japi; import org.junit.Test; diff --git a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/Address.java b/akka-actor/src/test/java/akka/stm/Address.java similarity index 85% rename from akka-actor/src/test/java/se/scalablesolutions/akka/stm/Address.java rename to akka-actor/src/test/java/akka/stm/Address.java index cb3057929f..55b30e991b 100644 --- a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/Address.java +++ b/akka-actor/src/test/java/akka/stm/Address.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.stm; +package akka.stm; public class Address { private String location; diff --git a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/CounterExample.java b/akka-actor/src/test/java/akka/stm/CounterExample.java similarity index 82% rename from akka-actor/src/test/java/se/scalablesolutions/akka/stm/CounterExample.java rename to akka-actor/src/test/java/akka/stm/CounterExample.java index 57a9a07daa..6af46ee0df 100644 --- a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/CounterExample.java +++ b/akka-actor/src/test/java/akka/stm/CounterExample.java @@ -1,7 +1,7 @@ -package se.scalablesolutions.akka.stm; +package akka.stm; -import se.scalablesolutions.akka.stm.Ref; -import se.scalablesolutions.akka.stm.local.Atomic; +import akka.stm.Ref; +import akka.stm.local.Atomic; public class CounterExample { final static Ref ref = new Ref(0); diff --git a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/JavaStmTests.java b/akka-actor/src/test/java/akka/stm/JavaStmTests.java similarity index 94% rename from akka-actor/src/test/java/se/scalablesolutions/akka/stm/JavaStmTests.java rename to akka-actor/src/test/java/akka/stm/JavaStmTests.java index 7204013808..15a9129655 100644 --- a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/JavaStmTests.java +++ b/akka-actor/src/test/java/akka/stm/JavaStmTests.java @@ -1,11 +1,11 @@ -package se.scalablesolutions.akka.stm; +package akka.stm; import static org.junit.Assert.*; import org.junit.Test; import org.junit.Before; -import se.scalablesolutions.akka.stm.*; -import se.scalablesolutions.akka.stm.local.Atomic; +import akka.stm.*; +import akka.stm.local.Atomic; import org.multiverse.api.ThreadLocalTransaction; import org.multiverse.api.TransactionConfiguration; diff --git a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/RefExample.java b/akka-actor/src/test/java/akka/stm/RefExample.java similarity index 85% rename from akka-actor/src/test/java/se/scalablesolutions/akka/stm/RefExample.java rename to akka-actor/src/test/java/akka/stm/RefExample.java index f590524fd7..22ffa17099 100644 --- a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/RefExample.java +++ b/akka-actor/src/test/java/akka/stm/RefExample.java @@ -1,7 +1,7 @@ -package se.scalablesolutions.akka.stm; +package akka.stm; -import se.scalablesolutions.akka.stm.Ref; -import se.scalablesolutions.akka.stm.local.Atomic; +import akka.stm.Ref; +import akka.stm.local.Atomic; public class RefExample { public static void main(String[] args) { diff --git a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/StmExamples.java b/akka-actor/src/test/java/akka/stm/StmExamples.java similarity index 74% rename from akka-actor/src/test/java/se/scalablesolutions/akka/stm/StmExamples.java rename to akka-actor/src/test/java/akka/stm/StmExamples.java index a8526f2dd0..3e8ca17e8f 100644 --- a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/StmExamples.java +++ b/akka-actor/src/test/java/akka/stm/StmExamples.java @@ -1,7 +1,7 @@ -package se.scalablesolutions.akka.stm; +package akka.stm; -import se.scalablesolutions.akka.stm.Ref; -import se.scalablesolutions.akka.stm.local.Atomic; +import akka.stm.Ref; +import akka.stm.local.Atomic; public class StmExamples { public static void main(String[] args) { diff --git a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/TransactionFactoryExample.java b/akka-actor/src/test/java/akka/stm/TransactionFactoryExample.java similarity index 87% rename from akka-actor/src/test/java/se/scalablesolutions/akka/stm/TransactionFactoryExample.java rename to akka-actor/src/test/java/akka/stm/TransactionFactoryExample.java index 00dd87b7c5..b8f189f47f 100644 --- a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/TransactionFactoryExample.java +++ b/akka-actor/src/test/java/akka/stm/TransactionFactoryExample.java @@ -1,7 +1,7 @@ -package se.scalablesolutions.akka.stm; +package akka.stm; -import se.scalablesolutions.akka.stm.*; -import se.scalablesolutions.akka.stm.local.Atomic; +import akka.stm.*; +import akka.stm.local.Atomic; import org.multiverse.api.ThreadLocalTransaction; import org.multiverse.api.TransactionConfiguration; diff --git a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/TransactionalMapExample.java b/akka-actor/src/test/java/akka/stm/TransactionalMapExample.java similarity index 87% rename from akka-actor/src/test/java/se/scalablesolutions/akka/stm/TransactionalMapExample.java rename to akka-actor/src/test/java/akka/stm/TransactionalMapExample.java index 7c4940c7a5..78dd1cd0ec 100644 --- a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/TransactionalMapExample.java +++ b/akka-actor/src/test/java/akka/stm/TransactionalMapExample.java @@ -1,7 +1,7 @@ -package se.scalablesolutions.akka.stm; +package akka.stm; -import se.scalablesolutions.akka.stm.*; -import se.scalablesolutions.akka.stm.local.Atomic; +import akka.stm.*; +import akka.stm.local.Atomic; public class TransactionalMapExample { public static void main(String[] args) { diff --git a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/TransactionalVectorExample.java b/akka-actor/src/test/java/akka/stm/TransactionalVectorExample.java similarity index 87% rename from akka-actor/src/test/java/se/scalablesolutions/akka/stm/TransactionalVectorExample.java rename to akka-actor/src/test/java/akka/stm/TransactionalVectorExample.java index 7274848beb..483bf65690 100644 --- a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/TransactionalVectorExample.java +++ b/akka-actor/src/test/java/akka/stm/TransactionalVectorExample.java @@ -1,7 +1,7 @@ -package se.scalablesolutions.akka.stm; +package akka.stm; -import se.scalablesolutions.akka.stm.*; -import se.scalablesolutions.akka.stm.local.Atomic; +import akka.stm.*; +import akka.stm.local.Atomic; public class TransactionalVectorExample { public static void main(String[] args) { diff --git a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/User.java b/akka-actor/src/test/java/akka/stm/User.java similarity index 83% rename from akka-actor/src/test/java/se/scalablesolutions/akka/stm/User.java rename to akka-actor/src/test/java/akka/stm/User.java index c9dc4b3723..5c148a21a4 100644 --- a/akka-actor/src/test/java/se/scalablesolutions/akka/stm/User.java +++ b/akka-actor/src/test/java/akka/stm/User.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.stm; +package akka.stm; public class User { private String name; diff --git a/akka-actor/src/test/scala/Messages.scala b/akka-actor/src/test/scala/Messages.scala index 7e22dd9d7c..fda3ba5b91 100644 --- a/akka-actor/src/test/scala/Messages.scala +++ b/akka-actor/src/test/scala/Messages.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka +package akka abstract class TestMessage diff --git a/akka-actor/src/test/scala/actor/actor/ActorFireForgetRequestReplySpec.scala b/akka-actor/src/test/scala/actor/actor/ActorFireForgetRequestReplySpec.scala index 0bff02a1a9..2ad5c82fa5 100644 --- a/akka-actor/src/test/scala/actor/actor/ActorFireForgetRequestReplySpec.scala +++ b/akka-actor/src/test/scala/actor/actor/ActorFireForgetRequestReplySpec.scala @@ -1,11 +1,11 @@ -package se.scalablesolutions.akka.actor +package akka.actor import java.util.concurrent.{TimeUnit, CyclicBarrier, TimeoutException} -import se.scalablesolutions.akka.config.Supervision._ +import akka.config.Supervision._ import org.scalatest.junit.JUnitSuite import org.junit.Test -import se.scalablesolutions.akka.dispatch.Dispatchers +import akka.dispatch.Dispatchers import Actor._ object ActorFireForgetRequestReplySpec { diff --git a/akka-actor/src/test/scala/actor/actor/ActorRefSpec.scala b/akka-actor/src/test/scala/actor/actor/ActorRefSpec.scala index 723ea14a73..dd620f6d7c 100644 --- a/akka-actor/src/test/scala/actor/actor/ActorRefSpec.scala +++ b/akka-actor/src/test/scala/actor/actor/ActorRefSpec.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -10,7 +10,7 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.actor._ +import akka.actor._ import java.util.concurrent.{CountDownLatch, TimeUnit} object ActorRefSpec { diff --git a/akka-actor/src/test/scala/actor/actor/AgentSpec.scala b/akka-actor/src/test/scala/actor/actor/AgentSpec.scala index 449fd89a7f..6a3af2bb75 100644 --- a/akka-actor/src/test/scala/actor/actor/AgentSpec.scala +++ b/akka-actor/src/test/scala/actor/actor/AgentSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.actor +package akka.actor import org.scalatest.Suite import org.scalatest.junit.JUnitRunner diff --git a/akka-actor/src/test/scala/actor/actor/Bench.scala b/akka-actor/src/test/scala/actor/actor/Bench.scala index ded90edad5..f043f5c92e 100644 --- a/akka-actor/src/test/scala/actor/actor/Bench.scala +++ b/akka-actor/src/test/scala/actor/actor/Bench.scala @@ -3,9 +3,9 @@ contributed by Julien Gaugaz inspired by the version contributed by Yura Taras and modified by Isaac Gouy */ -package se.scalablesolutions.akka.actor +package akka.actor -import se.scalablesolutions.akka.actor.Actor._ +import akka.actor.Actor._ object Chameneos { diff --git a/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala b/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala index dc6893c820..78519122af 100644 --- a/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala +++ b/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor import org.scalatest.junit.JUnitSuite import org.junit.Test @@ -17,6 +17,7 @@ object FSMActorSpec { val lockedLatch = new StandardLatch val unhandledLatch = new StandardLatch val terminatedLatch = new StandardLatch + val transitionLatch = new StandardLatch sealed trait LockState case object Locked extends LockState @@ -24,7 +25,12 @@ object FSMActorSpec { class Lock(code: String, timeout: Int) extends Actor with FSM[LockState, CodeState] { - inState(Locked) { + notifying { + case Transition(Locked, Open) => transitionLatch.open + case Transition(_, _) => () + } + + when(Locked) { case Event(digit: Char, CodeState(soFar, code)) => { soFar + digit match { case incomplete if incomplete.length < code.length => @@ -43,14 +49,14 @@ object FSMActorSpec { case Event("bye", _) => stop(Shutdown) } - inState(Open) { + when(Open) { case Event(StateTimeout, stateData) => { doLock goto(Locked) } } - setInitialState(Locked, CodeState("", code)) + startWith(Locked, CodeState("", code)) whenUnhandled { case Event(_, stateData) => { @@ -94,6 +100,7 @@ class FSMActorSpec extends JUnitSuite { lock ! '1' assert(unlockedLatch.tryAwait(1, TimeUnit.SECONDS)) + assert(transitionLatch.tryAwait(1, TimeUnit.SECONDS)) assert(lockedLatch.tryAwait(2, TimeUnit.SECONDS)) lock ! "not_handled" diff --git a/akka-actor/src/test/scala/actor/actor/ForwardActorSpec.scala b/akka-actor/src/test/scala/actor/actor/ForwardActorSpec.scala index e3ab0bded7..3a1efe1fe8 100644 --- a/akka-actor/src/test/scala/actor/actor/ForwardActorSpec.scala +++ b/akka-actor/src/test/scala/actor/actor/ForwardActorSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.actor +package akka.actor import java.util.concurrent.{TimeUnit, CountDownLatch} import org.scalatest.junit.JUnitSuite diff --git a/akka-actor/src/test/scala/actor/actor/HotSwapSpec.scala b/akka-actor/src/test/scala/actor/actor/HotSwapSpec.scala index 7caa194dbe..527bd544fb 100644 --- a/akka-actor/src/test/scala/actor/actor/HotSwapSpec.scala +++ b/akka-actor/src/test/scala/actor/actor/HotSwapSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.actor +package akka.actor import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers diff --git a/akka-actor/src/test/scala/actor/actor/ReceiveTimeoutSpec.scala b/akka-actor/src/test/scala/actor/actor/ReceiveTimeoutSpec.scala index a96d63e7fc..b47b6830b4 100644 --- a/akka-actor/src/test/scala/actor/actor/ReceiveTimeoutSpec.scala +++ b/akka-actor/src/test/scala/actor/actor/ReceiveTimeoutSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.actor +package akka.actor import org.scalatest.junit.JUnitSuite import org.junit.Test diff --git a/akka-actor/src/test/scala/actor/actor/TransactorSpec.scala b/akka-actor/src/test/scala/actor/actor/TransactorSpec.scala index dd23a76a88..be182df870 100644 --- a/akka-actor/src/test/scala/actor/actor/TransactorSpec.scala +++ b/akka-actor/src/test/scala/actor/actor/TransactorSpec.scala @@ -1,10 +1,10 @@ -package se.scalablesolutions.akka.actor +package akka.actor import java.util.concurrent.{TimeUnit, CountDownLatch} import org.scalatest.junit.JUnitSuite import org.junit.Test -import se.scalablesolutions.akka.stm.{Ref, TransactionalMap, TransactionalVector} +import akka.stm.{Ref, TransactionalMap, TransactionalVector} import Actor._ object TransactorSpec { diff --git a/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala b/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala index ba88ccd842..81937d7760 100644 --- a/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala +++ b/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor import java.lang.Thread.sleep @@ -11,7 +11,7 @@ import org.junit.Test import Actor._ import java.util.concurrent.{TimeUnit, CountDownLatch} -import se.scalablesolutions.akka.config.Supervision.{Permanent, LifeCycle, OneForOneStrategy} +import akka.config.Supervision.{Permanent, LifeCycle, OneForOneStrategy} import org.multiverse.api.latches.StandardLatch class RestartStrategySpec extends JUnitSuite { diff --git a/akka-actor/src/test/scala/actor/supervisor/SupervisorHierarchySpec.scala b/akka-actor/src/test/scala/actor/supervisor/SupervisorHierarchySpec.scala index 4091215571..bfdf9b600a 100644 --- a/akka-actor/src/test/scala/actor/supervisor/SupervisorHierarchySpec.scala +++ b/akka-actor/src/test/scala/actor/supervisor/SupervisorHierarchySpec.scala @@ -2,13 +2,13 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor import org.scalatest.junit.JUnitSuite import org.junit.Test import Actor._ -import se.scalablesolutions.akka.config.Supervision.OneForOneStrategy +import akka.config.Supervision.OneForOneStrategy import java.util.concurrent.{TimeUnit, CountDownLatch} diff --git a/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala b/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala index 17cfc94f83..08de79c400 100644 --- a/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala +++ b/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala @@ -1,12 +1,12 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers -import se.scalablesolutions.akka.dispatch.Dispatchers -import se.scalablesolutions.akka.config.Supervision.{SupervisorConfig, OneForOneStrategy, Supervise, Permanent} +import akka.dispatch.Dispatchers +import akka.config.Supervision.{SupervisorConfig, OneForOneStrategy, Supervise, Permanent} import java.util.concurrent.CountDownLatch class SupervisorMiscSpec extends WordSpec with MustMatchers { diff --git a/akka-actor/src/test/scala/actor/supervisor/SupervisorSpec.scala b/akka-actor/src/test/scala/actor/supervisor/SupervisorSpec.scala index 5f73c4ce8e..d01c064b3b 100644 --- a/akka-actor/src/test/scala/actor/supervisor/SupervisorSpec.scala +++ b/akka-actor/src/test/scala/actor/supervisor/SupervisorSpec.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor -import se.scalablesolutions.akka.config.Supervision._ -import se.scalablesolutions.akka.{OneWay, Die, Ping} +import akka.config.Supervision._ +import akka.{OneWay, Die, Ping} import Actor._ import org.scalatest.junit.JUnitSuite diff --git a/akka-actor/src/test/scala/dataflow/DataFlowSpec.scala b/akka-actor/src/test/scala/dataflow/DataFlowSpec.scala index d596ecfac1..a42ac353f2 100644 --- a/akka-actor/src/test/scala/dataflow/DataFlowSpec.scala +++ b/akka-actor/src/test/scala/dataflow/DataFlowSpec.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.dataflow +package akka.dataflow import org.scalatest.Spec import org.scalatest.Assertions @@ -11,11 +11,11 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.dispatch.DefaultCompletableFuture +import akka.dispatch.DefaultCompletableFuture import java.util.concurrent.{TimeUnit, CountDownLatch} import annotation.tailrec import java.util.concurrent.atomic.{AtomicLong, AtomicReference, AtomicInteger} -import se.scalablesolutions.akka.actor.ActorRegistry +import akka.actor.ActorRegistry @RunWith(classOf[JUnitRunner]) class DataFlowTest extends Spec with ShouldMatchers with BeforeAndAfterAll { diff --git a/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala b/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala index 1761f0195a..cdbc6e2ab4 100644 --- a/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala @@ -1,17 +1,17 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor.dispatch +package akka.actor.dispatch import org.scalatest.junit.JUnitSuite import org.junit.Test import org.scalatest.Assertions._ -import se.scalablesolutions.akka.dispatch._ -import se.scalablesolutions.akka.actor.{ActorRef, Actor} -import se.scalablesolutions.akka.actor.Actor._ +import akka.dispatch._ +import akka.actor.{ActorRef, Actor} +import akka.actor.Actor._ import java.util.concurrent.atomic.AtomicLong import java.util.concurrent. {ConcurrentHashMap, CountDownLatch, TimeUnit} -import se.scalablesolutions.akka.actor.dispatch.ActorModelSpec.MessageDispatcherInterceptor +import akka.actor.dispatch.ActorModelSpec.MessageDispatcherInterceptor object ActorModelSpec { diff --git a/akka-actor/src/test/scala/dispatch/DispatchersSpec.scala b/akka-actor/src/test/scala/dispatch/DispatchersSpec.scala index d10cf86db6..ba9398de2c 100644 --- a/akka-actor/src/test/scala/dispatch/DispatchersSpec.scala +++ b/akka-actor/src/test/scala/dispatch/DispatchersSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor.dispatch +package akka.actor.dispatch import java.util.concurrent.{CountDownLatch, TimeUnit} import org.scalatest.junit.JUnitSuite @@ -9,7 +9,7 @@ import org.junit.Test import net.lag.configgy.Config import scala.reflect.{Manifest} -import se.scalablesolutions.akka.dispatch._ +import akka.dispatch._ object DispatchersSpec { import Dispatchers._ diff --git a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala index f2f4787070..e9b34c17d3 100644 --- a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala @@ -1,10 +1,10 @@ -package se.scalablesolutions.akka.actor.dispatch +package akka.actor.dispatch import java.util.concurrent.{CountDownLatch, TimeUnit} import org.scalatest.junit.JUnitSuite import org.junit.Test -import se.scalablesolutions.akka.dispatch.{Dispatchers,ExecutorBasedEventDrivenDispatcher} -import se.scalablesolutions.akka.actor.Actor +import akka.dispatch.{Dispatchers,ExecutorBasedEventDrivenDispatcher} +import akka.actor.Actor import Actor._ import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} diff --git a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorsSpec.scala b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorsSpec.scala index fc8f1aa37f..66a02e0d33 100644 --- a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorsSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorsSpec.scala @@ -1,10 +1,10 @@ -package se.scalablesolutions.akka.actor.dispatch +package akka.actor.dispatch import org.scalatest.junit.JUnitSuite import org.junit.Test import org.scalatest.matchers.MustMatchers import java.util.concurrent.CountDownLatch -import se.scalablesolutions.akka.actor.Actor +import akka.actor.Actor import Actor._ /** diff --git a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala index fe45d79869..698ffe8063 100644 --- a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.actor.dispatch +package akka.actor.dispatch import org.scalatest.matchers.MustMatchers import org.scalatest.junit.JUnitSuite @@ -6,9 +6,9 @@ import org.scalatest.junit.JUnitSuite import org.junit.Test import java.util.concurrent.{TimeUnit, CountDownLatch} -import se.scalablesolutions.akka.actor.{IllegalActorStateException, Actor} +import akka.actor.{IllegalActorStateException, Actor} import Actor._ -import se.scalablesolutions.akka.dispatch.{MessageQueue, Dispatchers} +import akka.dispatch.{MessageQueue, Dispatchers} object ExecutorBasedEventDrivenWorkStealingDispatcherSpec { val delayableActorDispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher").build diff --git a/akka-actor/src/test/scala/dispatch/FutureSpec.scala b/akka-actor/src/test/scala/dispatch/FutureSpec.scala index 04316f8a3d..d8b03bb0da 100644 --- a/akka-actor/src/test/scala/dispatch/FutureSpec.scala +++ b/akka-actor/src/test/scala/dispatch/FutureSpec.scala @@ -1,8 +1,8 @@ -package se.scalablesolutions.akka.actor +package akka.actor import org.scalatest.junit.JUnitSuite import org.junit.Test -import se.scalablesolutions.akka.dispatch.Futures +import akka.dispatch.Futures import Actor._ import org.multiverse.api.latches.StandardLatch diff --git a/akka-actor/src/test/scala/dispatch/HawtDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/HawtDispatcherActorSpec.scala index 2c45f3388c..1d1b2c1e24 100644 --- a/akka-actor/src/test/scala/dispatch/HawtDispatcherActorSpec.scala +++ b/akka-actor/src/test/scala/dispatch/HawtDispatcherActorSpec.scala @@ -1,12 +1,12 @@ -package se.scalablesolutions.akka.actor.dispatch +package akka.actor.dispatch import java.util.concurrent.{CountDownLatch, TimeUnit} import org.scalatest.junit.JUnitSuite import org.junit.Test -import se.scalablesolutions.akka.dispatch.{HawtDispatcher, Dispatchers} -import se.scalablesolutions.akka.actor.Actor +import akka.dispatch.{HawtDispatcher, Dispatchers} +import akka.actor.Actor import Actor._ object HawtDispatcherActorSpec { diff --git a/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala b/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala index 6196a13490..a052ca690d 100644 --- a/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala +++ b/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor.dispatch +package akka.actor.dispatch import scala.collection.mutable.ListBuffer @@ -12,9 +12,9 @@ import java.io.IOException import java.nio.ByteBuffer import java.nio.channels.{SocketChannel, SelectionKey, ServerSocketChannel} -import se.scalablesolutions.akka.actor._ -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.dispatch.HawtDispatcher +import akka.actor._ +import akka.actor.Actor._ +import akka.dispatch.HawtDispatcher import org.fusesource.hawtdispatch.DispatchSource import org.fusesource.hawtdispatch.ScalaDispatch._ diff --git a/akka-actor/src/test/scala/dispatch/MailboxConfigSpec.scala b/akka-actor/src/test/scala/dispatch/MailboxConfigSpec.scala index 0dfd8c1c65..5dd0dfbe6d 100644 --- a/akka-actor/src/test/scala/dispatch/MailboxConfigSpec.scala +++ b/akka-actor/src/test/scala/dispatch/MailboxConfigSpec.scala @@ -1,12 +1,12 @@ -package se.scalablesolutions.akka.actor.dispatch +package akka.actor.dispatch import org.scalatest.junit.JUnitSuite import org.junit.Test -import se.scalablesolutions.akka.actor.Actor -import se.scalablesolutions.akka.util.Duration -import se.scalablesolutions.akka.dispatch._ +import akka.actor.Actor +import akka.util.Duration +import akka.dispatch._ import Actor._ import java.util.concurrent.{BlockingQueue, CountDownLatch, TimeUnit} diff --git a/akka-actor/src/test/scala/dispatch/ThreadBasedActorSpec.scala b/akka-actor/src/test/scala/dispatch/ThreadBasedActorSpec.scala index d69ee984d8..eee135ebab 100644 --- a/akka-actor/src/test/scala/dispatch/ThreadBasedActorSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ThreadBasedActorSpec.scala @@ -1,11 +1,11 @@ -package se.scalablesolutions.akka.actor.dispatch +package 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 akka.dispatch.Dispatchers +import akka.actor.Actor import Actor._ object ThreadBasedActorSpec { diff --git a/akka-actor/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala b/akka-actor/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala index 7ecef80e39..603b17e336 100644 --- a/akka-actor/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.dispatch +package akka.dispatch import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit @@ -9,7 +9,7 @@ import java.util.concurrent.locks.ReentrantLock import org.scalatest.junit.JUnitSuite import org.junit.{Test, Before} -import se.scalablesolutions.akka.actor.Actor +import akka.actor.Actor import Actor._ // FIXME use this test when we have removed the MessageInvoker classes diff --git a/akka-actor/src/test/scala/japi/JavaAPITest.scala b/akka-actor/src/test/scala/japi/JavaAPITest.scala index 721342b7af..ebaa317115 100644 --- a/akka-actor/src/test/scala/japi/JavaAPITest.scala +++ b/akka-actor/src/test/scala/japi/JavaAPITest.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.japi +package akka.japi import org.scalatest.junit.JUnitSuite diff --git a/akka-actor/src/test/scala/misc/ActorRegistrySpec.scala b/akka-actor/src/test/scala/misc/ActorRegistrySpec.scala index 1fe72d6c68..6148b04f53 100644 --- a/akka-actor/src/test/scala/misc/ActorRegistrySpec.scala +++ b/akka-actor/src/test/scala/misc/ActorRegistrySpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.actor +package akka.actor import org.scalatest.junit.JUnitSuite import org.junit.Test diff --git a/akka-actor/src/test/scala/misc/SchedulerSpec.scala b/akka-actor/src/test/scala/misc/SchedulerSpec.scala index 2278536c1d..a4471503da 100644 --- a/akka-actor/src/test/scala/misc/SchedulerSpec.scala +++ b/akka-actor/src/test/scala/misc/SchedulerSpec.scala @@ -1,9 +1,9 @@ -package se.scalablesolutions.akka.actor +package akka.actor import org.scalatest.junit.JUnitSuite import Actor._ import java.util.concurrent.{CountDownLatch, TimeUnit} -import se.scalablesolutions.akka.config.Supervision._ +import akka.config.Supervision._ import org.multiverse.api.latches.StandardLatch import org.junit.Test diff --git a/akka-actor/src/test/scala/routing/RoutingSpec.scala b/akka-actor/src/test/scala/routing/RoutingSpec.scala index a7cf233fb0..0f46b2340c 100644 --- a/akka-actor/src/test/scala/routing/RoutingSpec.scala +++ b/akka-actor/src/test/scala/routing/RoutingSpec.scala @@ -1,8 +1,8 @@ -package se.scalablesolutions.akka.actor.routing +package akka.actor.routing -import se.scalablesolutions.akka.actor.Actor -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.util.Logging +import akka.actor.Actor +import akka.actor.Actor._ +import akka.util.Logging import org.scalatest.Suite import org.junit.runner.RunWith @@ -12,7 +12,7 @@ import org.junit.Test import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.{CountDownLatch, TimeUnit} -import se.scalablesolutions.akka.routing._ +import akka.routing._ @RunWith(classOf[JUnitRunner]) class RoutingSpec extends junit.framework.TestCase with Suite with MustMatchers with Logging { @@ -134,7 +134,7 @@ class RoutingSpec extends junit.framework.TestCase with Suite with MustMatchers } @Test def testIsDefinedAt = { - import se.scalablesolutions.akka.actor.ActorRef + import akka.actor.ActorRef val (testMsg1,testMsg2,testMsg3,testMsg4) = ("test1","test2","test3","test4") diff --git a/akka-actor/src/test/scala/stm/JavaStmSpec.scala b/akka-actor/src/test/scala/stm/JavaStmSpec.scala index 70dcefd79e..6bd80c7b0e 100644 --- a/akka-actor/src/test/scala/stm/JavaStmSpec.scala +++ b/akka-actor/src/test/scala/stm/JavaStmSpec.scala @@ -1,5 +1,5 @@ -package se.scalablesolutions.akka.stm +package akka.stm import org.scalatest.junit.JUnitWrapperSuite -class JavaStmSpec extends JUnitWrapperSuite("se.scalablesolutions.akka.stm.JavaStmTests", Thread.currentThread.getContextClassLoader) +class JavaStmSpec extends JUnitWrapperSuite("akka.stm.JavaStmTests", Thread.currentThread.getContextClassLoader) diff --git a/akka-actor/src/test/scala/stm/RefSpec.scala b/akka-actor/src/test/scala/stm/RefSpec.scala index 2a8d39a065..1fde341756 100644 --- a/akka-actor/src/test/scala/stm/RefSpec.scala +++ b/akka-actor/src/test/scala/stm/RefSpec.scala @@ -1,11 +1,11 @@ -package se.scalablesolutions.akka.stm +package akka.stm import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers class RefSpec extends WordSpec with MustMatchers { - import se.scalablesolutions.akka.stm.local._ + import akka.stm.local._ "A Ref" should { diff --git a/akka-actor/src/test/scala/stm/StmSpec.scala b/akka-actor/src/test/scala/stm/StmSpec.scala index cadd419554..a13615f6f2 100644 --- a/akka-actor/src/test/scala/stm/StmSpec.scala +++ b/akka-actor/src/test/scala/stm/StmSpec.scala @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.stm +package akka.stm -import se.scalablesolutions.akka.actor.{Actor, Transactor} +import akka.actor.{Actor, Transactor} import Actor._ import org.multiverse.api.exceptions.ReadonlyException @@ -12,7 +12,7 @@ class StmSpec extends WordSpec with MustMatchers { "Local STM" should { - import se.scalablesolutions.akka.stm.local._ + import akka.stm.local._ "be able to do multiple consecutive atomic {..} statements" in { val ref = Ref(0) @@ -178,7 +178,7 @@ object GlobalTransactionVectorTestActor { class GlobalTransactionVectorTestActor extends Actor { import GlobalTransactionVectorTestActor._ - import se.scalablesolutions.akka.stm.global._ + import akka.stm.global._ private val vector: TransactionalVector[Int] = atomic { TransactionalVector(1) } diff --git a/akka-actor/src/test/scala/ticket/Ticket001Spec.scala b/akka-actor/src/test/scala/ticket/Ticket001Spec.scala index b94796d9a3..d4de2675fb 100644 --- a/akka-actor/src/test/scala/ticket/Ticket001Spec.scala +++ b/akka-actor/src/test/scala/ticket/Ticket001Spec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.actor.ticket +package akka.actor.ticket import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers diff --git a/akka-amqp/src/main/java/se/scalablesolutions/akka/amqp/ExampleSessionJava.java b/akka-amqp/src/main/java/akka/amqp/ExampleSessionJava.java similarity index 95% rename from akka-amqp/src/main/java/se/scalablesolutions/akka/amqp/ExampleSessionJava.java rename to akka-amqp/src/main/java/akka/amqp/ExampleSessionJava.java index 398feb17ce..05ed2c3437 100644 --- a/akka-amqp/src/main/java/se/scalablesolutions/akka/amqp/ExampleSessionJava.java +++ b/akka-amqp/src/main/java/akka/amqp/ExampleSessionJava.java @@ -1,17 +1,17 @@ -package se.scalablesolutions.akka.amqp; +package akka.amqp; import org.multiverse.api.latches.StandardLatch; import scala.Option; -import se.scalablesolutions.akka.actor.ActorRef; -import se.scalablesolutions.akka.actor.ActorRegistry; -import se.scalablesolutions.akka.actor.UntypedActor; -import se.scalablesolutions.akka.actor.UntypedActorFactory; +import akka.actor.ActorRef; +import akka.actor.ActorRegistry; +import akka.actor.UntypedActor; +import akka.actor.UntypedActorFactory; -import se.scalablesolutions.akka.amqp.rpc.RPC; -import se.scalablesolutions.akka.remote.protocol.RemoteProtocol; +import akka.amqp.rpc.RPC; +import akka.remote.protocol.RemoteProtocol; -import se.scalablesolutions.akka.japi.Function; -import se.scalablesolutions.akka.japi.Procedure; +import akka.japi.Function; +import akka.japi.Procedure; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQP.scala b/akka-amqp/src/main/scala/akka/amqp/AMQP.scala similarity index 98% rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQP.scala rename to akka-amqp/src/main/scala/akka/amqp/AMQP.scala index 662f2b6a24..04c8753049 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQP.scala +++ b/akka-amqp/src/main/scala/akka/amqp/AMQP.scala @@ -2,22 +2,22 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.amqp +package akka.amqp -import se.scalablesolutions.akka.actor.{Actor, ActorRef} -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.config.Supervision.OneForOneStrategy +import akka.actor.{Actor, ActorRef} +import akka.actor.Actor._ +import akka.config.Supervision.OneForOneStrategy import com.rabbitmq.client.{ReturnListener, ShutdownListener, ConnectionFactory} import ConnectionFactory._ import com.rabbitmq.client.AMQP.BasicProperties import java.lang.{String, IllegalArgumentException} import reflect.Manifest -import se.scalablesolutions.akka.japi.Procedure +import akka.japi.Procedure /** * AMQP Actor API. Implements Connection, Producer and Consumer materialized as Actors. * - * @see se.scalablesolutions.akka.amqp.ExampleSession + * @see akka.amqp.ExampleSession * * @author Irmo Manie */ diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQPMessage.scala b/akka-amqp/src/main/scala/akka/amqp/AMQPMessage.scala similarity index 97% rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQPMessage.scala rename to akka-amqp/src/main/scala/akka/amqp/AMQPMessage.scala index 7f1ef053de..88bcea866d 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQPMessage.scala +++ b/akka-amqp/src/main/scala/akka/amqp/AMQPMessage.scala @@ -2,9 +2,9 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.amqp +package akka.amqp -import se.scalablesolutions.akka.actor.ActorRef +import akka.actor.ActorRef import com.rabbitmq.client.AMQP.BasicProperties import com.rabbitmq.client.ShutdownSignalException diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala b/akka-amqp/src/main/scala/akka/amqp/ConsumerActor.scala similarity index 95% rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala rename to akka-amqp/src/main/scala/akka/amqp/ConsumerActor.scala index e5ff20cbb4..c14b1a6981 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala +++ b/akka-amqp/src/main/scala/akka/amqp/ConsumerActor.scala @@ -2,15 +2,15 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.amqp +package akka.amqp import collection.JavaConversions -import se.scalablesolutions.akka.util.Logging +import akka.util.Logging import com.rabbitmq.client.AMQP.BasicProperties import com.rabbitmq.client.{Channel, Envelope, DefaultConsumer} -import se.scalablesolutions.akka.amqp.AMQP.{NoActionDeclaration, ActiveDeclaration, PassiveDeclaration, ConsumerParameters} +import akka.amqp.AMQP.{NoActionDeclaration, ActiveDeclaration, PassiveDeclaration, ConsumerParameters} private[amqp] class ConsumerActor(consumerParameters: ConsumerParameters) extends FaultTolerantChannelActor( diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala b/akka-amqp/src/main/scala/akka/amqp/ExampleSession.scala similarity index 97% rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala rename to akka-amqp/src/main/scala/akka/amqp/ExampleSession.scala index 9bda7f54f4..8ca3961177 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala +++ b/akka-amqp/src/main/scala/akka/amqp/ExampleSession.scala @@ -2,16 +2,16 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.amqp +package akka.amqp import rpc.RPC import rpc.RPC.{RpcClientSerializer, RpcServerSerializer} -import se.scalablesolutions.akka.actor.{Actor, ActorRegistry} +import akka.actor.{Actor, ActorRegistry} import Actor._ import java.util.concurrent.{CountDownLatch, TimeUnit} import java.lang.String -import se.scalablesolutions.akka.amqp.AMQP._ -import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol +import akka.amqp.AMQP._ +import akka.remote.protocol.RemoteProtocol.AddressProtocol object ExampleSession { diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExchangeType.scala b/akka-amqp/src/main/scala/akka/amqp/ExchangeType.scala similarity index 94% rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExchangeType.scala rename to akka-amqp/src/main/scala/akka/amqp/ExchangeType.scala index 2c35a017e4..b29e1e7170 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExchangeType.scala +++ b/akka-amqp/src/main/scala/akka/amqp/ExchangeType.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.amqp +package akka.amqp sealed trait ExchangeType case object Direct extends ExchangeType { diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala b/akka-amqp/src/main/scala/akka/amqp/FaultTolerantChannelActor.scala similarity index 96% rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala rename to akka-amqp/src/main/scala/akka/amqp/FaultTolerantChannelActor.scala index 6617c62a44..78b532623a 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala +++ b/akka-amqp/src/main/scala/akka/amqp/FaultTolerantChannelActor.scala @@ -2,15 +2,15 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.amqp +package akka.amqp import collection.JavaConversions import java.lang.Throwable -import se.scalablesolutions.akka.actor.Actor +import akka.actor.Actor import Actor._ import com.rabbitmq.client.{ShutdownSignalException, Channel, ShutdownListener} import scala.PartialFunction -import se.scalablesolutions.akka.amqp.AMQP._ +import akka.amqp.AMQP._ abstract private[amqp] class FaultTolerantChannelActor( exchangeParameters: Option[ExchangeParameters], channelParameters: Option[ChannelParameters]) extends Actor { diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala b/akka-amqp/src/main/scala/akka/amqp/FaultTolerantConnectionActor.scala similarity index 93% rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala rename to akka-amqp/src/main/scala/akka/amqp/FaultTolerantConnectionActor.scala index 066075f26c..b202e84b18 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala +++ b/akka-amqp/src/main/scala/akka/amqp/FaultTolerantConnectionActor.scala @@ -2,14 +2,14 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.amqp +package akka.amqp import java.util.{TimerTask, Timer} import java.io.IOException import com.rabbitmq.client._ -import se.scalablesolutions.akka.amqp.AMQP.ConnectionParameters -import se.scalablesolutions.akka.config.Supervision.{ Permanent, OneForOneStrategy } -import se.scalablesolutions.akka.actor.{Exit, Actor} +import akka.amqp.AMQP.ConnectionParameters +import akka.config.Supervision.{ Permanent, OneForOneStrategy } +import akka.actor.{Exit, Actor} private[amqp] class FaultTolerantConnectionActor(connectionParameters: ConnectionParameters) extends Actor { import connectionParameters._ diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ProducerActor.scala b/akka-amqp/src/main/scala/akka/amqp/ProducerActor.scala similarity index 95% rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ProducerActor.scala rename to akka-amqp/src/main/scala/akka/amqp/ProducerActor.scala index bb3448f8b5..8aabd26649 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ProducerActor.scala +++ b/akka-amqp/src/main/scala/akka/amqp/ProducerActor.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.amqp +package akka.amqp import com.rabbitmq.client._ -import se.scalablesolutions.akka.amqp.AMQP.ProducerParameters +import akka.amqp.AMQP.ProducerParameters private[amqp] class ProducerActor(producerParameters: ProducerParameters) extends FaultTolerantChannelActor( diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RPC.scala b/akka-amqp/src/main/scala/akka/amqp/rpc/RPC.scala similarity index 98% rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RPC.scala rename to akka-amqp/src/main/scala/akka/amqp/rpc/RPC.scala index ed0f8be7e1..d4e40c0e3d 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RPC.scala +++ b/akka-amqp/src/main/scala/akka/amqp/rpc/RPC.scala @@ -1,12 +1,12 @@ -package se.scalablesolutions.akka.amqp.rpc +package akka.amqp.rpc -import se.scalablesolutions.akka.amqp.AMQP._ +import akka.amqp.AMQP._ import com.google.protobuf.Message -import se.scalablesolutions.akka.actor.{Actor, ActorRef} +import akka.actor.{Actor, ActorRef} import Actor._ -import se.scalablesolutions.akka.amqp._ +import akka.amqp._ import reflect.Manifest -import se.scalablesolutions.akka.japi +import akka.japi object RPC { diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcClientActor.scala b/akka-amqp/src/main/scala/akka/amqp/rpc/RpcClientActor.scala similarity index 91% rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcClientActor.scala rename to akka-amqp/src/main/scala/akka/amqp/rpc/RpcClientActor.scala index 90fe3ac66a..cae8587fb1 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcClientActor.scala +++ b/akka-amqp/src/main/scala/akka/amqp/rpc/RpcClientActor.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.amqp +package akka.amqp import com.rabbitmq.client.{Channel, RpcClient} import rpc.RPC.RpcClientSerializer -import se.scalablesolutions.akka.amqp.AMQP.{ChannelParameters, ExchangeParameters} +import akka.amqp.AMQP.{ChannelParameters, ExchangeParameters} class RpcClientActor[I,O]( exchangeParameters: ExchangeParameters, diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcServerActor.scala b/akka-amqp/src/main/scala/akka/amqp/rpc/RpcServerActor.scala similarity index 91% rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcServerActor.scala rename to akka-amqp/src/main/scala/akka/amqp/rpc/RpcServerActor.scala index 5f6b4b713c..40443c63ad 100644 --- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcServerActor.scala +++ b/akka-amqp/src/main/scala/akka/amqp/rpc/RpcServerActor.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.amqp +package akka.amqp import rpc.RPC.RpcServerSerializer -import se.scalablesolutions.akka.actor.{ActorRef, Actor} +import akka.actor.{ActorRef, Actor} import com.rabbitmq.client.AMQP.BasicProperties class RpcServerActor[I,O]( diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConnectionRecoveryTestIntegration.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPConnectionRecoveryTestIntegration.scala similarity index 88% rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConnectionRecoveryTestIntegration.scala rename to akka-amqp/src/test/scala/akka/amqp/test/AMQPConnectionRecoveryTestIntegration.scala index 0187ef0cbe..64f6c99040 100644 --- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConnectionRecoveryTestIntegration.scala +++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPConnectionRecoveryTestIntegration.scala @@ -1,15 +1,15 @@ -package se.scalablesolutions.akka.amqp.test +package akka.amqp.test /** * Copyright (C) 2009-2010 Scalable Solutions AB */ import java.util.concurrent.TimeUnit -import se.scalablesolutions.akka.actor.{Actor, ActorRef} +import akka.actor.{Actor, ActorRef} import org.multiverse.api.latches.StandardLatch import com.rabbitmq.client.ShutdownSignalException -import se.scalablesolutions.akka.amqp._ -import se.scalablesolutions.akka.amqp.AMQP.ConnectionParameters +import akka.amqp._ +import akka.amqp.AMQP.ConnectionParameters import org.scalatest.matchers.MustMatchers import org.scalatest.junit.JUnitSuite import org.junit.Test diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerChannelRecoveryTestIntegration.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerChannelRecoveryTestIntegration.scala similarity index 90% rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerChannelRecoveryTestIntegration.scala rename to akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerChannelRecoveryTestIntegration.scala index ba96439d58..72c7590f86 100644 --- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerChannelRecoveryTestIntegration.scala +++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerChannelRecoveryTestIntegration.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.amqp.test +package akka.amqp.test /** * Copyright (C) 2009-2010 Scalable Solutions AB @@ -6,14 +6,14 @@ package se.scalablesolutions.akka.amqp.test import org.multiverse.api.latches.StandardLatch import com.rabbitmq.client.ShutdownSignalException -import se.scalablesolutions.akka.amqp._ +import akka.amqp._ import org.scalatest.matchers.MustMatchers import java.util.concurrent.TimeUnit import org.junit.Test -import se.scalablesolutions.akka.amqp.AMQP._ +import akka.amqp.AMQP._ import org.scalatest.junit.JUnitSuite -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.actor.{Actor, ActorRef} +import akka.actor.Actor._ +import akka.actor.{Actor, ActorRef} class AMQPConsumerChannelRecoveryTestIntegration extends JUnitSuite with MustMatchers { diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerConnectionRecoveryTestIntegration.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerConnectionRecoveryTestIntegration.scala similarity index 94% rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerConnectionRecoveryTestIntegration.scala rename to akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerConnectionRecoveryTestIntegration.scala index 1e8fc23ac6..1752afe400 100644 --- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerConnectionRecoveryTestIntegration.scala +++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerConnectionRecoveryTestIntegration.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.amqp.test +package akka.amqp.test /** * Copyright (C) 2009-2010 Scalable Solutions AB @@ -6,13 +6,13 @@ package se.scalablesolutions.akka.amqp.test import org.multiverse.api.latches.StandardLatch import com.rabbitmq.client.ShutdownSignalException -import se.scalablesolutions.akka.amqp._ +import akka.amqp._ import org.scalatest.matchers.MustMatchers import java.util.concurrent.TimeUnit import org.junit.Test -import se.scalablesolutions.akka.amqp.AMQP._ +import akka.amqp.AMQP._ import org.scalatest.junit.JUnitSuite -import se.scalablesolutions.akka.actor.{Actor, ActorRef} +import akka.actor.{Actor, ActorRef} import Actor._ class AMQPConsumerConnectionRecoveryTestIntegration extends JUnitSuite with MustMatchers { diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualAcknowledgeTestIntegration.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerManualAcknowledgeTestIntegration.scala similarity index 90% rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualAcknowledgeTestIntegration.scala rename to akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerManualAcknowledgeTestIntegration.scala index 03f39a669d..25435d24e5 100644 --- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualAcknowledgeTestIntegration.scala +++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerManualAcknowledgeTestIntegration.scala @@ -1,18 +1,18 @@ -package se.scalablesolutions.akka.amqp.test +package akka.amqp.test /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -import se.scalablesolutions.akka.actor.Actor._ +import akka.actor.Actor._ import org.scalatest.matchers.MustMatchers -import se.scalablesolutions.akka.amqp._ +import akka.amqp._ import org.junit.Test import java.util.concurrent.{CountDownLatch, TimeUnit} import org.multiverse.api.latches.StandardLatch import org.scalatest.junit.JUnitSuite -import se.scalablesolutions.akka.amqp.AMQP._ -import se.scalablesolutions.akka.actor.{Actor, ActorRef} +import akka.amqp.AMQP._ +import akka.actor.{Actor, ActorRef} class AMQPConsumerManualAcknowledgeTestIntegration extends JUnitSuite with MustMatchers { diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualRejectTestIntegration.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerManualRejectTestIntegration.scala similarity index 86% rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualRejectTestIntegration.scala rename to akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerManualRejectTestIntegration.scala index ecdb6fb785..69e5aa8aa9 100644 --- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualRejectTestIntegration.scala +++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerManualRejectTestIntegration.scala @@ -1,18 +1,18 @@ -package se.scalablesolutions.akka.amqp.test +package akka.amqp.test /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -import se.scalablesolutions.akka.actor.Actor._ +import akka.actor.Actor._ import org.scalatest.matchers.MustMatchers -import se.scalablesolutions.akka.amqp._ +import akka.amqp._ import org.junit.Test import java.util.concurrent.{CountDownLatch, TimeUnit} -import se.scalablesolutions.akka.amqp.AMQP.{ExchangeParameters, ConsumerParameters, ChannelParameters, ProducerParameters} +import akka.amqp.AMQP.{ExchangeParameters, ConsumerParameters, ChannelParameters, ProducerParameters} import org.multiverse.api.latches.StandardLatch import org.scalatest.junit.JUnitSuite -import se.scalablesolutions.akka.actor.{Actor, ActorRef} +import akka.actor.{Actor, ActorRef} class AMQPConsumerManualRejectTestIntegration extends JUnitSuite with MustMatchers { diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerMessageTestIntegration.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerMessageTestIntegration.scala similarity index 83% rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerMessageTestIntegration.scala rename to akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerMessageTestIntegration.scala index 5b057d25b2..820edf8624 100644 --- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerMessageTestIntegration.scala +++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerMessageTestIntegration.scala @@ -1,18 +1,18 @@ -package se.scalablesolutions.akka.amqp.test +package akka.amqp.test /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -import se.scalablesolutions.akka.amqp._ +import akka.amqp._ import org.multiverse.api.latches.StandardLatch -import se.scalablesolutions.akka.actor.Actor._ +import akka.actor.Actor._ import org.scalatest.matchers.MustMatchers import java.util.concurrent.{CountDownLatch, TimeUnit} -import se.scalablesolutions.akka.amqp.AMQP.{ExchangeParameters, ConsumerParameters, ChannelParameters, ProducerParameters} +import akka.amqp.AMQP.{ExchangeParameters, ConsumerParameters, ChannelParameters, ProducerParameters} import org.scalatest.junit.JUnitSuite import org.junit.Test -import se.scalablesolutions.akka.actor.Actor +import akka.actor.Actor class AMQPConsumerMessageTestIntegration extends JUnitSuite with MustMatchers { diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerChannelRecoveryTestIntegration.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPProducerChannelRecoveryTestIntegration.scala similarity index 86% rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerChannelRecoveryTestIntegration.scala rename to akka-amqp/src/test/scala/akka/amqp/test/AMQPProducerChannelRecoveryTestIntegration.scala index c51c134e40..4b64f946e0 100644 --- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerChannelRecoveryTestIntegration.scala +++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPProducerChannelRecoveryTestIntegration.scala @@ -1,16 +1,16 @@ -package se.scalablesolutions.akka.amqp.test +package akka.amqp.test /** * Copyright (C) 2009-2010 Scalable Solutions AB */ import java.util.concurrent.TimeUnit -import se.scalablesolutions.akka.actor.{Actor, ActorRef} +import akka.actor.{Actor, ActorRef} import org.multiverse.api.latches.StandardLatch import com.rabbitmq.client.ShutdownSignalException -import se.scalablesolutions.akka.amqp._ +import akka.amqp._ import org.scalatest.matchers.MustMatchers -import se.scalablesolutions.akka.amqp.AMQP.{ExchangeParameters, ChannelParameters, ProducerParameters, ConnectionParameters} +import akka.amqp.AMQP.{ExchangeParameters, ChannelParameters, ProducerParameters, ConnectionParameters} import org.scalatest.junit.JUnitSuite import org.junit.Test diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerConnectionRecoveryTestIntegration.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPProducerConnectionRecoveryTestIntegration.scala similarity index 86% rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerConnectionRecoveryTestIntegration.scala rename to akka-amqp/src/test/scala/akka/amqp/test/AMQPProducerConnectionRecoveryTestIntegration.scala index 253a55f10b..00ac02e1c4 100644 --- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerConnectionRecoveryTestIntegration.scala +++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPProducerConnectionRecoveryTestIntegration.scala @@ -1,16 +1,16 @@ -package se.scalablesolutions.akka.amqp.test +package akka.amqp.test /** * Copyright (C) 2009-2010 Scalable Solutions AB */ import java.util.concurrent.TimeUnit -import se.scalablesolutions.akka.actor.{Actor, ActorRef} +import akka.actor.{Actor, ActorRef} import org.multiverse.api.latches.StandardLatch import com.rabbitmq.client.ShutdownSignalException -import se.scalablesolutions.akka.amqp._ +import akka.amqp._ import org.scalatest.matchers.MustMatchers -import se.scalablesolutions.akka.amqp.AMQP.{ExchangeParameters, ChannelParameters, ProducerParameters, ConnectionParameters} +import akka.amqp.AMQP.{ExchangeParameters, ChannelParameters, ProducerParameters, ConnectionParameters} import org.scalatest.junit.JUnitSuite import org.junit.Test diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerMessageTestIntegration.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPProducerMessageTestIntegration.scala similarity index 85% rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerMessageTestIntegration.scala rename to akka-amqp/src/test/scala/akka/amqp/test/AMQPProducerMessageTestIntegration.scala index dfc3ccd294..037af3e179 100644 --- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerMessageTestIntegration.scala +++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPProducerMessageTestIntegration.scala @@ -1,18 +1,18 @@ -package se.scalablesolutions.akka.amqp.test +package akka.amqp.test /** * Copyright (C) 2009-2010 Scalable Solutions AB */ import java.util.concurrent.TimeUnit -import se.scalablesolutions.akka.actor.ActorRef +import akka.actor.ActorRef import org.multiverse.api.latches.StandardLatch -import se.scalablesolutions.akka.amqp._ +import akka.amqp._ import com.rabbitmq.client.ReturnListener import com.rabbitmq.client.AMQP.BasicProperties import java.lang.String import org.scalatest.matchers.MustMatchers -import se.scalablesolutions.akka.amqp.AMQP.{ExchangeParameters, ProducerParameters} +import akka.amqp.AMQP.{ExchangeParameters, ProducerParameters} import org.scalatest.junit.JUnitSuite import org.junit.Test diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProtobufProducerConsumerTestIntegration.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPProtobufProducerConsumerTestIntegration.scala similarity index 86% rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProtobufProducerConsumerTestIntegration.scala rename to akka-amqp/src/test/scala/akka/amqp/test/AMQPProtobufProducerConsumerTestIntegration.scala index 1502d1e40b..668db09c78 100644 --- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProtobufProducerConsumerTestIntegration.scala +++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPProtobufProducerConsumerTestIntegration.scala @@ -1,16 +1,16 @@ -package se.scalablesolutions.akka.amqp.test +package akka.amqp.test /** * Copyright (C) 2009-2010 Scalable Solutions AB */ import org.scalatest.matchers.MustMatchers import org.scalatest.junit.JUnitSuite -import se.scalablesolutions.akka.amqp.AMQP +import akka.amqp.AMQP import org.junit.Test import org.multiverse.api.latches.StandardLatch import java.util.concurrent.TimeUnit -import se.scalablesolutions.akka.amqp.rpc.RPC -import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol +import akka.amqp.rpc.RPC +import akka.remote.protocol.RemoteProtocol.AddressProtocol class AMQPProtobufProducerConsumerTestIntegration extends JUnitSuite with MustMatchers { diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcClientServerTestIntegration.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPRpcClientServerTestIntegration.scala similarity index 89% rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcClientServerTestIntegration.scala rename to akka-amqp/src/test/scala/akka/amqp/test/AMQPRpcClientServerTestIntegration.scala index 0e9b6a152d..c8cea4bf7c 100644 --- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcClientServerTestIntegration.scala +++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPRpcClientServerTestIntegration.scala @@ -1,19 +1,19 @@ -package se.scalablesolutions.akka.amqp.test +package akka.amqp.test /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -import se.scalablesolutions.akka.amqp._ +import akka.amqp._ import rpc.RPC import rpc.RPC.{RpcClientSerializer, RpcServerSerializer} -import se.scalablesolutions.akka.actor.Actor._ +import akka.actor.Actor._ import org.scalatest.matchers.MustMatchers import java.util.concurrent.{CountDownLatch, TimeUnit} -import se.scalablesolutions.akka.amqp.AMQP._ +import akka.amqp.AMQP._ import org.scalatest.junit.JUnitSuite import org.junit.Test -import se.scalablesolutions.akka.actor.Actor +import akka.actor.Actor class AMQPRpcClientServerTestIntegration extends JUnitSuite with MustMatchers { diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcProtobufTestIntegration.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPRpcProtobufTestIntegration.scala similarity index 86% rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcProtobufTestIntegration.scala rename to akka-amqp/src/test/scala/akka/amqp/test/AMQPRpcProtobufTestIntegration.scala index 1e4383e8d7..dddc2f8432 100644 --- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcProtobufTestIntegration.scala +++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPRpcProtobufTestIntegration.scala @@ -1,14 +1,14 @@ -package se.scalablesolutions.akka.amqp.test +package akka.amqp.test /** * Copyright (C) 2009-2010 Scalable Solutions AB */ import org.scalatest.matchers.MustMatchers import org.scalatest.junit.JUnitSuite -import se.scalablesolutions.akka.amqp.AMQP -import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol +import akka.amqp.AMQP +import akka.remote.protocol.RemoteProtocol.AddressProtocol import org.junit.Test -import se.scalablesolutions.akka.amqp.rpc.RPC +import akka.amqp.rpc.RPC import org.multiverse.api.latches.StandardLatch import java.util.concurrent.TimeUnit diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcStringTestIntegration.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPRpcStringTestIntegration.scala similarity index 89% rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcStringTestIntegration.scala rename to akka-amqp/src/test/scala/akka/amqp/test/AMQPRpcStringTestIntegration.scala index fb36af74ab..1610757ab5 100644 --- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcStringTestIntegration.scala +++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPRpcStringTestIntegration.scala @@ -1,13 +1,13 @@ -package se.scalablesolutions.akka.amqp.test +package akka.amqp.test /** * Copyright (C) 2009-2010 Scalable Solutions AB */ import org.scalatest.matchers.MustMatchers import org.scalatest.junit.JUnitSuite -import se.scalablesolutions.akka.amqp.AMQP +import akka.amqp.AMQP import org.junit.Test -import se.scalablesolutions.akka.amqp.rpc.RPC +import akka.amqp.rpc.RPC import org.multiverse.api.latches.StandardLatch import java.util.concurrent.TimeUnit diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPStringProducerConsumerTestIntegration.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPStringProducerConsumerTestIntegration.scala similarity index 89% rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPStringProducerConsumerTestIntegration.scala rename to akka-amqp/src/test/scala/akka/amqp/test/AMQPStringProducerConsumerTestIntegration.scala index a9de971815..972fd0917d 100644 --- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPStringProducerConsumerTestIntegration.scala +++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPStringProducerConsumerTestIntegration.scala @@ -1,15 +1,15 @@ -package se.scalablesolutions.akka.amqp.test +package akka.amqp.test /** * Copyright (C) 2009-2010 Scalable Solutions AB */ import org.scalatest.matchers.MustMatchers import org.scalatest.junit.JUnitSuite -import se.scalablesolutions.akka.amqp.AMQP +import akka.amqp.AMQP import org.junit.Test import org.multiverse.api.latches.StandardLatch import java.util.concurrent.TimeUnit -import se.scalablesolutions.akka.amqp.rpc.RPC +import akka.amqp.rpc.RPC class AMQPStringProducerConsumerTestIntegration extends JUnitSuite with MustMatchers { diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPTest.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPTest.scala similarity index 77% rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPTest.scala rename to akka-amqp/src/test/scala/akka/amqp/test/AMQPTest.scala index 2a35df0a77..b9415c929a 100644 --- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPTest.scala +++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPTest.scala @@ -2,9 +2,9 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.amqp.test +package akka.amqp.test -import se.scalablesolutions.akka.amqp.AMQP +import akka.amqp.AMQP object AMQPTest { diff --git a/akka-camel/src/main/java/se/scalablesolutions/akka/camel/consume.java b/akka-camel/src/main/java/akka/camel/consume.java similarity index 79% rename from akka-camel/src/main/java/se/scalablesolutions/akka/camel/consume.java rename to akka-camel/src/main/java/akka/camel/consume.java index e2ea003894..90faa14372 100644 --- a/akka-camel/src/main/java/se/scalablesolutions/akka/camel/consume.java +++ b/akka-camel/src/main/java/akka/camel/consume.java @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.camel; +package akka.camel; import java.lang.annotation.ElementType; import java.lang.annotation.Retention; @@ -10,7 +10,7 @@ import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; /** - * Annotation used by implementations of {@link se.scalablesolutions.akka.actor.TypedActor} + * Annotation used by implementations of {@link akka.actor.TypedActor} * (on method-level) to define consumer endpoints. * * @author Martin Krasser diff --git a/akka-camel/src/main/resources/META-INF/services/org/apache/camel/component/actor b/akka-camel/src/main/resources/META-INF/services/org/apache/camel/component/actor index a2141db8a9..386928c5a8 100644 --- a/akka-camel/src/main/resources/META-INF/services/org/apache/camel/component/actor +++ b/akka-camel/src/main/resources/META-INF/services/org/apache/camel/component/actor @@ -1 +1 @@ -class=se.scalablesolutions.akka.camel.component.ActorComponent \ No newline at end of file +class=akka.camel.component.ActorComponent \ No newline at end of file diff --git a/akka-camel/src/main/resources/META-INF/services/org/apache/camel/component/typed-actor b/akka-camel/src/main/resources/META-INF/services/org/apache/camel/component/typed-actor index e004d887b3..02efe457e6 100644 --- a/akka-camel/src/main/resources/META-INF/services/org/apache/camel/component/typed-actor +++ b/akka-camel/src/main/resources/META-INF/services/org/apache/camel/component/typed-actor @@ -1 +1 @@ -class=se.scalablesolutions.akka.camel.component.TypedActorComponent \ No newline at end of file +class=akka.camel.component.TypedActorComponent \ No newline at end of file diff --git a/akka-camel/src/main/scala/CamelContextLifecycle.scala b/akka-camel/src/main/scala/CamelContextLifecycle.scala index 3996cd7baf..69f513edfc 100644 --- a/akka-camel/src/main/scala/CamelContextLifecycle.scala +++ b/akka-camel/src/main/scala/CamelContextLifecycle.scala @@ -2,16 +2,16 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.camel +package akka.camel import java.util.Map import org.apache.camel.{ProducerTemplate, CamelContext} import org.apache.camel.impl.DefaultCamelContext -import se.scalablesolutions.akka.camel.component.TypedActorComponent -import se.scalablesolutions.akka.japi.{Option => JOption} -import se.scalablesolutions.akka.util.Logging +import akka.camel.component.TypedActorComponent +import akka.japi.{Option => JOption} +import akka.util.Logging /** * Manages the lifecycle of a CamelContext. Allowed transitions are diff --git a/akka-camel/src/main/scala/CamelService.scala b/akka-camel/src/main/scala/CamelService.scala index b8aaacfe5b..cf2183da0f 100644 --- a/akka-camel/src/main/scala/CamelService.scala +++ b/akka-camel/src/main/scala/CamelService.scala @@ -1,17 +1,17 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.camel +package akka.camel import java.util.concurrent.CountDownLatch import org.apache.camel.CamelContext -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.actor.{AspectInitRegistry, ActorRegistry} -import se.scalablesolutions.akka.config.Config._ -import se.scalablesolutions.akka.japi.{Option => JOption} -import se.scalablesolutions.akka.util.{Logging, Bootable} +import akka.actor.Actor._ +import akka.actor.{AspectInitRegistry, ActorRegistry} +import akka.config.Config._ +import akka.japi.{Option => JOption} +import akka.util.{Logging, Bootable} /** * Publishes (untyped) consumer actors and typed consumer actors via Camel endpoints. Actors diff --git a/akka-camel/src/main/scala/Consumer.scala b/akka-camel/src/main/scala/Consumer.scala index db04c46abf..8b5d7971dc 100644 --- a/akka-camel/src/main/scala/Consumer.scala +++ b/akka-camel/src/main/scala/Consumer.scala @@ -2,9 +2,9 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.camel +package akka.camel -import se.scalablesolutions.akka.actor._ +import akka.actor._ import java.net.InetSocketAddress diff --git a/akka-camel/src/main/scala/ConsumerPublisher.scala b/akka-camel/src/main/scala/ConsumerPublisher.scala index f1bb5d7ab3..7b7ac491a9 100644 --- a/akka-camel/src/main/scala/ConsumerPublisher.scala +++ b/akka-camel/src/main/scala/ConsumerPublisher.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.camel +package akka.camel import collection.mutable.ListBuffer @@ -11,9 +11,9 @@ import java.util.concurrent.CountDownLatch import org.apache.camel.builder.RouteBuilder -import se.scalablesolutions.akka.actor._ -import se.scalablesolutions.akka.camel.component.TypedActorComponent -import se.scalablesolutions.akka.util.Logging +import akka.actor._ +import akka.camel.component.TypedActorComponent +import akka.util.Logging /** * @author Martin Krasser @@ -64,10 +64,10 @@ private[camel] object ConsumerPublisher extends Logging { * Actor that publishes consumer actors and typed actor methods at Camel endpoints. * The Camel context used for publishing is obtained via CamelContextManager.context. * This actor accepts messages of type - * se.scalablesolutions.akka.camel.ConsumerRegistered, - * se.scalablesolutions.akka.camel.ConsumerUnregistered, - * se.scalablesolutions.akka.camel.ConsumerMethodRegistered and - * se.scalablesolutions.akka.camel.ConsumerMethodUnregistered. + * akka.camel.ConsumerRegistered, + * akka.camel.ConsumerUnregistered, + * akka.camel.ConsumerMethodRegistered and + * akka.camel.ConsumerMethodUnregistered. * * @author Martin Krasser */ diff --git a/akka-camel/src/main/scala/Message.scala b/akka-camel/src/main/scala/Message.scala index 55fc4e58f9..aa1fcbd083 100644 --- a/akka-camel/src/main/scala/Message.scala +++ b/akka-camel/src/main/scala/Message.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.camel +package akka.camel import java.util.{Map => JMap, Set => JSet} @@ -10,7 +10,7 @@ import scala.collection.JavaConversions._ import org.apache.camel.{Exchange, Message => CamelMessage} import org.apache.camel.util.ExchangeHelper -import se.scalablesolutions.akka.japi.{Function => JFunction} +import akka.japi.{Function => JFunction} /** * An immutable representation of a Camel message. diff --git a/akka-camel/src/main/scala/Producer.scala b/akka-camel/src/main/scala/Producer.scala index 2924590c9e..ae23ae8c4e 100644 --- a/akka-camel/src/main/scala/Producer.scala +++ b/akka-camel/src/main/scala/Producer.scala @@ -2,14 +2,14 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.camel +package akka.camel import CamelMessageConversion.toExchangeAdapter import org.apache.camel._ import org.apache.camel.processor.SendProcessor -import se.scalablesolutions.akka.actor.{Actor, ActorRef, UntypedActor} +import akka.actor.{Actor, ActorRef, UntypedActor} /** * Support trait for producing messages to Camel endpoints. diff --git a/akka-camel/src/main/scala/component/ActorComponent.scala b/akka-camel/src/main/scala/component/ActorComponent.scala index 3fd4ebf055..81b5c49cff 100644 --- a/akka-camel/src/main/scala/component/ActorComponent.scala +++ b/akka-camel/src/main/scala/component/ActorComponent.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.camel.component +package akka.camel.component import java.net.InetSocketAddress import java.util.{Map => JMap} @@ -12,11 +12,11 @@ import java.util.concurrent.atomic.AtomicReference import org.apache.camel._ import org.apache.camel.impl.{DefaultProducer, DefaultEndpoint, DefaultComponent} -import se.scalablesolutions.akka.actor._ -import se.scalablesolutions.akka.camel.{Failure, Message} -import se.scalablesolutions.akka.camel.CamelMessageConversion.toExchangeAdapter -import se.scalablesolutions.akka.dispatch.{CompletableFuture, MessageInvocation, MessageDispatcher} -import se.scalablesolutions.akka.stm.TransactionConfig +import akka.actor._ +import akka.camel.{Failure, Message} +import akka.camel.CamelMessageConversion.toExchangeAdapter +import akka.dispatch.{CompletableFuture, MessageInvocation, MessageDispatcher} +import akka.stm.TransactionConfig import scala.reflect.BeanProperty @@ -33,8 +33,8 @@ object ActorComponent { /** * Camel component for sending messages to and receiving replies from (untyped) actors. * - * @see se.scalablesolutions.akka.camel.component.ActorEndpoint - * @see se.scalablesolutions.akka.camel.component.ActorProducer + * @see akka.camel.component.ActorEndpoint + * @see akka.camel.component.ActorProducer * * @author Martin Krasser */ @@ -69,8 +69,8 @@ class ActorComponent extends DefaultComponent { * If the URI contains an actor identifier, a message with a CamelActorIdentifier * header overrides the identifier in the endpoint URI. * - * @see se.scalablesolutions.akka.camel.component.ActorComponent - * @see se.scalablesolutions.akka.camel.component.ActorProducer + * @see akka.camel.component.ActorComponent + * @see akka.camel.component.ActorProducer * @author Martin Krasser */ @@ -118,8 +118,8 @@ class ActorEndpoint(uri: String, * ! operator. * * - * @see se.scalablesolutions.akka.camel.component.ActorComponent - * @see se.scalablesolutions.akka.camel.component.ActorEndpoint + * @see akka.camel.component.ActorComponent + * @see akka.camel.component.ActorEndpoint * * @author Martin Krasser */ diff --git a/akka-camel/src/main/scala/component/TypedActorComponent.scala b/akka-camel/src/main/scala/component/TypedActorComponent.scala index f172cc808b..f4a7f1b099 100644 --- a/akka-camel/src/main/scala/component/TypedActorComponent.scala +++ b/akka-camel/src/main/scala/component/TypedActorComponent.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.camel.component +package akka.camel.component import java.util.Map import java.util.concurrent.ConcurrentHashMap @@ -36,7 +36,7 @@ class TypedActorComponent extends BeanComponent { * bean holder that uses typedActorRegistry for getting access to typed * actors (beans). * - * @see se.scalablesolutions.akka.camel.component.TypedActorHolder + * @see akka.camel.component.TypedActorHolder */ override def createEndpoint(uri: String, remaining: String, parameters: Map[String, AnyRef]) = { val endpoint = new BeanEndpoint(uri, this) @@ -60,7 +60,7 @@ class TypedActorHolder(typedActorRegistry: Map[String, AnyRef], context: CamelCo extends RegistryBean(context, name) { /** - * Returns an se.scalablesolutions.akka.camel.component.TypedActorInfo instance. + * Returns an akka.camel.component.TypedActorInfo instance. */ override def getBeanInfo: BeanInfo = new TypedActorInfo(getContext, getBean.getClass, getParameterMappingStrategy) diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/MessageJavaTestBase.java b/akka-camel/src/test/java/akka/camel/MessageJavaTestBase.java similarity index 95% rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/MessageJavaTestBase.java rename to akka-camel/src/test/java/akka/camel/MessageJavaTestBase.java index 0db2f2b214..38e0b95692 100644 --- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/MessageJavaTestBase.java +++ b/akka-camel/src/test/java/akka/camel/MessageJavaTestBase.java @@ -1,12 +1,12 @@ -package se.scalablesolutions.akka.camel; +package akka.camel; import org.apache.camel.NoTypeConversionAvailableException; import org.junit.BeforeClass; import org.junit.Test; -import se.scalablesolutions.akka.camel.CamelContextManager; -import se.scalablesolutions.akka.camel.Message; -import se.scalablesolutions.akka.japi.Function; +import akka.camel.CamelContextManager; +import akka.camel.Message; +import akka.japi.Function; import java.io.InputStream; import java.util.*; diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteTypedConsumer.java b/akka-camel/src/test/java/akka/camel/SampleRemoteTypedConsumer.java similarity index 64% rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteTypedConsumer.java rename to akka-camel/src/test/java/akka/camel/SampleRemoteTypedConsumer.java index 5fd39f07d9..41a3c3f057 100644 --- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteTypedConsumer.java +++ b/akka-camel/src/test/java/akka/camel/SampleRemoteTypedConsumer.java @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.camel; +package akka.camel; -import se.scalablesolutions.akka.camel.consume; +import akka.camel.consume; /** * @author Martin Krasser diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteTypedConsumerImpl.java b/akka-camel/src/test/java/akka/camel/SampleRemoteTypedConsumerImpl.java similarity index 72% rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteTypedConsumerImpl.java rename to akka-camel/src/test/java/akka/camel/SampleRemoteTypedConsumerImpl.java index f6b0076e73..d7fb463b44 100644 --- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteTypedConsumerImpl.java +++ b/akka-camel/src/test/java/akka/camel/SampleRemoteTypedConsumerImpl.java @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.camel; +package akka.camel; -import se.scalablesolutions.akka.actor.TypedActor; +import akka.actor.TypedActor; /** * @author Martin Krasser diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteUntypedConsumer.java b/akka-camel/src/test/java/akka/camel/SampleRemoteUntypedConsumer.java similarity index 86% rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteUntypedConsumer.java rename to akka-camel/src/test/java/akka/camel/SampleRemoteUntypedConsumer.java index f9cc4ac71c..85ccb2638b 100644 --- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteUntypedConsumer.java +++ b/akka-camel/src/test/java/akka/camel/SampleRemoteUntypedConsumer.java @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.camel; +package akka.camel; -import se.scalablesolutions.akka.camel.RemoteUntypedConsumerActor; +import akka.camel.RemoteUntypedConsumerActor; /** * @author Martin Krasser diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedActor.java b/akka-camel/src/test/java/akka/camel/SampleTypedActor.java similarity index 72% rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedActor.java rename to akka-camel/src/test/java/akka/camel/SampleTypedActor.java index e335e08b84..798d07a66c 100644 --- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedActor.java +++ b/akka-camel/src/test/java/akka/camel/SampleTypedActor.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.camel; +package akka.camel; /** * @author Martin Krasser diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedActorImpl.java b/akka-camel/src/test/java/akka/camel/SampleTypedActorImpl.java similarity index 69% rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedActorImpl.java rename to akka-camel/src/test/java/akka/camel/SampleTypedActorImpl.java index aeb0878af7..773e3ec3ec 100644 --- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedActorImpl.java +++ b/akka-camel/src/test/java/akka/camel/SampleTypedActorImpl.java @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.camel; +package akka.camel; -import se.scalablesolutions.akka.actor.TypedActor; +import akka.actor.TypedActor; /** * @author Martin Krasser diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedConsumer.java b/akka-camel/src/test/java/akka/camel/SampleTypedConsumer.java similarity index 83% rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedConsumer.java rename to akka-camel/src/test/java/akka/camel/SampleTypedConsumer.java index a6a695f8d0..26283d8e61 100644 --- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedConsumer.java +++ b/akka-camel/src/test/java/akka/camel/SampleTypedConsumer.java @@ -1,9 +1,9 @@ -package se.scalablesolutions.akka.camel; +package akka.camel; import org.apache.camel.Body; import org.apache.camel.Header; -import se.scalablesolutions.akka.camel.consume; +import akka.camel.consume; /** * @author Martin Krasser diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedConsumerImpl.java b/akka-camel/src/test/java/akka/camel/SampleTypedConsumerImpl.java similarity index 86% rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedConsumerImpl.java rename to akka-camel/src/test/java/akka/camel/SampleTypedConsumerImpl.java index 4fa00f2da0..3bbe7a9442 100644 --- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedConsumerImpl.java +++ b/akka-camel/src/test/java/akka/camel/SampleTypedConsumerImpl.java @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.camel; +package akka.camel; -import se.scalablesolutions.akka.actor.TypedActor; +import akka.actor.TypedActor; /** * @author Martin Krasser diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedSingleConsumer.java b/akka-camel/src/test/java/akka/camel/SampleTypedSingleConsumer.java similarity index 61% rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedSingleConsumer.java rename to akka-camel/src/test/java/akka/camel/SampleTypedSingleConsumer.java index 1ef6614020..ff0b7bc715 100644 --- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedSingleConsumer.java +++ b/akka-camel/src/test/java/akka/camel/SampleTypedSingleConsumer.java @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.camel; +package akka.camel; -import se.scalablesolutions.akka.camel.consume; +import akka.camel.consume; /** * @author Martin Krasser diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedSingleConsumerImpl.java b/akka-camel/src/test/java/akka/camel/SampleTypedSingleConsumerImpl.java similarity index 66% rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedSingleConsumerImpl.java rename to akka-camel/src/test/java/akka/camel/SampleTypedSingleConsumerImpl.java index 608a74d5e9..27fbfdaa0d 100644 --- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedSingleConsumerImpl.java +++ b/akka-camel/src/test/java/akka/camel/SampleTypedSingleConsumerImpl.java @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.camel; +package akka.camel; -import se.scalablesolutions.akka.actor.TypedActor; +import akka.actor.TypedActor; /** * @author Martin Krasser diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedActor.java b/akka-camel/src/test/java/akka/camel/SampleUntypedActor.java similarity index 60% rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedActor.java rename to akka-camel/src/test/java/akka/camel/SampleUntypedActor.java index d8cb1dd1b9..b899b01447 100644 --- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedActor.java +++ b/akka-camel/src/test/java/akka/camel/SampleUntypedActor.java @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.camel; +package akka.camel; -import se.scalablesolutions.akka.actor.UntypedActor; +import akka.actor.UntypedActor; /** * @author Martin Krasser diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedConsumer.java b/akka-camel/src/test/java/akka/camel/SampleUntypedConsumer.java similarity index 82% rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedConsumer.java rename to akka-camel/src/test/java/akka/camel/SampleUntypedConsumer.java index 60dbe453ef..99300836c1 100644 --- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedConsumer.java +++ b/akka-camel/src/test/java/akka/camel/SampleUntypedConsumer.java @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.camel; +package akka.camel; -import se.scalablesolutions.akka.camel.UntypedConsumerActor; +import akka.camel.UntypedConsumerActor; /** * @author Martin Krasser diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedConsumerBlocking.java b/akka-camel/src/test/java/akka/camel/SampleUntypedConsumerBlocking.java similarity index 93% rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedConsumerBlocking.java rename to akka-camel/src/test/java/akka/camel/SampleUntypedConsumerBlocking.java index 2bc19a6d08..b5b22a04ae 100644 --- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedConsumerBlocking.java +++ b/akka-camel/src/test/java/akka/camel/SampleUntypedConsumerBlocking.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.camel; +package akka.camel; /** * @author Martin Krasser diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedForwardingProducer.java b/akka-camel/src/test/java/akka/camel/SampleUntypedForwardingProducer.java similarity index 91% rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedForwardingProducer.java rename to akka-camel/src/test/java/akka/camel/SampleUntypedForwardingProducer.java index 84dfa8fb86..3161c0f2d8 100644 --- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedForwardingProducer.java +++ b/akka-camel/src/test/java/akka/camel/SampleUntypedForwardingProducer.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.camel; +package akka.camel; /** * @author Martin Krasser diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedReplyingProducer.java b/akka-camel/src/test/java/akka/camel/SampleUntypedReplyingProducer.java similarity index 82% rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedReplyingProducer.java rename to akka-camel/src/test/java/akka/camel/SampleUntypedReplyingProducer.java index cc3fbf110d..09b7b86502 100644 --- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedReplyingProducer.java +++ b/akka-camel/src/test/java/akka/camel/SampleUntypedReplyingProducer.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.camel; +package akka.camel; /** * @author Martin Krasser diff --git a/akka-camel/src/test/scala/CamelContextLifecycleTest.scala b/akka-camel/src/test/scala/CamelContextLifecycleTest.scala index 6e6889c295..910373738f 100644 --- a/akka-camel/src/test/scala/CamelContextLifecycleTest.scala +++ b/akka-camel/src/test/scala/CamelContextLifecycleTest.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.camel +package akka.camel import org.apache.camel.impl.{DefaultProducerTemplate, DefaultCamelContext} import org.junit.Test diff --git a/akka-camel/src/test/scala/CamelExchangeAdapterTest.scala b/akka-camel/src/test/scala/CamelExchangeAdapterTest.scala index 55a8fc98b1..3996179b5b 100644 --- a/akka-camel/src/test/scala/CamelExchangeAdapterTest.scala +++ b/akka-camel/src/test/scala/CamelExchangeAdapterTest.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.camel +package akka.camel import org.apache.camel.impl.{DefaultCamelContext, DefaultExchange} import org.apache.camel.ExchangePattern diff --git a/akka-camel/src/test/scala/CamelMessageAdapterTest.scala b/akka-camel/src/test/scala/CamelMessageAdapterTest.scala index ca9c9ba66d..0c20ae1c29 100644 --- a/akka-camel/src/test/scala/CamelMessageAdapterTest.scala +++ b/akka-camel/src/test/scala/CamelMessageAdapterTest.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.camel +package akka.camel import org.apache.camel.impl.DefaultMessage import org.junit.Test diff --git a/akka-camel/src/test/scala/CamelServiceManagerTest.scala b/akka-camel/src/test/scala/CamelServiceManagerTest.scala index 712ffec70b..48ab29c6b8 100644 --- a/akka-camel/src/test/scala/CamelServiceManagerTest.scala +++ b/akka-camel/src/test/scala/CamelServiceManagerTest.scala @@ -1,9 +1,9 @@ -package se.scalablesolutions.akka.camel +package akka.camel import org.scalatest.{BeforeAndAfterAll, WordSpec} import org.scalatest.matchers.MustMatchers -import se.scalablesolutions.akka.actor.ActorRegistry +import akka.actor.ActorRegistry /** * @author Martin Krasser diff --git a/akka-camel/src/test/scala/ConsumerRegisteredTest.scala b/akka-camel/src/test/scala/ConsumerRegisteredTest.scala index 787142d50a..d7bf5fc2c3 100644 --- a/akka-camel/src/test/scala/ConsumerRegisteredTest.scala +++ b/akka-camel/src/test/scala/ConsumerRegisteredTest.scala @@ -1,9 +1,9 @@ -package se.scalablesolutions.akka.camel +package akka.camel import org.junit.Test import org.scalatest.junit.JUnitSuite -import se.scalablesolutions.akka.actor.{Actor, UntypedActor} +import akka.actor.{Actor, UntypedActor} class ConsumerRegisteredTest extends JUnitSuite { import ConsumerRegisteredTest._ diff --git a/akka-camel/src/test/scala/ConsumerTest.scala b/akka-camel/src/test/scala/ConsumerTest.scala index 1a126e3d52..181e59ed52 100644 --- a/akka-camel/src/test/scala/ConsumerTest.scala +++ b/akka-camel/src/test/scala/ConsumerTest.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.camel +package akka.camel import java.util.concurrent.{TimeoutException, CountDownLatch, TimeUnit} @@ -7,8 +7,8 @@ import org.apache.camel.builder.RouteBuilder import org.scalatest.{BeforeAndAfterAll, WordSpec} import org.scalatest.matchers.MustMatchers -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.actor._ +import akka.actor.Actor._ +import akka.actor._ /** * @author Martin Krasser diff --git a/akka-camel/src/test/scala/MessageJavaTest.scala b/akka-camel/src/test/scala/MessageJavaTest.scala index 80accd7432..3c95887eb4 100644 --- a/akka-camel/src/test/scala/MessageJavaTest.scala +++ b/akka-camel/src/test/scala/MessageJavaTest.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.camel +package akka.camel import org.scalatest.junit.JUnitSuite diff --git a/akka-camel/src/test/scala/MessageScalaTest.scala b/akka-camel/src/test/scala/MessageScalaTest.scala index aedd9171cd..5f43db596b 100644 --- a/akka-camel/src/test/scala/MessageScalaTest.scala +++ b/akka-camel/src/test/scala/MessageScalaTest.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.camel +package akka.camel import java.io.InputStream diff --git a/akka-camel/src/test/scala/ProducerFeatureTest.scala b/akka-camel/src/test/scala/ProducerFeatureTest.scala index fefd6afbfe..66bcbe9186 100644 --- a/akka-camel/src/test/scala/ProducerFeatureTest.scala +++ b/akka-camel/src/test/scala/ProducerFeatureTest.scala @@ -1,12 +1,12 @@ -package se.scalablesolutions.akka.camel +package akka.camel import org.apache.camel.{Exchange, Processor} import org.apache.camel.builder.RouteBuilder import org.apache.camel.component.mock.MockEndpoint import org.scalatest.{GivenWhenThen, BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec} -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.actor.{ActorRef, Actor, ActorRegistry} +import akka.actor.Actor._ +import akka.actor.{ActorRef, Actor, ActorRegistry} class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach with GivenWhenThen { import ProducerFeatureTest._ diff --git a/akka-camel/src/test/scala/PublishRequestorTest.scala b/akka-camel/src/test/scala/PublishRequestorTest.scala index 7cfced57e1..30e2782132 100644 --- a/akka-camel/src/test/scala/PublishRequestorTest.scala +++ b/akka-camel/src/test/scala/PublishRequestorTest.scala @@ -1,13 +1,13 @@ -package se.scalablesolutions.akka.camel +package akka.camel import java.util.concurrent.{CountDownLatch, TimeUnit} import org.junit.{Before, After, Test} import org.scalatest.junit.JUnitSuite -import se.scalablesolutions.akka.actor._ -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.camel.support.{SetExpectedMessageCount => SetExpectedTestMessageCount, _} +import akka.actor._ +import akka.actor.Actor._ +import akka.camel.support.{SetExpectedMessageCount => SetExpectedTestMessageCount, _} class PublishRequestorTest extends JUnitSuite { import PublishRequestorTest._ diff --git a/akka-camel/src/test/scala/RemoteConsumerTest.scala b/akka-camel/src/test/scala/RemoteConsumerTest.scala index 2218aac25a..77a1d9e757 100644 --- a/akka-camel/src/test/scala/RemoteConsumerTest.scala +++ b/akka-camel/src/test/scala/RemoteConsumerTest.scala @@ -1,12 +1,12 @@ -package se.scalablesolutions.akka.camel +package akka.camel import java.util.concurrent.{CountDownLatch, TimeUnit} import org.scalatest.{GivenWhenThen, BeforeAndAfterAll, FeatureSpec} -import se.scalablesolutions.akka.actor._ -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer} +import akka.actor._ +import akka.actor.Actor._ +import akka.remote.{RemoteClient, RemoteServer} /** * @author Martin Krasser diff --git a/akka-camel/src/test/scala/UntypedProducerFeatureTest.scala b/akka-camel/src/test/scala/UntypedProducerFeatureTest.scala index 0d268785b6..18930f9ab4 100644 --- a/akka-camel/src/test/scala/UntypedProducerFeatureTest.scala +++ b/akka-camel/src/test/scala/UntypedProducerFeatureTest.scala @@ -1,12 +1,12 @@ -package se.scalablesolutions.akka.camel +package akka.camel import org.apache.camel.{Exchange, Processor} import org.apache.camel.builder.RouteBuilder import org.apache.camel.component.mock.MockEndpoint import org.scalatest.{GivenWhenThen, BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec} -import se.scalablesolutions.akka.actor.UntypedActor._ -import se.scalablesolutions.akka.actor.ActorRegistry +import akka.actor.UntypedActor._ +import akka.actor.ActorRegistry class UntypedProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach with GivenWhenThen { import UntypedProducerFeatureTest._ diff --git a/akka-camel/src/test/scala/component/ActorComponentFeatureTest.scala b/akka-camel/src/test/scala/component/ActorComponentFeatureTest.scala index cc9f750aae..993fe72096 100644 --- a/akka-camel/src/test/scala/component/ActorComponentFeatureTest.scala +++ b/akka-camel/src/test/scala/component/ActorComponentFeatureTest.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.camel.component +package akka.camel.component import java.util.concurrent.{TimeUnit, CountDownLatch} @@ -7,10 +7,10 @@ import org.apache.camel.builder.RouteBuilder import org.apache.camel.component.mock.MockEndpoint import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec} -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.actor.{ActorRegistry, Actor} -import se.scalablesolutions.akka.camel.{Failure, Message, CamelContextManager} -import se.scalablesolutions.akka.camel.support._ +import akka.actor.Actor._ +import akka.actor.{ActorRegistry, Actor} +import akka.camel.{Failure, Message, CamelContextManager} +import akka.camel.support._ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach { import ActorComponentFeatureTest._ diff --git a/akka-camel/src/test/scala/component/ActorComponentTest.scala b/akka-camel/src/test/scala/component/ActorComponentTest.scala index bc4a225f6a..0af9f00213 100644 --- a/akka-camel/src/test/scala/component/ActorComponentTest.scala +++ b/akka-camel/src/test/scala/component/ActorComponentTest.scala @@ -1,11 +1,11 @@ -package se.scalablesolutions.akka.camel.component +package akka.camel.component import org.apache.camel.{Endpoint, AsyncProcessor} import org.apache.camel.impl.DefaultCamelContext import org.junit._ import org.scalatest.junit.JUnitSuite -import se.scalablesolutions.akka.actor.uuidFrom +import akka.actor.uuidFrom class ActorComponentTest extends JUnitSuite { val component: ActorComponent = ActorComponentTest.actorComponent diff --git a/akka-camel/src/test/scala/component/ActorProducerTest.scala b/akka-camel/src/test/scala/component/ActorProducerTest.scala index bfbbf3e515..62ccf9cb0e 100644 --- a/akka-camel/src/test/scala/component/ActorProducerTest.scala +++ b/akka-camel/src/test/scala/component/ActorProducerTest.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.camel.component +package akka.camel.component import ActorComponentTest._ @@ -10,10 +10,10 @@ import org.junit.{After, Test} import org.scalatest.junit.JUnitSuite import org.scalatest.BeforeAndAfterAll -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.actor.ActorRegistry -import se.scalablesolutions.akka.camel.{Failure, Message} -import se.scalablesolutions.akka.camel.support._ +import akka.actor.Actor._ +import akka.actor.ActorRegistry +import akka.camel.{Failure, Message} +import akka.camel.support._ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll { import ActorProducerTest._ diff --git a/akka-camel/src/test/scala/component/TypedActorComponentFeatureTest.scala b/akka-camel/src/test/scala/component/TypedActorComponentFeatureTest.scala index e1f169187a..e6e1293a54 100644 --- a/akka-camel/src/test/scala/component/TypedActorComponentFeatureTest.scala +++ b/akka-camel/src/test/scala/component/TypedActorComponentFeatureTest.scala @@ -1,12 +1,12 @@ -package se.scalablesolutions.akka.camel.component +package akka.camel.component import org.apache.camel._ import org.apache.camel.builder.RouteBuilder import org.apache.camel.impl.{DefaultCamelContext, SimpleRegistry} import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec} -import se.scalablesolutions.akka.actor.{ActorRegistry, TypedActor} -import se.scalablesolutions.akka.camel._ +import akka.actor.{ActorRegistry, TypedActor} +import akka.camel._ /** * @author Martin Krasser diff --git a/akka-camel/src/test/scala/support/TestSupport.scala b/akka-camel/src/test/scala/support/TestSupport.scala index 8e1322e14f..4744d774f5 100644 --- a/akka-camel/src/test/scala/support/TestSupport.scala +++ b/akka-camel/src/test/scala/support/TestSupport.scala @@ -1,11 +1,11 @@ -package se.scalablesolutions.akka.camel.support +package akka.camel.support import java.util.concurrent.{TimeUnit, CountDownLatch} import collection.mutable.Buffer -import se.scalablesolutions.akka.camel.Message -import se.scalablesolutions.akka.actor.Actor +import akka.camel.Message +import akka.actor.Actor import TestSupport._ diff --git a/akka-http/src/main/scala/AkkaBroadcaster.scala b/akka-http/src/main/scala/AkkaBroadcaster.scala index 8aae04bc86..963058f052 100644 --- a/akka-http/src/main/scala/AkkaBroadcaster.scala +++ b/akka-http/src/main/scala/AkkaBroadcaster.scala @@ -2,13 +2,13 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.comet +package akka.comet import org.atmosphere.cpr.{AtmosphereResourceEvent, AtmosphereResource} -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.actor.Actor -import se.scalablesolutions.akka.dispatch.Dispatchers +import akka.actor.Actor._ +import akka.actor.Actor +import akka.dispatch.Dispatchers import org.atmosphere.jersey.util.JerseyBroadcasterUtil object AkkaBroadcaster { diff --git a/akka-http/src/main/scala/AkkaClusterBroadcastFilter.scala b/akka-http/src/main/scala/AkkaClusterBroadcastFilter.scala index 7ea963872f..373e238a60 100644 --- a/akka-http/src/main/scala/AkkaClusterBroadcastFilter.scala +++ b/akka-http/src/main/scala/AkkaClusterBroadcastFilter.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.comet +package akka.comet -import se.scalablesolutions.akka.actor.Actor -import se.scalablesolutions.akka.remote.Cluster +import akka.actor.Actor +import akka.remote.Cluster import scala.reflect.BeanProperty import org.atmosphere.cpr.{BroadcastFilter, ClusterBroadcastFilter, Broadcaster} diff --git a/akka-http/src/main/scala/AkkaCometServlet.scala b/akka-http/src/main/scala/AkkaCometServlet.scala index 6afb216c9b..5b15096c92 100644 --- a/akka-http/src/main/scala/AkkaCometServlet.scala +++ b/akka-http/src/main/scala/AkkaCometServlet.scala @@ -2,9 +2,9 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.comet +package akka.comet -import se.scalablesolutions.akka.util.Logging +import akka.util.Logging import java.util.{List => JList} import javax.servlet.{ServletConfig,ServletContext} @@ -43,7 +43,7 @@ class AtmosphereRestServlet extends ServletContainer with AtmosphereServletProce * Used by the Akka Kernel to bootstrap REST and Comet. */ class AkkaServlet extends AtmosphereServlet { - import se.scalablesolutions.akka.config.Config.{config => c} + import akka.config.Config.{config => c} /* * Configure Atmosphere and Jersey (default, fall-back values) diff --git a/akka-http/src/main/scala/AkkaLoader.scala b/akka-http/src/main/scala/AkkaLoader.scala index e7169e94d2..d8afac67bc 100644 --- a/akka-http/src/main/scala/AkkaLoader.scala +++ b/akka-http/src/main/scala/AkkaLoader.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.servlet +package akka.servlet -import se.scalablesolutions.akka.config.Config -import se.scalablesolutions.akka.util.{Logging, Bootable} +import akka.config.Config +import akka.util.{Logging, Bootable} /* * This class is responsible for booting up a stack of bundles and then shutting them down diff --git a/akka-http/src/main/scala/DefaultAkkaLoader.scala b/akka-http/src/main/scala/DefaultAkkaLoader.scala index 8fb7ed4e5b..f7ab1f074b 100644 --- a/akka-http/src/main/scala/DefaultAkkaLoader.scala +++ b/akka-http/src/main/scala/DefaultAkkaLoader.scala @@ -2,14 +2,14 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.http +package akka.http -import se.scalablesolutions.akka.config.Config -import se.scalablesolutions.akka.util.{Logging, Bootable} -import se.scalablesolutions.akka.camel.CamelService -import se.scalablesolutions.akka.remote.BootableRemoteActorService -import se.scalablesolutions.akka.actor.BootableActorLoaderService -import se.scalablesolutions.akka.servlet.AkkaLoader +import akka.config.Config +import akka.util.{Logging, Bootable} +import akka.camel.CamelService +import akka.remote.BootableRemoteActorService +import akka.actor.BootableActorLoaderService +import akka.servlet.AkkaLoader class DefaultAkkaLoader extends AkkaLoader { def boot(): Unit = boot(true, @@ -22,7 +22,7 @@ class DefaultAkkaLoader extends AkkaLoader { /** * Can be used to boot Akka * - * java -cp ... se.scalablesolutions.akka.http.Main + * java -cp ... akka.http.Main */ object Main extends DefaultAkkaLoader { def main(args: Array[String]) = boot diff --git a/akka-http/src/main/scala/EmbeddedAppServer.scala b/akka-http/src/main/scala/EmbeddedAppServer.scala index 580f3430db..db2b418ccb 100644 --- a/akka-http/src/main/scala/EmbeddedAppServer.scala +++ b/akka-http/src/main/scala/EmbeddedAppServer.scala @@ -2,15 +2,15 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.http +package akka.http 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 akka.actor.BootableActorLoaderService +import akka.util.{Bootable, Logging} +import akka.comet.AkkaServlet import org.eclipse.jetty.xml.XmlConfiguration import org.eclipse.jetty.server.{Handler, Server} @@ -22,7 +22,7 @@ import org.eclipse.jetty.server.handler.{HandlerList, HandlerCollection, Context trait EmbeddedAppServer extends Bootable with Logging { self : BootableActorLoaderService => - import se.scalablesolutions.akka.config.Config._ + import akka.config.Config._ val REST_HOSTNAME = config.getString("akka.rest.hostname", "localhost") val REST_PORT = config.getInt("akka.rest.port", 9998) diff --git a/akka-http/src/main/scala/Initializer.scala b/akka-http/src/main/scala/Initializer.scala index c1cd8bfc87..a470949821 100644 --- a/akka-http/src/main/scala/Initializer.scala +++ b/akka-http/src/main/scala/Initializer.scala @@ -2,13 +2,13 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.servlet +package akka.servlet -import se.scalablesolutions.akka.remote.BootableRemoteActorService -import se.scalablesolutions.akka.actor.BootableActorLoaderService -import se.scalablesolutions.akka.camel.CamelService -import se.scalablesolutions.akka.config.Config -import se.scalablesolutions.akka.util.{Logging, Bootable} +import akka.remote.BootableRemoteActorService +import akka.actor.BootableActorLoaderService +import akka.camel.CamelService +import akka.config.Config +import akka.util.{Logging, Bootable} import javax.servlet.{ServletContextListener, ServletContextEvent} @@ -18,7 +18,7 @@ import javax.servlet.{ServletContextListener, ServletContextEvent} * * ... * - * se.scalablesolutions.akka.servlet.Initializer + * akka.servlet.Initializer * * ... * diff --git a/akka-http/src/main/scala/ListWriter.scala b/akka-http/src/main/scala/ListWriter.scala index 6556004e38..3a2c69d02a 100644 --- a/akka-http/src/main/scala/ListWriter.scala +++ b/akka-http/src/main/scala/ListWriter.scala @@ -1,10 +1,10 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.rest +package akka.rest import java.io.OutputStream -import se.scalablesolutions.akka.serialization.Serializer +import akka.serialization.Serializer import javax.ws.rs.core.{MultivaluedMap, MediaType} import javax.ws.rs.ext.{MessageBodyWriter, Provider} import javax.ws.rs.Produces diff --git a/akka-http/src/main/scala/Security.scala b/akka-http/src/main/scala/Security.scala index 2db1e4981b..b6aec9a1f0 100644 --- a/akka-http/src/main/scala/Security.scala +++ b/akka-http/src/main/scala/Security.scala @@ -20,12 +20,12 @@ * All cred to the Lift team (www.liftweb.com), especially David Pollak and Tim Perrett */ -package se.scalablesolutions.akka.security +package akka.security -import se.scalablesolutions.akka.actor.{Scheduler, Actor, ActorRef, ActorRegistry, IllegalActorStateException} -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.config.Config -import se.scalablesolutions.akka.util.Logging +import akka.actor.{Scheduler, Actor, ActorRef, ActorRegistry, IllegalActorStateException} +import akka.actor.Actor._ +import akka.config.Config +import akka.util.Logging import com.sun.jersey.api.model.AbstractMethod import com.sun.jersey.spi.container.{ResourceFilterFactory, ContainerRequest, ContainerRequestFilter, ContainerResponse, ContainerResponseFilter, ResourceFilter} diff --git a/akka-http/src/test/scala/AllTest.scala b/akka-http/src/test/scala/AllTest.scala index 8bb7eb7923..0b473507dd 100644 --- a/akka-http/src/test/scala/AllTest.scala +++ b/akka-http/src/test/scala/AllTest.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.security +package akka.security import junit.framework.Test import junit.framework.TestCase diff --git a/akka-http/src/test/scala/SecuritySpec.scala b/akka-http/src/test/scala/SecuritySpec.scala index 5931df69d0..36069e50ad 100644 --- a/akka-http/src/test/scala/SecuritySpec.scala +++ b/akka-http/src/test/scala/SecuritySpec.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.security +package akka.security -import se.scalablesolutions.akka.config.Supervision._ -import se.scalablesolutions.akka.actor.Actor._ +import akka.config.Supervision._ +import akka.actor.Actor._ import org.scalatest.Suite import org.scalatest.junit.JUnitSuite diff --git a/akka-jta/src/main/scala/AtomikosTransactionService.scala b/akka-jta/src/main/scala/AtomikosTransactionService.scala index 4acbb1a013..f2af9c01bf 100644 --- a/akka-jta/src/main/scala/AtomikosTransactionService.scala +++ b/akka-jta/src/main/scala/AtomikosTransactionService.scala @@ -2,15 +2,15 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.jta +package akka.jta import javax.transaction.{TransactionManager, SystemException} import com.atomikos.icatch.jta.{J2eeTransactionManager, J2eeUserTransaction} import com.atomikos.icatch.config.{TSInitInfo, UserTransactionService, UserTransactionServiceImp} -import se.scalablesolutions.akka.config.Config._ -import se.scalablesolutions.akka.util.Duration +import akka.config.Config._ +import akka.util.Duration object AtomikosTransactionService extends AtomikosTransactionService diff --git a/akka-jta/src/main/scala/JTA.scala b/akka-jta/src/main/scala/JTA.scala index d3c1ade168..1f637fc17b 100644 --- a/akka-jta/src/main/scala/JTA.scala +++ b/akka-jta/src/main/scala/JTA.scala @@ -2,17 +2,17 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.jta +package akka.jta import javax.transaction.{TransactionManager, UserTransaction, Transaction => JtaTransaction, SystemException, Status, Synchronization, TransactionSynchronizationRegistry} import javax.naming.{InitialContext, Context, NamingException} -import se.scalablesolutions.akka.config.Config._ -import se.scalablesolutions.akka.util.Logging -import se.scalablesolutions.akka.stm.Transaction -import se.scalablesolutions.akka.AkkaException +import akka.config.Config._ +import akka.util.Logging +import akka.stm.Transaction +import akka.AkkaException class JtaConfigurationException(message: String) extends AkkaException(message) @@ -22,7 +22,7 @@ class JtaConfigurationException(message: String) extends AkkaException(message) * @author Jonas Bonér */ object TransactionContainer extends Logging { - val AKKA_JTA_TRANSACTION_SERVICE_CLASS = "se.scalablesolutions.akka.jta.AtomikosTransactionService" + val AKKA_JTA_TRANSACTION_SERVICE_CLASS = "akka.jta.AtomikosTransactionService" val DEFAULT_USER_TRANSACTION_NAME = "java:comp/UserTransaction" val FALLBACK_TRANSACTION_MANAGER_NAMES = "java:comp/TransactionManager" :: "java:appserver/TransactionManager" :: diff --git a/akka-jta/src/main/scala/TransactionContext.scala b/akka-jta/src/main/scala/TransactionContext.scala index 79d6f7366d..ca92e5aa75 100644 --- a/akka-jta/src/main/scala/TransactionContext.scala +++ b/akka-jta/src/main/scala/TransactionContext.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.jta +package akka.jta import javax.transaction.{Transaction, Status, TransactionManager, Synchronization} -import se.scalablesolutions.akka.util.Logging -import se.scalablesolutions.akka.config.Config._ +import akka.util.Logging +import akka.config.Config._ /** * The TransactionContext object manages the transactions. diff --git a/akka-jta/src/main/scala/TransactionProtocol.scala b/akka-jta/src/main/scala/TransactionProtocol.scala index 487dece483..11965df9e6 100644 --- a/akka-jta/src/main/scala/TransactionProtocol.scala +++ b/akka-jta/src/main/scala/TransactionProtocol.scala @@ -2,9 +2,9 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.jta +package akka.jta -import se.scalablesolutions.akka.util.Logging +import akka.util.Logging import java.util.{List => JList} import java.util.concurrent.CopyOnWriteArrayList diff --git a/akka-jta/src/test/scala/ReflectiveAccessSpec.scala b/akka-jta/src/test/scala/ReflectiveAccessSpec.scala index c43af4cd23..4666098f80 100644 --- a/akka-jta/src/test/scala/ReflectiveAccessSpec.scala +++ b/akka-jta/src/test/scala/ReflectiveAccessSpec.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.jta +package akka.jta import org.scalatest.junit.JUnitSuite import org.junit.Test -import se.scalablesolutions.akka.util.ReflectiveAccess +import akka.util.ReflectiveAccess class ReflectiveAccessSpec extends JUnitSuite { @Test def ensureReflectiveAccessCanLoadTransactionContainer { diff --git a/akka-karaf/akka-features/src/main/resources/features.xml b/akka-karaf/akka-features/src/main/resources/features.xml index 067e64f193..ad96d7bf05 100644 --- a/akka-karaf/akka-features/src/main/resources/features.xml +++ b/akka-karaf/akka-features/src/main/resources/features.xml @@ -8,15 +8,15 @@ - mvn:se.scalablesolutions.akka.akka-wrap/dispatch-json_2.8.0.RC3_osgi/0.7.4 + mvn:akka.akka-wrap/dispatch-json_2.8.0.RC3_osgi/0.7.4 mvn:org.objenesis/objenesis/1.2 mvn:sjson.json/sjson/0.6-SNAPSHOT sjson - mvn:se.scalablesolutions.akka.akka-wrap/jgroups-wrapper_2.8.0.RC3_osgi/2.9.0.GA + mvn:akka.akka-wrap/jgroups-wrapper_2.8.0.RC3_osgi/2.9.0.GA mvn:org.jboss.netty/netty/3.2.0.CR1 - mvn:se.scalablesolutions.akka/akka-remote_2.8.0.RC3_osgi/0.9 + mvn:akka/akka-remote_2.8.0.RC3_osgi/0.9 diff --git a/akka-kernel/src/main/scala/Kernel.scala b/akka-kernel/src/main/scala/Kernel.scala index d31163eb65..76d63728e6 100644 --- a/akka-kernel/src/main/scala/Kernel.scala +++ b/akka-kernel/src/main/scala/Kernel.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.kernel +package akka.kernel -import se.scalablesolutions.akka.http.{ EmbeddedAppServer, DefaultAkkaLoader } -import se.scalablesolutions.akka.remote.BootableRemoteActorService +import akka.http.{ EmbeddedAppServer, DefaultAkkaLoader } +import akka.remote.BootableRemoteActorService object Main { def main(args: Array[String]) = Kernel.boot diff --git a/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraSession.scala b/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraSession.scala index 34ef7eca62..b8474812ab 100644 --- a/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraSession.scala +++ b/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraSession.scala @@ -2,15 +2,15 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.cassandra +package akka.persistence.cassandra import java.io.{Flushable, Closeable} -import se.scalablesolutions.akka.persistence.common._ -import se.scalablesolutions.akka.util.Logging -import se.scalablesolutions.akka.util.Helpers._ -import se.scalablesolutions.akka.serialization.Serializer -import se.scalablesolutions.akka.config.Config.config +import akka.persistence.common._ +import akka.util.Logging +import akka.util.Helpers._ +import akka.serialization.Serializer +import akka.config.Config.config import scala.collection.mutable.Map @@ -27,8 +27,8 @@ trait CassandraSession extends Closeable with Flushable { import scala.collection.JavaConversions._ import java.util.{Map => JMap, List => JList} - protected val client: Cassandra.Client - protected val keyspace: String + val client: Cassandra.Client + val keyspace: String val obtainedAt: Long val consistencyLevel: ConsistencyLevel 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 b3867b9492..166ccbe676 100644 --- a/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorage.scala +++ b/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorage.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.cassandra +package akka.persistence.cassandra -import se.scalablesolutions.akka.stm._ -import se.scalablesolutions.akka.persistence.common._ -import se.scalablesolutions.akka.actor.{newUuid} +import akka.stm._ +import akka.persistence.common._ +import akka.actor.{newUuid} object CassandraStorage extends Storage { type ElementType = Array[Byte] @@ -14,14 +14,17 @@ object CassandraStorage extends Storage { def newMap: PersistentMap[ElementType, ElementType] = newMap(newUuid.toString) def newVector: PersistentVector[ElementType] = newVector(newUuid.toString) def newRef: PersistentRef[ElementType] = newRef(newUuid.toString) + override def newQueue: PersistentQueue[ElementType] = newQueue(newUuid.toString) def getMap(id: String): PersistentMap[ElementType, ElementType] = newMap(id) def getVector(id: String): PersistentVector[ElementType] = newVector(id) def getRef(id: String): PersistentRef[ElementType] = newRef(id) + override def getQueue(id: String): PersistentQueue[ElementType] = newQueue(id) def newMap(id: String): PersistentMap[ElementType, ElementType] = new CassandraPersistentMap(id) def newVector(id: String): PersistentVector[ElementType] = new CassandraPersistentVector(id) def newRef(id: String): PersistentRef[ElementType] = new CassandraPersistentRef(id) + override def newQueue(id: String): PersistentQueue[ElementType] = new CassandraPersistentQueue(id) } /** @@ -49,3 +52,8 @@ class CassandraPersistentRef(id: String) extends PersistentRef[Array[Byte]] { val uuid = id val storage = CassandraStorageBackend } + +class CassandraPersistentQueue(id: String) extends PersistentQueue[Array[Byte]] { + val uuid = id + val storage = CassandraStorageBackend +} diff --git a/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorageBackend.scala b/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorageBackend.scala index f9e456aeae..03cc0048f3 100644 --- a/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorageBackend.scala +++ b/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorageBackend.scala @@ -2,48 +2,52 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.cassandra +package akka.persistence.cassandra -import se.scalablesolutions.akka.stm._ -import se.scalablesolutions.akka.persistence.common._ -import se.scalablesolutions.akka.util.Logging -import se.scalablesolutions.akka.util.Helpers._ -import se.scalablesolutions.akka.config.Config.config +import akka.stm._ +import akka.persistence.common._ +import akka.util.Logging +import akka.util.Helpers._ +import akka.config.Config.config import org.apache.cassandra.thrift._ +import java.lang.String +import collection.JavaConversions +import collection.immutable.{TreeMap, Iterable} +import java.util.{Map => JMap, HashMap => JHMap, List => JList, ArrayList => JAList} /** * @author Jonas Bonér */ -private[akka] object CassandraStorageBackend extends - MapStorageBackend[Array[Byte], Array[Byte]] with - VectorStorageBackend[Array[Byte]] with - RefStorageBackend[Array[Byte]] with - Logging { + +private[akka] object CassandraStorageBackend extends CommonStorageBackend { + + import CommonStorageBackend._ type ElementType = Array[Byte] - val KEYSPACE = "akka" - val MAP_COLUMN_PARENT = new ColumnParent("map") + val KEYSPACE = "akka" + val MAP_COLUMN_PARENT = new ColumnParent("map") val VECTOR_COLUMN_PARENT = new ColumnParent("vector") - val REF_COLUMN_PARENT = new ColumnParent("ref") - val REF_KEY = "item".getBytes("UTF-8") - val EMPTY_BYTE_ARRAY = new Array[Byte](0) + val REF_COLUMN_PARENT = new ColumnParent("ref") + val QUEUE_COLUMN_PARENT = new ColumnParent("queue") + val REF_KEY = "item".getBytes("UTF-8") + val EMPTY_BYTE_ARRAY = new Array[Byte](0) val CASSANDRA_SERVER_HOSTNAME = config.getString("akka.storage.cassandra.hostname", "127.0.0.1") - val CASSANDRA_SERVER_PORT = config.getInt("akka.storage.cassandra.port", 9160) + val CASSANDRA_SERVER_PORT = config.getInt("akka.storage.cassandra.port", 9160) val CONSISTENCY_LEVEL = { config.getString("akka.storage.cassandra.consistency-level", "QUORUM") match { - case "ZERO" => ConsistencyLevel.ZERO - case "ONE" => ConsistencyLevel.ONE - case "QUORUM" => ConsistencyLevel.QUORUM - case "DCQUORUM" => ConsistencyLevel.DCQUORUM + case "ZERO" => ConsistencyLevel.ZERO + case "ONE" => ConsistencyLevel.ONE + case "QUORUM" => ConsistencyLevel.QUORUM + case "DCQUORUM" => ConsistencyLevel.DCQUORUM case "DCQUORUMSYNC" => ConsistencyLevel.DCQUORUMSYNC - case "ALL" => ConsistencyLevel.ALL - case "ANY" => ConsistencyLevel.ANY - case unknown => throw new IllegalArgumentException( + case "ALL" => ConsistencyLevel.ALL + case "ANY" => ConsistencyLevel.ANY + case unknown => throw new IllegalArgumentException( "Cassandra consistency level [" + unknown + "] is not supported." + - "\n\tExpected one of [ZERO, ONE, QUORUM, DCQUORUM, DCQUORUMSYNC, ALL, ANY] in the akka.conf configuration file.") + "\n\tExpected one of [ZERO, ONE, QUORUM, DCQUORUM, DCQUORUMSYNC, ALL, ANY] in the akka.conf configuration file.") } } val IS_ASCENDING = true @@ -57,185 +61,100 @@ private[akka] object CassandraStorageBackend extends protocol, CONSISTENCY_LEVEL) - // =============================================================== - // For Ref - // =============================================================== - def insertRefStorageFor(name: String, element: Array[Byte]) = { - val columnPath = new ColumnPath(REF_COLUMN_PARENT.getColumn_family) - columnPath.setColumn(REF_KEY) - sessions.withSession { - _ ++| (name, - columnPath, - element, - System.currentTimeMillis, - CONSISTENCY_LEVEL) + + + class CassandraAccess(parent: ColumnParent) extends CommonStorageBackendAccess { + + def path(key: Array[Byte]): ColumnPath = { + new ColumnPath(parent.getColumn_family).setColumn(key) } - } - def getRefStorageFor(name: String): Option[Array[Byte]] = { - val columnPath = new ColumnPath(REF_COLUMN_PARENT.getColumn_family) - columnPath.setColumn(REF_KEY) - try { - val column: Option[ColumnOrSuperColumn] = sessions.withSession { - _ | (name, columnPath) + def delete(owner: String, key: Array[Byte]) = { + sessions.withSession{ + session => { + session -- (owner, path(key), System.currentTimeMillis, CONSISTENCY_LEVEL) + } } - if (column.isDefined) Some(column.get.getColumn.value) - else None - } catch { - case e => None } - } - - // =============================================================== - // For Vector - // =============================================================== - - def insertVectorStorageEntryFor(name: String, element: Array[Byte]) = { - val columnPath = new ColumnPath(VECTOR_COLUMN_PARENT.getColumn_family) - columnPath.setColumn(intToBytes(getVectorStorageSizeFor(name))) - sessions.withSession { - _ ++| (name, - columnPath, - element, - System.currentTimeMillis, - CONSISTENCY_LEVEL) - } - } - - def insertVectorStorageEntriesFor(name: String, elements: List[Array[Byte]]) = - elements.foreach(insertVectorStorageEntryFor(name, _)) - - def updateVectorStorageEntryFor(name: String, index: Int, elem: Array[Byte]) = { - val columnPath = new ColumnPath(VECTOR_COLUMN_PARENT.getColumn_family) - columnPath.setColumn(intToBytes(index)) - sessions.withSession { - _ ++| (name, - columnPath, - elem, - System.currentTimeMillis, - CONSISTENCY_LEVEL) - } - } - - def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = { - val columnPath = new ColumnPath(VECTOR_COLUMN_PARENT.getColumn_family) - columnPath.setColumn(intToBytes(index)) - val column: Option[ColumnOrSuperColumn] = sessions.withSession { - _ | (name, columnPath) - } - if (column.isDefined) column.get.column.value - else throw new NoSuchElementException("No element for vector [" + name + "] and index [" + index + "]") - } - - /** - * 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]] = { - val startBytes = if (start.isDefined) intToBytes(start.get) else null - val finishBytes = if (finish.isDefined) intToBytes(finish.get) else null - val columns: List[ColumnOrSuperColumn] = sessions.withSession { - _ / (name, - VECTOR_COLUMN_PARENT, - startBytes, finishBytes, - IS_ASCENDING, - count, - CONSISTENCY_LEVEL) - } - columns.map(column => column.getColumn.value) - } - - def getVectorStorageSizeFor(name: String): Int = { - sessions.withSession { - _ |# (name, VECTOR_COLUMN_PARENT) - } - } - - // =============================================================== - // For Map - // =============================================================== - - def insertMapStorageEntryFor(name: String, key: Array[Byte], element: Array[Byte]) = { - val columnPath = new ColumnPath(MAP_COLUMN_PARENT.getColumn_family) - columnPath.setColumn(key) - sessions.withSession { - _ ++| (name, - columnPath, - element, - System.currentTimeMillis, - CONSISTENCY_LEVEL) - } - } - - def insertMapStorageEntriesFor(name: String, entries: List[Tuple2[Array[Byte], Array[Byte]]]) = { - val batch = new scala.collection.mutable.HashMap[String, List[ColumnOrSuperColumn]] - for (entry <- entries) { - val columnOrSuperColumn = new ColumnOrSuperColumn - columnOrSuperColumn.setColumn(new Column(entry._1, entry._2, System.currentTimeMillis)) - batch += (MAP_COLUMN_PARENT.getColumn_family -> List(columnOrSuperColumn)) - } - sessions.withSession { - _ ++| (name, batch, CONSISTENCY_LEVEL) - } - } - - def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = { - try { - val columnPath = new ColumnPath(MAP_COLUMN_PARENT.getColumn_family) - columnPath.setColumn(key) - val column: Option[ColumnOrSuperColumn] = sessions.withSession { - _ | (name, columnPath) + def getAll(owner: String, keys: Iterable[Array[Byte]]): Map[Array[Byte], Array[Byte]] = { + sessions.withSession{ + session => { + var predicate = new SlicePredicate().setColumn_names(JavaConversions.asList(keys.toList)) + val cols = session / (owner, parent, predicate, CONSISTENCY_LEVEL) + var map = new TreeMap[Array[Byte], Array[Byte]]()(ordering) + cols.foreach{ + cosc => map += cosc.getColumn.getName -> cosc.getColumn.getValue + } + map + } } - if (column.isDefined) Some(column.get.getColumn.value) - else None - } catch { - case e => None } - } - def getMapStorageFor(name: String): List[Tuple2[Array[Byte], Array[Byte]]] = { - val size = getMapStorageSizeFor(name) - sessions.withSession { session => - val columns = session / - (name, MAP_COLUMN_PARENT, - EMPTY_BYTE_ARRAY, EMPTY_BYTE_ARRAY, - true, size, CONSISTENCY_LEVEL) - for { - columnOrSuperColumn <- columns - entry = (columnOrSuperColumn.column.name, columnOrSuperColumn.column.value) - } yield entry + def getValue(owner: String, key: Array[Byte], default: Array[Byte]) = { + sessions.withSession{ + session => { + try + { + session | (owner, path(key), CONSISTENCY_LEVEL) match { + case Some(cosc) => cosc.getColumn.getValue + case None => default + } + } catch { + case e: NotFoundException => default + } + } + } } - } - def getMapStorageSizeFor(name: String): Int = sessions.withSession { - _ |# (name, MAP_COLUMN_PARENT) - } - - def removeMapStorageFor(name: String): Unit = removeMapStorageFor(name, null) - - def removeMapStorageFor(name: String, key: Array[Byte]): Unit = { - val keyBytes = if (key eq null) null else key - val columnPath = new ColumnPath(MAP_COLUMN_PARENT.getColumn_family) - columnPath.setColumn(keyBytes) - sessions.withSession { - _ -- (name, - columnPath, - System.currentTimeMillis, - CONSISTENCY_LEVEL) + def put(owner: String, key: Array[Byte], value: Array[Byte]) = { + sessions.withSession{ + session => { + session ++| (owner, path(key), value, System.currentTimeMillis, CONSISTENCY_LEVEL) + } + } } + + def drop() = { + sessions.withSession{ + session => { + val slices = session.client.get_range_slices(session.keyspace, parent, + new SlicePredicate().setSlice_range(new SliceRange().setStart(Array.empty[Byte]).setFinish(Array.empty[Byte])), + new KeyRange().setStart_key("").setEnd_key(""), CONSISTENCY_LEVEL) + + val mutations = new JHMap[String, JMap[String, JList[Mutation]]] + JavaConversions.asIterable(slices).foreach{ + keySlice: KeySlice => { + val key = keySlice.getKey + val keyMutations = JavaConversions.asMap(mutations).getOrElse(key, { + val km = new JHMap[String, JList[Mutation]] + mutations.put(key, km) + km + }) + val amutation = new JAList[Mutation] + val cols = new JAList[Array[Byte]] + keyMutations.put(parent.getColumn_family, amutation) + JavaConversions.asIterable(keySlice.getColumns) foreach { + cosc: ColumnOrSuperColumn => { + cols.add(cosc.getColumn.getName) + } + } + amutation.add(new Mutation().setDeletion(new Deletion(System.currentTimeMillis).setPredicate(new SlicePredicate().setColumn_names(cols)))) + + } + } + session.client.batch_mutate(session.keyspace, mutations, CONSISTENCY_LEVEL) + } + } + } + } - def getMapStorageRangeFor(name: String, start: Option[Array[Byte]], finish: Option[Array[Byte]], count: Int): - List[Tuple2[Array[Byte], Array[Byte]]] = { - val startBytes = if (start.isDefined) start.get else null - val finishBytes = if (finish.isDefined) finish.get else null - val columns: List[ColumnOrSuperColumn] = sessions.withSession { - _ / (name, MAP_COLUMN_PARENT, startBytes, finishBytes, IS_ASCENDING, count, CONSISTENCY_LEVEL) - } - columns.map(column => (column.getColumn.name, column.getColumn.value)) - } + def queueAccess = new CassandraAccess(QUEUE_COLUMN_PARENT) + + def mapAccess = new CassandraAccess(MAP_COLUMN_PARENT) + + def vectorAccess = new CassandraAccess(VECTOR_COLUMN_PARENT) + + def refAccess = new CassandraAccess(REF_COLUMN_PARENT) } diff --git a/akka-persistence/akka-persistence-cassandra/src/test/scala/CassandraPersistentActorSpec.scala b/akka-persistence/akka-persistence-cassandra/src/test/scala/CassandraPersistentActorSpec.scala index c4ac603208..c877a06a1e 100644 --- a/akka-persistence/akka-persistence-cassandra/src/test/scala/CassandraPersistentActorSpec.scala +++ b/akka-persistence/akka-persistence-cassandra/src/test/scala/CassandraPersistentActorSpec.scala @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.persistence.cassandra +package akka.persistence.cassandra -import se.scalablesolutions.akka.actor.{Actor, ActorRef, Transactor} +import akka.actor.{Actor, ActorRef, Transactor} import Actor._ import org.junit.Test diff --git a/akka-persistence/akka-persistence-cassandra/src/test/scala/CassandraStorageBackendCompatibilityTest.scala b/akka-persistence/akka-persistence-cassandra/src/test/scala/CassandraStorageBackendCompatibilityTest.scala new file mode 100644 index 0000000000..500dfc8977 --- /dev/null +++ b/akka-persistence/akka-persistence-cassandra/src/test/scala/CassandraStorageBackendCompatibilityTest.scala @@ -0,0 +1,51 @@ +package akka.persistence.cassandra + + +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import akka.persistence.common.{QueueStorageBackendTest, VectorStorageBackendTest, MapStorageBackendTest, RefStorageBackendTest} + +@RunWith(classOf[JUnitRunner]) +class CassandraRefStorageBackendTestIntegration extends RefStorageBackendTest { + def dropRefs = { + CassandraStorageBackend.refAccess.drop + } + + + def storage = CassandraStorageBackend +} + +@RunWith(classOf[JUnitRunner]) +class CassandraMapStorageBackendTestIntegration extends MapStorageBackendTest { + def dropMaps = { + CassandraStorageBackend.mapAccess.drop + } + + + def storage = CassandraStorageBackend +} + +@RunWith(classOf[JUnitRunner]) +class CassandraVectorStorageBackendTestIntegration extends VectorStorageBackendTest { + def dropVectors = { + CassandraStorageBackend.vectorAccess.drop + } + + + def storage = CassandraStorageBackend +} + +@RunWith(classOf[JUnitRunner]) +class CassandraQueueStorageBackendTestIntegration extends QueueStorageBackendTest { + def dropQueues = { + CassandraStorageBackend.queueAccess.drop + } + + + def storage = CassandraStorageBackend +} + + + + + diff --git a/akka-persistence/akka-persistence-cassandra/src/test/scala/CassandraTicket343TestIntegration.scala b/akka-persistence/akka-persistence-cassandra/src/test/scala/CassandraTicket343TestIntegration.scala new file mode 100644 index 0000000000..d295e0a605 --- /dev/null +++ b/akka-persistence/akka-persistence-cassandra/src/test/scala/CassandraTicket343TestIntegration.scala @@ -0,0 +1,23 @@ +/** + * Copyright (C) 2009-2010 Scalable Solutions AB + */ + +package akka.persistence.cassandra + + +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import akka.persistence.common._ + +@RunWith(classOf[JUnitRunner]) +class CassandraTicket343TestIntegration extends Ticket343Test { + def dropMapsAndVectors: Unit = { + CassandraStorageBackend.vectorAccess.drop + CassandraStorageBackend.mapAccess.drop + } + + def getVector: (String) => PersistentVector[Array[Byte]] = CassandraStorage.getVector + + def getMap: (String) => PersistentMap[Array[Byte], Array[Byte]] = CassandraStorage.getMap + +} \ No newline at end of file diff --git a/akka-persistence/akka-persistence-common/src/main/scala/CommonStorageBackend.scala b/akka-persistence/akka-persistence-common/src/main/scala/CommonStorageBackend.scala new file mode 100644 index 0000000000..e7f16593a0 --- /dev/null +++ b/akka-persistence/akka-persistence-common/src/main/scala/CommonStorageBackend.scala @@ -0,0 +1,630 @@ +/** + * Copyright (C) 2009-2010 Scalable Solutions AB + */ + +package akka.persistence.common + +import akka.util.Logging +import java.lang.String +import java.nio.ByteBuffer +import collection.Map +import java.util.{Map => JMap} +import akka.persistence.common.PersistentMapBinary.COrdering._ +import collection.immutable._ + + +private[akka] trait CommonStorageBackendAccess { + import CommonStorageBackend._ + + /*abstract*/ + def getValue(owner: String, key: Array[Byte], default: Array[Byte]): Array[Byte] + + def put(owner: String, key: Array[Byte], value: Array[Byte]): Unit + + def delete(owner: String, key: Array[Byte]): Unit + + def getAll(owner: String, keys: Iterable[Array[Byte]]): Map[Array[Byte], Array[Byte]] + + def drop(): Unit + + /*concrete*/ + def decodeKey(owner: String, key: Array[Byte]) = key + + def delete(owner: String, index: Int): Unit = delete(owner, IntSerializer.toBytes(index)) + + def getValue(owner: String, index: Int): Array[Byte] = getValue(owner, IntSerializer.toBytes(index)) + + def getValue(owner: String, key: Array[Byte]): Array[Byte] = getValue(owner, key, null) + + def put(owner: String, index: Int, value: Array[Byte]): Unit = put(owner, IntSerializer.toBytes(index), value) +} + +private[akka] trait KVStorageBackendAccess extends CommonStorageBackendAccess { + import CommonStorageBackend._ + import KVStorageBackend._ + + def put(key: Array[Byte], value: Array[Byte]): Unit + + def getValue(key: Array[Byte]): Array[Byte] + + def getValue(key: Array[Byte], default: Array[Byte]): Array[Byte] + + def getAll(keys: Iterable[Array[Byte]]): Map[Array[Byte], Array[Byte]] + + def delete(key: Array[Byte]): Unit + + override def decodeKey(owner: String, key: Array[Byte]): Array[Byte] = { + val mapKeyLength = key.length - IntSerializer.bytesPerInt - owner.getBytes("UTF-8").length + val mapkey = new Array[Byte](mapKeyLength) + System.arraycopy(key, key.length - mapKeyLength, mapkey, 0, mapKeyLength) + mapkey + } + + + override def put(owner: String, key: Array[Byte], value: Array[Byte]): Unit = { + put(getKey(owner, key), value) + } + + override def put(owner: String, index: Int, value: Array[Byte]): Unit = { + put(getIndexedKey(owner, index), value) + } + + + override def getValue(owner: String, key: Array[Byte]): Array[Byte] = { + getValue(getKey(owner, key)) + } + + override def getValue(owner: String, index: Int): Array[Byte] = { + getValue(getIndexedKey(owner, index)) + } + + + override def getValue(owner: String, key: Array[Byte], default: Array[Byte]): Array[Byte] = { + getValue(getKey(owner, key), default) + } + + + override def getAll(owner: String, keys: Iterable[Array[Byte]]): Map[Array[Byte], Array[Byte]] = { + getAll(keys.map { + getKey(owner, _) + }) + } + + override def delete(owner: String, index: Int): Unit = { + delete(getIndexedKey(owner, index)) + } + + override def delete(owner: String, key: Array[Byte]): Unit = { + delete(getKey(owner, key)) + } +} + +private[akka] object CommonStorageBackendAccess { + implicit def stringToByteArray(st: String): Array[Byte] = { + st.getBytes("UTF-8") + } +} + +private[akka] object CommonStorageBackend { + val nullMapValueHeader = 0x00.byteValue + val nullMapValue: Array[Byte] = Array(nullMapValueHeader) + val notNullMapValueHeader: Byte = 0xff.byteValue + + def getStoredMapValue(value: Array[Byte]): Array[Byte] = { + value match { + case null => nullMapValue + case value => { + val stored = new Array[Byte](value.length + 1) + stored(0) = notNullMapValueHeader + System.arraycopy(value, 0, stored, 1, value.length) + stored + } + } + } + + def getMapValueFromStored(value: Array[Byte]): Array[Byte] = { + + if (value(0) == nullMapValueHeader) { + null + } else if (value(0) == notNullMapValueHeader) { + val returned = new Array[Byte](value.length - 1) + System.arraycopy(value, 1, returned, 0, value.length - 1) + returned + } else { + throw new StorageException("unknown header byte on map value:" + value(0)) + } + } + + object IntSerializer { + val bytesPerInt = java.lang.Integer.SIZE / java.lang.Byte.SIZE + + def toBytes(i: Int) = ByteBuffer.wrap(new Array[Byte](bytesPerInt)).putInt(i).array() + + def fromBytes(bytes: Array[Byte]) = ByteBuffer.wrap(bytes).getInt() + + def toString(obj: Int) = obj.toString + + def fromString(str: String) = str.toInt + } + + object SortedSetSerializer { + def toBytes(set: SortedSet[Array[Byte]]): Array[Byte] = { + val length = set.foldLeft(0) { + (total, bytes) => { + total + bytes.length + IntSerializer.bytesPerInt + } + } + val allBytes = new Array[Byte](length) + val written = set.foldLeft(0) { + (total, bytes) => { + val sizeBytes = IntSerializer.toBytes(bytes.length) + System.arraycopy(sizeBytes, 0, allBytes, total, sizeBytes.length) + System.arraycopy(bytes, 0, allBytes, total + sizeBytes.length, bytes.length) + total + sizeBytes.length + bytes.length + } + } + require(length == written, "Bytes Written Did not equal Calculated Length, written %d, length %d".format(written, length)) + allBytes + } + + def fromBytes(bytes: Array[Byte]): SortedSet[Array[Byte]] = { + var set = new TreeSet[Array[Byte]] + if (bytes.length > IntSerializer.bytesPerInt) { + var pos = 0 + while (pos < bytes.length) { + val lengthBytes = new Array[Byte](IntSerializer.bytesPerInt) + System.arraycopy(bytes, pos, lengthBytes, 0, IntSerializer.bytesPerInt) + pos += IntSerializer.bytesPerInt + val length = IntSerializer.fromBytes(lengthBytes) + val item = new Array[Byte](length) + System.arraycopy(bytes, pos, item, 0, length) + set = set + item + pos += length + } + } + set + } + + } + +} + +private[akka] object KVStorageBackend { + import CommonStorageBackend._ + /** + * Concat the ownerlenght+owner+key+ of owner so owned data will be colocated + * Store the length of owner as first byte to work around the rare case + * where ownerbytes1 + keybytes1 == ownerbytes2 + keybytes2 but ownerbytes1 != ownerbytes2 + */ + + def getKey(owner: String, key: Array[Byte]): Array[Byte] = { + val ownerBytes: Array[Byte] = owner.getBytes("UTF-8") + val ownerLenghtBytes: Array[Byte] = IntSerializer.toBytes(owner.length) + val theKey = new Array[Byte](ownerLenghtBytes.length + ownerBytes.length + key.length) + System.arraycopy(ownerLenghtBytes, 0, theKey, 0, ownerLenghtBytes.length) + System.arraycopy(ownerBytes, 0, theKey, ownerLenghtBytes.length, ownerBytes.length) + System.arraycopy(key, 0, theKey, ownerLenghtBytes.length + ownerBytes.length, key.length) + theKey + } + + def getIndexedKey(owner: String, index: Int): Array[Byte] = { + getKey(owner, IntSerializer.toBytes(index)) + } + +} + +private[akka] trait CommonStorageBackend extends MapStorageBackend[Array[Byte], Array[Byte]] with VectorStorageBackend[Array[Byte]] with RefStorageBackend[Array[Byte]] with QueueStorageBackend[Array[Byte]] with Logging { + import CommonStorageBackend._ + val mapKeysIndex = IntSerializer.toBytes(-1) + val vectorHeadIndex = IntSerializer.toBytes(-1) + val vectorTailIndex = IntSerializer.toBytes(-2) + val queueHeadIndex = IntSerializer.toBytes(-1) + val queueTailIndex = IntSerializer.toBytes(-2) + val zero = IntSerializer.toBytes(0) + val refItem = "refItem".getBytes("UTF-8") + + implicit val ordering = ArrayOrdering + + + def refAccess: CommonStorageBackendAccess + + def vectorAccess: CommonStorageBackendAccess + + def mapAccess: CommonStorageBackendAccess + + def queueAccess: CommonStorageBackendAccess + + + def getRefStorageFor(name: String): Option[Array[Byte]] = { + val result: Array[Byte] = refAccess.getValue(name, refItem) + Option(result) + } + + def insertRefStorageFor(name: String, element: Array[Byte]) = { + element match { + case null => refAccess.delete(name, refItem) + case _ => refAccess.put(name, refItem, element) + } + } + + + def getMapStorageRangeFor(name: String, start: Option[Array[Byte]], finish: Option[Array[Byte]], count: Int): List[(Array[Byte], Array[Byte])] = { + val allkeys: SortedSet[Array[Byte]] = getMapKeys(name) + val range = allkeys.rangeImpl(start, finish).take(count) + getKeyValues(name, range) + } + + def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = { + val keys = getMapKeys(name) + getKeyValues(name, keys) + } + + private def getKeyValues(name: String, keys: SortedSet[Array[Byte]]): List[(Array[Byte], Array[Byte])] = { + val all: Map[Array[Byte], Array[Byte]] = + mapAccess.getAll(name, keys) + + var returned = new TreeMap[Array[Byte], Array[Byte]]()(ordering) + all.foreach { + (entry) => { + entry match { + case (namePlusKey: Array[Byte], value: Array[Byte]) => { + //need to fix here + returned += mapAccess.decodeKey(name, namePlusKey) -> getMapValueFromStored(value) + } + } + } + } + returned.toList + } + + def getMapStorageSizeFor(name: String): Int = { + val keys = getMapKeys(name) + keys.size + } + + def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = { + val result: Array[Byte] = mapAccess.getValue(name, key) + result match { + case null => None + case _ => Some(getMapValueFromStored(result)) + } + } + + def removeMapStorageFor(name: String, key: Array[Byte]) = { + var keys = getMapKeys(name) + keys -= key + putMapKeys(name, keys) + mapAccess.delete(name, key) + } + + def removeMapStorageFor(name: String) = { + val keys = getMapKeys(name) + keys.foreach { + key => + mapAccess.delete(name, key) + log.debug("deleted key %s for %s", key, name) + } + mapAccess.delete(name, mapKeysIndex) + } + + def insertMapStorageEntryFor(name: String, key: Array[Byte], value: Array[Byte]) = { + mapAccess.put(name, key, getStoredMapValue(value)) + var keys = getMapKeys(name) + keys += key + putMapKeys(name, keys) + } + + def insertMapStorageEntriesFor(name: String, entries: List[(Array[Byte], Array[Byte])]) = { + val newKeys = entries.map { + case (key, value) => { + mapAccess.put(name, key, getStoredMapValue(value)) + key + } + } + var keys = getMapKeys(name) + keys ++= newKeys + putMapKeys(name, keys) + } + + def putMapKeys(name: String, keys: SortedSet[Array[Byte]]) = { + mapAccess.put(name, mapKeysIndex, SortedSetSerializer.toBytes(keys)) + } + + def getMapKeys(name: String): SortedSet[Array[Byte]] = { + SortedSetSerializer.fromBytes(mapAccess.getValue(name, mapKeysIndex, Array.empty[Byte])) + } + + def getVectorStorageSizeFor(name: String): Int = { + getVectorMetadata(name).size + } + + def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): List[Array[Byte]] = { + val mdata = getVectorMetadata(name) + + val st = start.getOrElse(0) + var cnt = + if (finish.isDefined) { + val f = finish.get + if (f >= st) (f - st) else count + } else { + count + } + if (cnt > (mdata.size - st)) { + cnt = mdata.size - st + } + + val ret = mdata.getRangeIndexes(st, count).toList map { + index: Int => { + log.debug("getting:" + index) + vectorAccess.getValue(name, index) + } + } + ret + } + + def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = { + val mdata = getVectorMetadata(name) + if (mdata.size > 0 && index < mdata.size) { + vectorAccess.getValue(name, mdata.getRangeIndexes(index, 1)(0)) + } else { + throw new StorageException("In Vector:" + name + " No such Index:" + index) + } + } + + def updateVectorStorageEntryFor(name: String, index: Int, elem: Array[Byte]) = { + val mdata = getVectorMetadata(name) + if (mdata.size > 0 && index < mdata.size) { + elem match { + case null => vectorAccess.delete(name, mdata.getRangeIndexes(index, 1)(0)) + case _ => vectorAccess.put(name, mdata.getRangeIndexes(index, 1)(0), elem) + } + } else { + throw new StorageException("In Vector:" + name + " No such Index:" + index) + } + } + + def insertVectorStorageEntriesFor(name: String, elements: List[Array[Byte]]) = { + elements.foreach { + insertVectorStorageEntryFor(name, _) + } + + } + + def insertVectorStorageEntryFor(name: String, element: Array[Byte]) = { + val mdata = getVectorMetadata(name) + if (mdata.canInsert) { + element match { + case null => vectorAccess.delete(name, mdata.head) + case _ => vectorAccess.put(name, mdata.head, element) + } + vectorAccess.put(name, vectorHeadIndex, IntSerializer.toBytes(mdata.nextInsert)) + } else { + throw new IllegalStateException("The vector %s is full".format(name)) + } + + } + + + override def removeVectorStorageEntryFor(name: String) = { + val mdata = getVectorMetadata(name) + if (mdata.canRemove) { + vectorAccess.put(name, vectorTailIndex, IntSerializer.toBytes(mdata.nextRemove)) + try + { + vectorAccess.delete(name, mdata.tail) + } catch { + case e: Exception => log.warn("Exception while trying to clean up a popped element from the vector, this is acceptable") + } + + } else { + //blow up or not? + } + } + + def getVectorMetadata(name: String): VectorMetadata = { + val head = vectorAccess.getValue(name, vectorHeadIndex, zero) + val tail = vectorAccess.getValue(name, vectorTailIndex, zero) + VectorMetadata(IntSerializer.fromBytes(head), IntSerializer.fromBytes(tail)) + } + + def getOrDefaultToZero(map: Map[Array[Byte], Array[Byte]], key: Array[Byte]): Int = { + map.get(key) match { + case Some(value) => IntSerializer.fromBytes(value) + case None => 0 + } + } + + + def remove(name: String): Boolean = { + val mdata = getQueueMetadata(name) + mdata.getActiveIndexes foreach { + index => + queueAccess.delete(name, index) + } + queueAccess.delete(name, queueHeadIndex) + queueAccess.delete(name, queueTailIndex) + true + } + + def peek(name: String, start: Int, count: Int): List[Array[Byte]] = { + val mdata = getQueueMetadata(name) + val ret = mdata.getPeekIndexes(start, count).toList map { + index: Int => { + log.debug("peeking:" + index) + queueAccess.getValue(name, index) + } + } + ret + } + + def size(name: String): Int = { + getQueueMetadata(name).size + } + + def dequeue(name: String): Option[Array[Byte]] = { + val mdata = getQueueMetadata(name) + if (mdata.canDequeue) { + try + { + val dequeued = queueAccess.getValue(name, mdata.head) + queueAccess.put(name, queueHeadIndex, IntSerializer.toBytes(mdata.nextDequeue)) + Some(dequeued) + } finally { + try + { + queueAccess.delete(name, mdata.head) + } catch { + //a failure to delete is ok, just leaves a K-V in Voldemort that will be overwritten if the queue ever wraps around + case e: Exception => log.warn(e, "caught an exception while deleting a dequeued element, however this will not cause any inconsistency in the queue") + } + } + } else { + None + } + } + + def enqueue(name: String, item: Array[Byte]): Option[Int] = { + val mdata = getQueueMetadata(name) + if (mdata.canEnqueue) { + item match { + case null => queueAccess.delete(name, mdata.tail) + case _ => queueAccess.put(name, mdata.tail, item) + } + queueAccess.put(name, queueTailIndex, IntSerializer.toBytes(mdata.nextEnqueue)) + Some(mdata.size + 1) + } else { + None + } + } + + def getQueueMetadata(name: String): QueueMetadata = { + val head = queueAccess.getValue(name, vectorHeadIndex, zero) + val tail = queueAccess.getValue(name, vectorTailIndex, zero) + QueueMetadata(IntSerializer.fromBytes(head), IntSerializer.fromBytes(tail)) + } + + + //wrapper for null + + + case class QueueMetadata(head: Int, tail: Int) { + //queue is an sequence with indexes from 0 to Int.MAX_VALUE + //wraps around when one pointer gets to max value + //head has an element in it. + //tail is the next slot to write to. + + def size = { + if (tail >= head) { + tail - head + } else { + //queue has wrapped + (Integer.MAX_VALUE - head) + (tail + 1) + } + } + + def canEnqueue = { + //the -1 stops the tail from catching the head on a wrap around + size < Integer.MAX_VALUE - 1 + } + + def canDequeue = { + size > 0 + } + + def getActiveIndexes(): IndexedSeq[Int] = { + if (tail >= head) { + Range(head, tail) + } else { + //queue has wrapped + val headRange = Range.inclusive(head, Integer.MAX_VALUE) + (if (tail > 0) { + headRange ++ Range(0, tail) + } else { + headRange + }) + } + } + + def getPeekIndexes(start: Int, count: Int): IndexedSeq[Int] = { + val indexes = getActiveIndexes + if (indexes.size < start) { + IndexedSeq.empty[Int] + } else { + indexes.drop(start).take(count) + } + } + + def nextEnqueue = { + tail match { + case Integer.MAX_VALUE => 0 + case _ => tail + 1 + } + } + + def nextDequeue = { + head match { + case Integer.MAX_VALUE => 0 + case _ => head + 1 + } + } + } + + case class VectorMetadata(head: Int, tail: Int) { + def size = { + if (head >= tail) { + head - tail + } else { + //queue has wrapped + (Integer.MAX_VALUE - tail) + (head + 1) + } + } + + def canInsert = { + //the -1 stops the tail from catching the head on a wrap around + size < Integer.MAX_VALUE - 1 + } + + def canRemove = { + size > 0 + } + + def getActiveIndexes(): IndexedSeq[Int] = { + if (head >= tail) { + Range(tail, head) + } else { + //queue has wrapped + val headRange = Range.inclusive(tail, Integer.MAX_VALUE) + (if (head > 0) { + headRange ++ Range(0, head) + } else { + headRange + }) + } + } + + def getRangeIndexes(start: Int, count: Int): IndexedSeq[Int] = { + val indexes = getActiveIndexes.reverse + if (indexes.size < start) { + IndexedSeq.empty[Int] + } else { + indexes.drop(start).take(count) + } + } + + def nextInsert = { + head match { + case Integer.MAX_VALUE => 0 + case _ => head + 1 + } + } + + def nextRemove = { + tail match { + case Integer.MAX_VALUE => 0 + case _ => tail + 1 + } + } + } + + +} \ No newline at end of file diff --git a/akka-persistence/akka-persistence-common/src/main/scala/KVStorageBackend.scala b/akka-persistence/akka-persistence-common/src/main/scala/KVStorageBackend.scala deleted file mode 100644 index 25c1b0b1f1..0000000000 --- a/akka-persistence/akka-persistence-common/src/main/scala/KVStorageBackend.scala +++ /dev/null @@ -1,476 +0,0 @@ -/** - * Copyright (C) 2009-2010 Scalable Solutions AB - */ - -package se.scalablesolutions.akka.persistence.common - -import se.scalablesolutions.akka.stm._ -import se.scalablesolutions.akka.persistence.common._ -import se.scalablesolutions.akka.util.Logging -import se.scalablesolutions.akka.util.Helpers._ -import se.scalablesolutions.akka.config.Config.config - -import java.lang.String -import collection.JavaConversions -import java.nio.ByteBuffer -import collection.Map -import collection.mutable.ArrayBuffer -import java.util.{ Properties, Map => JMap } -import se.scalablesolutions.akka.persistence.common.PersistentMapBinary.COrdering._ -import collection.immutable._ - - -private [akka] trait KVAccess { - def put(key: Array[Byte], value: Array[Byte]) - def getValue(key: Array[Byte]): Array[Byte] - def getValue(key: Array[Byte], default: Array[Byte]): Array[Byte] - def getAll(keys: Iterable[Array[Byte]]): Map[Array[Byte], Array[Byte]] - def delete(key: Array[Byte]) - def drop() - } - -private [akka] object KVAccess { - implicit def stringToByteArray(st: String): Array[Byte] = { - st.getBytes("UTF-8") - } -} - -private[akka] trait KVStorageBackend extends MapStorageBackend[Array[Byte], Array[Byte]] with VectorStorageBackend[Array[Byte]] with RefStorageBackend[Array[Byte]] with QueueStorageBackend[Array[Byte]] with Logging { - - val nullMapValueHeader = 0x00.byteValue - val nullMapValue: Array[Byte] = Array(nullMapValueHeader) - val notNullMapValueHeader: Byte = 0xff.byteValue - val underscoreBytesUTF8 = "_".getBytes("UTF-8") - val mapKeysIndex = getIndexedBytes(-1) - val vectorSizeIndex = getIndexedBytes(-1) - val queueHeadIndex = getIndexedBytes(-1) - val queueTailIndex = getIndexedBytes(-2) - - implicit val ordering = ArrayOrdering - import KVAccess._ - - - def refAccess: KVAccess - def vectorAccess: KVAccess - def mapAccess: KVAccess - def queueAccess: KVAccess - - def getRefStorageFor(name: String): Option[Array[Byte]] = { - val result: Array[Byte] = refAccess.getValue(name) - Option(result) - } - - def insertRefStorageFor(name: String, element: Array[Byte]) = { - element match { - case null => refAccess.delete(name) - case _ => refAccess.put(name, element) - } - } - - def getMapStorageRangeFor(name: String, start: Option[Array[Byte]], finish: Option[Array[Byte]], count: Int): List[(Array[Byte], Array[Byte])] = { - val allkeys: SortedSet[Array[Byte]] = getMapKeys(name) - val range = allkeys.rangeImpl(start, finish).take(count) - getKeyValues(name, range) - } - - def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = { - val keys = getMapKeys(name) - getKeyValues(name, keys) - } - - private def getKeyValues(name: String, keys: SortedSet[Array[Byte]]): List[(Array[Byte], Array[Byte])] = { - val all: Map[Array[Byte], Array[Byte]] = - mapAccess.getAll(keys.map { mapKey => - getKey(name, mapKey) - }) - - var returned = new TreeMap[Array[Byte], Array[Byte]]()(ordering) - all.foreach { (entry) => - { - entry match { - case (namePlusKey: Array[Byte], value: Array[Byte]) => { - returned += getMapKeyFromKey(name, namePlusKey) -> getMapValueFromStored(value) - } - } - } - } - returned.toList - } - - def getMapStorageSizeFor(name: String): Int = { - val keys = getMapKeys(name) - keys.size - } - - def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = { - val result: Array[Byte] = mapAccess.getValue(getKey(name, key)) - result match { - case null => None - case _ => Some(getMapValueFromStored(result)) - } - } - - def removeMapStorageFor(name: String, key: Array[Byte]) = { - var keys = getMapKeys(name) - keys -= key - putMapKeys(name, keys) - mapAccess.delete(getKey(name, key)) - } - - def removeMapStorageFor(name: String) = { - val keys = getMapKeys(name) - keys.foreach { key => - mapAccess.delete(getKey(name, key)) - log.debug("deleted key %s for %s", key, name) - } - mapAccess.delete(getKey(name, mapKeysIndex)) - } - - def insertMapStorageEntryFor(name: String, key: Array[Byte], value: Array[Byte]) = { - mapAccess.put(getKey(name, key), getStoredMapValue(value)) - var keys = getMapKeys(name) - keys += key - putMapKeys(name, keys) - } - - def insertMapStorageEntriesFor(name: String, entries: List[(Array[Byte], Array[Byte])]) = { - val newKeys = entries.map { - case (key, value) => { - mapAccess.put(getKey(name, key), getStoredMapValue(value)) - key - } - } - var keys = getMapKeys(name) - keys ++= newKeys - putMapKeys(name, keys) - } - - def putMapKeys(name: String, keys: SortedSet[Array[Byte]]) = { - mapAccess.put(getKey(name, mapKeysIndex), SortedSetSerializer.toBytes(keys)) - } - - def getMapKeys(name: String): SortedSet[Array[Byte]] = { - SortedSetSerializer.fromBytes(mapAccess.getValue(getKey(name, mapKeysIndex), Array.empty[Byte])) - } - - def getVectorStorageSizeFor(name: String): Int = { - IntSerializer.fromBytes(vectorAccess.getValue(getKey(name, vectorSizeIndex), IntSerializer.toBytes(0))) - } - - def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): List[Array[Byte]] = { - val size = getVectorStorageSizeFor(name) - val st = start.getOrElse(0) - var cnt = - if (finish.isDefined) { - val f = finish.get - if (f >= st) (f - st) else count - } else { - count - } - if (cnt > (size - st)) { - cnt = size - st - } - - val seq: IndexedSeq[Array[Byte]] = (st until st + cnt).map { index => - getIndexedKey(name, (size - 1) - index) - } //read backwards - - val all: Map[Array[Byte], Array[Byte]] = vectorAccess.getAll(seq) - - var storage = new ArrayBuffer[Array[Byte]](seq.size) - storage = storage.padTo(seq.size, Array.empty[Byte]) - var idx = 0; - seq.foreach { key => - { - if (all.isDefinedAt(key)) { - storage.update(idx, all.get(key).get) - } - idx += 1 - } - } - - storage.toList - } - - def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = { - val size = getVectorStorageSizeFor(name) - if (size > 0 && index < size) { - vectorAccess.getValue(getIndexedKey(name, /*read backwards*/ (size - 1) - index)) - } else { - throw new StorageException("In Vector:" + name + " No such Index:" + index) - } - } - - def updateVectorStorageEntryFor(name: String, index: Int, elem: Array[Byte]) = { - val size = getVectorStorageSizeFor(name) - if (size > 0 && index < size) { - elem match { - case null => vectorAccess.delete(getIndexedKey(name, /*read backwards*/ (size - 1) - index)) - case _ => vectorAccess.put(getIndexedKey(name, /*read backwards*/ (size - 1) - index), elem) - } - } else { - throw new StorageException("In Vector:" + name + " No such Index:" + index) - } - } - - def insertVectorStorageEntriesFor(name: String, elements: List[Array[Byte]]) = { - var size = getVectorStorageSizeFor(name) - elements.foreach { element => - if (element != null) { - vectorAccess.put(getIndexedKey(name, size), element) - } - size += 1 - } - vectorAccess.put(getKey(name, vectorSizeIndex), IntSerializer.toBytes(size)) - } - - def insertVectorStorageEntryFor(name: String, element: Array[Byte]) = { - insertVectorStorageEntriesFor(name, List(element)) - } - - def remove(name: String): Boolean = { - val mdata = getQueueMetadata(name) - mdata.getActiveIndexes foreach { index => - queueAccess.delete(getIndexedKey(name, index)) - } - queueAccess.delete(getKey(name, queueHeadIndex)) - queueAccess.delete(getKey(name, queueTailIndex)) - true - } - - def peek(name: String, start: Int, count: Int): List[Array[Byte]] = { - val mdata = getQueueMetadata(name) - val ret = mdata.getPeekIndexes(start, count).toList map { index: Int => - { - log.debug("peeking:" + index) - queueAccess.getValue(getIndexedKey(name, index)) - } - } - ret - } - - def size(name: String): Int = { - getQueueMetadata(name).size - } - - def dequeue(name: String): Option[Array[Byte]] = { - val mdata = getQueueMetadata(name) - if (mdata.canDequeue) { - val key = getIndexedKey(name, mdata.head) - try { - val dequeued = queueAccess.getValue(key) - queueAccess.put(getKey(name, queueHeadIndex), IntSerializer.toBytes(mdata.nextDequeue)) - Some(dequeued) - } finally { - try { - queueAccess.delete(key) - } catch { - //a failure to delete is ok, just leaves a K-V in Voldemort that will be overwritten if the queue ever wraps around - case e: Exception => log.warn(e, "caught an exception while deleting a dequeued element, however this will not cause any inconsistency in the queue") - } - } - } else { - None - } - } - - def enqueue(name: String, item: Array[Byte]): Option[Int] = { - val mdata = getQueueMetadata(name) - if (mdata.canEnqueue) { - val key = getIndexedKey(name, mdata.tail) - item match { - case null => queueAccess.delete(key) - case _ => queueAccess.put(key, item) - } - queueAccess.put(getKey(name, queueTailIndex), IntSerializer.toBytes(mdata.nextEnqueue)) - Some(mdata.size + 1) - } else { - None - } - } - - def getQueueMetadata(name: String): QueueMetadata = { - val keys = List(getKey(name, queueHeadIndex), getKey(name, queueTailIndex)) - val qdata = queueAccess.getAll(keys) - val values = keys.map { - qdata.get(_) match { - case Some(value) => IntSerializer.fromBytes(value) - case None => 0 - } - } - QueueMetadata(values.head, values.tail.head) - } - - /** - * Concat the ownerlenght+owner+key+ of owner so owned data will be colocated - * Store the length of owner as first byte to work around the rare case - * where ownerbytes1 + keybytes1 == ownerbytes2 + keybytes2 but ownerbytes1 != ownerbytes2 - */ - - def getKey(owner: String, key: Array[Byte]): Array[Byte] = { - val ownerBytes: Array[Byte] = owner.getBytes("UTF-8") - val ownerLenghtBytes: Array[Byte] = IntSerializer.toBytes(owner.length) - val theKey = new Array[Byte](ownerLenghtBytes.length + ownerBytes.length + key.length) - System.arraycopy(ownerLenghtBytes, 0, theKey, 0, ownerLenghtBytes.length) - System.arraycopy(ownerBytes, 0, theKey, ownerLenghtBytes.length, ownerBytes.length) - System.arraycopy(key, 0, theKey, ownerLenghtBytes.length + ownerBytes.length, key.length) - theKey - } - - def getIndexedBytes(index: Int): Array[Byte] = { - val indexbytes = IntSerializer.toBytes(index) - val theIndexKey = new Array[Byte](underscoreBytesUTF8.length + indexbytes.length) - System.arraycopy(underscoreBytesUTF8, 0, theIndexKey, 0, underscoreBytesUTF8.length) - System.arraycopy(indexbytes, 0, theIndexKey, underscoreBytesUTF8.length, indexbytes.length) - theIndexKey - } - - def getIndexedKey(owner: String, index: Int): Array[Byte] = { - getKey(owner, getIndexedBytes(index)) - } - - def getIndexFromVectorValueKey(owner: String, key: Array[Byte]): Int = { - val indexBytes = new Array[Byte](IntSerializer.bytesPerInt) - System.arraycopy(key, key.length - IntSerializer.bytesPerInt, indexBytes, 0, IntSerializer.bytesPerInt) - IntSerializer.fromBytes(indexBytes) - } - - def getMapKeyFromKey(owner: String, key: Array[Byte]): Array[Byte] = { - val mapKeyLength = key.length - IntSerializer.bytesPerInt - owner.getBytes("UTF-8").length - val mapkey = new Array[Byte](mapKeyLength) - System.arraycopy(key, key.length - mapKeyLength, mapkey, 0, mapKeyLength) - mapkey - } - - //wrapper for null - def getStoredMapValue(value: Array[Byte]): Array[Byte] = { - value match { - case null => nullMapValue - case value => { - val stored = new Array[Byte](value.length + 1) - stored(0) = notNullMapValueHeader - System.arraycopy(value, 0, stored, 1, value.length) - stored - } - } - } - - def getMapValueFromStored(value: Array[Byte]): Array[Byte] = { - - if (value(0) == nullMapValueHeader) { - null - } else if (value(0) == notNullMapValueHeader) { - val returned = new Array[Byte](value.length - 1) - System.arraycopy(value, 1, returned, 0, value.length - 1) - returned - } else { - throw new StorageException("unknown header byte on map value:" + value(0)) - } - } - - case class QueueMetadata(head: Int, tail: Int) { - //queue is an sequence with indexes from 0 to Int.MAX_VALUE - //wraps around when one pointer gets to max value - //head has an element in it. - //tail is the next slot to write to. - def size = { - if (tail >= head) { - tail - head - } else { - //queue has wrapped - (Integer.MAX_VALUE - head) + (tail + 1) - } - } - - def canEnqueue = { - //the -1 stops the tail from catching the head on a wrap around - size < Integer.MAX_VALUE - 1 - } - - def canDequeue = { size > 0 } - - def getActiveIndexes(): IndexedSeq[Int] = { - if (tail >= head) { - Range(head, tail) - } else { - //queue has wrapped - val headRange = Range.inclusive(head, Integer.MAX_VALUE) - (if (tail > 0) { headRange ++ Range(0, tail) } else { headRange }) - } - } - - def getPeekIndexes(start: Int, count: Int): IndexedSeq[Int] = { - val indexes = getActiveIndexes - if (indexes.size < start) { IndexedSeq.empty[Int] } else { indexes.drop(start).take(count) } - } - - def nextEnqueue = { - tail match { - case Integer.MAX_VALUE => 0 - case _ => tail + 1 - } - } - - def nextDequeue = { - head match { - case Integer.MAX_VALUE => 0 - case _ => head + 1 - } - } - } - - object IntSerializer { - val bytesPerInt = java.lang.Integer.SIZE / java.lang.Byte.SIZE - - def toBytes(i: Int) = ByteBuffer.wrap(new Array[Byte](bytesPerInt)).putInt(i).array() - - def fromBytes(bytes: Array[Byte]) = ByteBuffer.wrap(bytes).getInt() - - def toString(obj: Int) = obj.toString - - def fromString(str: String) = str.toInt - } - - object SortedSetSerializer { - def toBytes(set: SortedSet[Array[Byte]]): Array[Byte] = { - val length = set.foldLeft(0) { (total, bytes) => - { - total + bytes.length + IntSerializer.bytesPerInt - } - } - val allBytes = new Array[Byte](length) - val written = set.foldLeft(0) { (total, bytes) => - { - val sizeBytes = IntSerializer.toBytes(bytes.length) - System.arraycopy(sizeBytes, 0, allBytes, total, sizeBytes.length) - System.arraycopy(bytes, 0, allBytes, total + sizeBytes.length, bytes.length) - total + sizeBytes.length + bytes.length - } - } - require(length == written, "Bytes Written Did not equal Calculated Length, written %d, length %d".format(written, length)) - allBytes - } - - def fromBytes(bytes: Array[Byte]): SortedSet[Array[Byte]] = { - import se.scalablesolutions.akka.persistence.common.PersistentMapBinary.COrdering._ - - var set = new TreeSet[Array[Byte]] - if (bytes.length > IntSerializer.bytesPerInt) { - var pos = 0 - while (pos < bytes.length) { - val lengthBytes = new Array[Byte](IntSerializer.bytesPerInt) - System.arraycopy(bytes, pos, lengthBytes, 0, IntSerializer.bytesPerInt) - pos += IntSerializer.bytesPerInt - val length = IntSerializer.fromBytes(lengthBytes) - val item = new Array[Byte](length) - System.arraycopy(bytes, pos, item, 0, length) - set = set + item - pos += length - } - } - set - } - - } - -} \ No newline at end of file diff --git a/akka-persistence/akka-persistence-common/src/main/scala/Pool.scala b/akka-persistence/akka-persistence-common/src/main/scala/Pool.scala index 063cf3fe63..3e205dcfe9 100644 --- a/akka-persistence/akka-persistence-common/src/main/scala/Pool.scala +++ b/akka-persistence/akka-persistence-common/src/main/scala/Pool.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.common +package akka.persistence.common import org.apache.commons.pool._ import org.apache.commons.pool.impl._ 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 fa4ae1c358..b9e7c3195d 100644 --- a/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala +++ b/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.common +package 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.japi.{Option => JOption} +import akka.stm._ +import akka.stm.TransactionManagement.transaction +import akka.util.Logging +import akka.japi.{Option => JOption} import collection.mutable.ArraySeq // FIXME move to 'stm' package + add message with more info @@ -455,7 +455,7 @@ trait PersistentVector[T] extends IndexedSeq[T] with Transactional with Committa (entry: @unchecked) 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) => //.. + case LogEntry(_, _, POP) => storage.removeVectorStorageEntryFor(uuid) } } appendOnlyTxLog.clear @@ -516,9 +516,10 @@ trait PersistentVector[T] extends IndexedSeq[T] with Transactional with Committa * Removes the tail element of this vector. */ def pop: T = { - register - appendOnlyTxLog + LogEntry(None, None, POP) - throw new UnsupportedOperationException("PersistentVector::pop is not implemented") + register + val curr = replay + appendOnlyTxLog + LogEntry(None, None, POP) + curr.last } def update(index: Int, newElem: T) = { 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 7e6a95f9a1..44833f3535 100644 --- a/akka-persistence/akka-persistence-common/src/main/scala/StorageBackend.scala +++ b/akka-persistence/akka-persistence-common/src/main/scala/StorageBackend.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.common +package akka.persistence.common // abstracts persistence storage trait StorageBackend @@ -27,6 +27,10 @@ trait VectorStorageBackend[T] extends StorageBackend { def getVectorStorageEntryFor(name: String, index: Int): T def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): List[T] def getVectorStorageSizeFor(name: String): Int + def removeVectorStorageEntryFor(name:String):Unit = { + //should remove the "tail" if supported + throw new UnsupportedOperationException("VectorStorageBackend.removeVectorStorageEntry is not supported") + } } // for Ref diff --git a/akka-persistence/akka-persistence-common/src/test/scala/MapStorageBackendTest.scala b/akka-persistence/akka-persistence-common/src/test/scala/MapStorageBackendTest.scala index 395d0ef269..602ee5f34b 100644 --- a/akka-persistence/akka-persistence-common/src/test/scala/MapStorageBackendTest.scala +++ b/akka-persistence/akka-persistence-common/src/test/scala/MapStorageBackendTest.scala @@ -2,14 +2,14 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.common +package akka.persistence.common import org.scalatest.matchers.ShouldMatchers -import se.scalablesolutions.akka.util.Logging +import akka.util.Logging import org.scalatest.{BeforeAndAfterEach, Spec} import scala.util.Random import collection.immutable.{TreeMap, HashMap, HashSet} -import se.scalablesolutions.akka.persistence.common.PersistentMapBinary.COrdering._ +import akka.persistence.common.PersistentMapBinary.COrdering._ /** diff --git a/akka-persistence/akka-persistence-common/src/test/scala/QueueStorageBackendTest.scala b/akka-persistence/akka-persistence-common/src/test/scala/QueueStorageBackendTest.scala index 3eb89e3db5..3f655ef434 100644 --- a/akka-persistence/akka-persistence-common/src/test/scala/QueueStorageBackendTest.scala +++ b/akka-persistence/akka-persistence-common/src/test/scala/QueueStorageBackendTest.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.common +package akka.persistence.common import org.scalatest.matchers.ShouldMatchers -import se.scalablesolutions.akka.util.Logging +import akka.util.Logging import org.scalatest.{BeforeAndAfterEach, Spec} import scala.util.Random diff --git a/akka-persistence/akka-persistence-common/src/test/scala/RefStorageBackendTest.scala b/akka-persistence/akka-persistence-common/src/test/scala/RefStorageBackendTest.scala index 37902cf7c9..9e20c76317 100644 --- a/akka-persistence/akka-persistence-common/src/test/scala/RefStorageBackendTest.scala +++ b/akka-persistence/akka-persistence-common/src/test/scala/RefStorageBackendTest.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.common +package akka.persistence.common import org.scalatest.matchers.ShouldMatchers -import se.scalablesolutions.akka.util.Logging +import akka.util.Logging import org.scalatest.{BeforeAndAfterEach, Spec} /** diff --git a/akka-persistence/akka-persistence-common/src/test/scala/SortedSetStorageBackendTest.scala b/akka-persistence/akka-persistence-common/src/test/scala/SortedSetStorageBackendTest.scala index 2a9d3ab324..9337aa935b 100644 --- a/akka-persistence/akka-persistence-common/src/test/scala/SortedSetStorageBackendTest.scala +++ b/akka-persistence/akka-persistence-common/src/test/scala/SortedSetStorageBackendTest.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.common +package akka.persistence.common import org.scalatest.matchers.ShouldMatchers -import se.scalablesolutions.akka.util.Logging +import akka.util.Logging import org.scalatest.{BeforeAndAfterEach, Spec} /** diff --git a/akka-persistence/akka-persistence-common/src/test/scala/Ticket343Test.scala b/akka-persistence/akka-persistence-common/src/test/scala/Ticket343Test.scala index 1884621aaa..641f83d569 100644 --- a/akka-persistence/akka-persistence-common/src/test/scala/Ticket343Test.scala +++ b/akka-persistence/akka-persistence-common/src/test/scala/Ticket343Test.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.common +package akka.persistence.common import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -10,11 +10,11 @@ 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.Supervision.{OneForOneStrategy, Permanent} +import akka.actor.{Actor, ActorRef} +import akka.config.Supervision.{OneForOneStrategy, Permanent} import Actor._ -import se.scalablesolutions.akka.stm.global._ -import se.scalablesolutions.akka.util.Logging +import akka.stm.global._ +import akka.util.Logging import StorageObj._ @@ -34,6 +34,7 @@ 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 object VPOP case class VGET_AFTER_VADD(vsToAdd: List[String], isToFetch: List[Int]) case class VADD_WITH_SLICE(vsToAdd: List[String], start: Int, cnt: Int) @@ -178,6 +179,10 @@ object StorageObj { fooVector.slice(Some(s), None, c) } self.reply(l.map(new String(_))) + + case VPOP => + val p = atomic{fooVector.pop} + self.reply(p) } } } @@ -192,7 +197,6 @@ Spec with def getVector: String => PersistentVector[Array[Byte]] - def dropMapsAndVectors: Unit override def beforeEach { @@ -301,11 +305,14 @@ Spec with val proc = actorOf[SampleVectorStorage] 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) + // add 4 elements in separate transactions //also test add + pop of a 5th element + + (proc !! VADD("ticktock")).getOrElse("VADD failed") should equal(1) + (proc !! VADD("debasish")).getOrElse("VADD failed") should equal(2) + (proc !! VADD("maulindu")).getOrElse("VADD failed") should equal(3) + (proc !! VADD("ramanendu")).getOrElse("VADD failed") should equal(4) + (proc !! VADD("nilanjan")).getOrElse("VADD failed") should equal(5) + (proc !! VPOP).getOrElse("VPOP failed") should equal("ticktock".getBytes) new String((proc !! VGET(0)).get.asInstanceOf[Array[Byte]]) should equal("nilanjan") new String((proc !! VGET(1)).get.asInstanceOf[Array[Byte]]) should equal("ramanendu") diff --git a/akka-persistence/akka-persistence-common/src/test/scala/VectorStorageBackendTest.scala b/akka-persistence/akka-persistence-common/src/test/scala/VectorStorageBackendTest.scala index e677f8fe66..8d732a866f 100644 --- a/akka-persistence/akka-persistence-common/src/test/scala/VectorStorageBackendTest.scala +++ b/akka-persistence/akka-persistence-common/src/test/scala/VectorStorageBackendTest.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.common +package akka.persistence.common import org.scalatest.matchers.ShouldMatchers -import se.scalablesolutions.akka.util.Logging +import akka.util.Logging import org.scalatest.{BeforeAndAfterEach, Spec} import scala.util.Random @@ -29,16 +29,19 @@ trait VectorStorageBackendTest extends Spec with ShouldMatchers with BeforeAndAf } - describe("A Properly functioning VectorStorageBackend") { it("should insertVectorStorageEntry as a logical prepend operation to the existing list") { val vector = "insertSingleTest" val rand = new Random(3).nextInt(100) - val values = (0 to rand).toList.map {i: Int => vector + "value" + i} + val values = (0 to rand).toList.map{ + i: Int => vector + "value" + i + } storage.getVectorStorageSizeFor(vector) should be(0) - values.foreach {s: String => storage.insertVectorStorageEntryFor(vector, s.getBytes)} + values.foreach{ + s: String => storage.insertVectorStorageEntryFor(vector, s.getBytes) + } val shouldRetrieve = values.reverse - (0 to rand).foreach { + (0 to rand).foreach{ i: Int => { shouldRetrieve(i) should be(new String(storage.getVectorStorageEntryFor(vector, i))) } @@ -48,11 +51,15 @@ trait VectorStorageBackendTest extends Spec with ShouldMatchers with BeforeAndAf it("should insertVectorStorageEntries as a logical prepend operation to the existing list") { val vector = "insertMultiTest" val rand = new Random(3).nextInt(100) - val values = (0 to rand).toList.map {i: Int => vector + "value" + i} + val values = (0 to rand).toList.map{ + i: Int => vector + "value" + i + } storage.getVectorStorageSizeFor(vector) should be(0) - storage.insertVectorStorageEntriesFor(vector, values.map {s: String => s.getBytes}) + storage.insertVectorStorageEntriesFor(vector, values.map{ + s: String => s.getBytes + }) val shouldRetrieve = values.reverse - (0 to rand).foreach { + (0 to rand).foreach{ i: Int => { shouldRetrieve(i) should be(new String(storage.getVectorStorageEntryFor(vector, i))) } @@ -62,9 +69,13 @@ trait VectorStorageBackendTest extends Spec with ShouldMatchers with BeforeAndAf it("should successfully update entries") { val vector = "updateTest" val rand = new Random(3).nextInt(100) - val values = (0 to rand).toList.map {i: Int => vector + "value" + i} + val values = (0 to rand).toList.map{ + i: Int => vector + "value" + i + } val urand = new Random(3).nextInt(rand) - storage.insertVectorStorageEntriesFor(vector, values.map {s: String => s.getBytes}) + storage.insertVectorStorageEntriesFor(vector, values.map{ + s: String => s.getBytes + }) val toUpdate = "updated" + values.reverse(urand) storage.updateVectorStorageEntryFor(vector, urand, toUpdate.getBytes) toUpdate should be(new String(storage.getVectorStorageEntryFor(vector, urand))) @@ -73,9 +84,13 @@ trait VectorStorageBackendTest extends Spec with ShouldMatchers with BeforeAndAf it("should return the correct value from getVectorStorageFor") { val vector = "getTest" val rand = new Random(3).nextInt(100) - val values = (0 to rand).toList.map {i: Int => vector + "value" + i} + val values = (0 to rand).toList.map{ + i: Int => vector + "value" + i + } val urand = new Random(3).nextInt(rand) - storage.insertVectorStorageEntriesFor(vector, values.map {s: String => s.getBytes}) + storage.insertVectorStorageEntriesFor(vector, values.map{ + s: String => s.getBytes + }) values.reverse(urand) should be(new String(storage.getVectorStorageEntryFor(vector, urand))) } @@ -83,18 +98,59 @@ trait VectorStorageBackendTest extends Spec with ShouldMatchers with BeforeAndAf val vector = "getTest" val rand = new Random(3).nextInt(100) val drand = new Random(3).nextInt(rand) - val values = (0 to rand).toList.map {i: Int => vector + "value" + i} - storage.insertVectorStorageEntriesFor(vector, values.map {s: String => s.getBytes}) - values.reverse should be(storage.getVectorStorageRangeFor(vector, None, None, rand + 1).map {b: Array[Byte] => new String(b)}) - (0 to drand).foreach { + val values = (0 to rand).toList.map{ + i: Int => vector + "value" + i + } + storage.insertVectorStorageEntriesFor(vector, values.map{ + s: String => s.getBytes + }) + values.reverse should be(storage.getVectorStorageRangeFor(vector, None, None, rand + 1).map{ + b: Array[Byte] => new String(b) + }) + (0 to drand).foreach{ i: Int => { val value: String = vector + "value" + (rand - i) log.debug(value) - List(value) should be(storage.getVectorStorageRangeFor(vector, Some(i), None, 1).map {b: Array[Byte] => new String(b)}) + List(value) should be(storage.getVectorStorageRangeFor(vector, Some(i), None, 1).map{ + b: Array[Byte] => new String(b) + }) } } } + + it("should support remove properly") { + val vector = "removeTest" + val rand = new Random(3).nextInt(100) + val values = (0 to rand).toList.map{ + i: Int => vector + "value" + i + } + storage.insertVectorStorageEntriesFor(vector, values.map{ + s: String => s.getBytes + }) + storage.getVectorStorageSizeFor(vector) should be(values.size) + (1 to rand).foreach{ + i: Int => { + storage.removeVectorStorageEntryFor(vector) + values.reverse.dropRight(i) should be(storage.getVectorStorageRangeFor(vector, None, None, rand + 1 - i).map{ + b: Array[Byte] => new String(b) + }) + } + + } + storage.removeVectorStorageEntryFor(vector) + storage.getVectorStorageSizeFor(vector) should be(0) + storage.insertVectorStorageEntriesFor(vector, values.map{ + s: String => s.getBytes + }) + storage.getVectorStorageSizeFor(vector) should be(values.size) + values.foreach{ + s => storage.removeVectorStorageEntryFor(vector) + } + storage.getVectorStorageSizeFor(vector) should be(0) + + } + it("should behave properly when the range used in getVectorStorageRangeFor has indexes outside the current size of the vector") { //what is proper? } @@ -109,13 +165,17 @@ trait VectorStorageBackendTest extends Spec with ShouldMatchers with BeforeAndAf it("shoud throw a Storage exception when there is an attempt to retrieve an index larger than the Vector") { val vector = "tooLargeRetrieve" storage.insertVectorStorageEntryFor(vector, null) - evaluating {storage.getVectorStorageEntryFor(vector, 9)} should produce[StorageException] + evaluating{ + storage.getVectorStorageEntryFor(vector, 9) + } should produce[StorageException] } it("shoud throw a Storage exception when there is an attempt to update an index larger than the Vector") { val vector = "tooLargeUpdate" storage.insertVectorStorageEntryFor(vector, null) - evaluating {storage.updateVectorStorageEntryFor(vector, 9, null)} should produce[StorageException] + evaluating{ + storage.updateVectorStorageEntryFor(vector, 9, null) + } should produce[StorageException] } } diff --git a/akka-persistence/akka-persistence-couchdb/src/main/scala/CouchDBStorage.scala b/akka-persistence/akka-persistence-couchdb/src/main/scala/CouchDBStorage.scala index 80441b562b..14a5917aca 100644 --- a/akka-persistence/akka-persistence-couchdb/src/main/scala/CouchDBStorage.scala +++ b/akka-persistence/akka-persistence-couchdb/src/main/scala/CouchDBStorage.scala @@ -1,8 +1,8 @@ -package se.scalablesolutions.akka.persistence.couchdb +package akka.persistence.couchdb -import se.scalablesolutions.akka.actor.{newUuid} -import se.scalablesolutions.akka.stm._ -import se.scalablesolutions.akka.persistence.common._ +import akka.actor.{newUuid} +import akka.stm._ +import akka.persistence.common._ object CouchDBStorage extends Storage { type ElementType = Array[Byte] diff --git a/akka-persistence/akka-persistence-couchdb/src/main/scala/CouchDBStorageBackend.scala b/akka-persistence/akka-persistence-couchdb/src/main/scala/CouchDBStorageBackend.scala index dcda8e8d3d..1515cb7210 100644 --- a/akka-persistence/akka-persistence-couchdb/src/main/scala/CouchDBStorageBackend.scala +++ b/akka-persistence/akka-persistence-couchdb/src/main/scala/CouchDBStorageBackend.scala @@ -1,9 +1,9 @@ -package se.scalablesolutions.akka.persistence.couchdb +package akka.persistence.couchdb -import se.scalablesolutions.akka.stm._ -import se.scalablesolutions.akka.persistence.common._ -import se.scalablesolutions.akka.util.Logging -import se.scalablesolutions.akka.config.Config.config +import akka.stm._ +import akka.persistence.common._ +import akka.util.Logging +import akka.config.Config.config import org.apache.commons.httpclient.methods.{GetMethod, PostMethod, PutMethod, DeleteMethod} diff --git a/akka-persistence/akka-persistence-couchdb/src/test/scala/CouchDBPersistentActorSpec.scala b/akka-persistence/akka-persistence-couchdb/src/test/scala/CouchDBPersistentActorSpec.scala index 341e5dcf84..1e8da99da5 100644 --- a/akka-persistence/akka-persistence-couchdb/src/test/scala/CouchDBPersistentActorSpec.scala +++ b/akka-persistence/akka-persistence-couchdb/src/test/scala/CouchDBPersistentActorSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.persistence.couchdb +package akka.persistence.couchdb import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -6,7 +6,7 @@ import org.scalatest.BeforeAndAfterEach import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.actor.{Transactor, Actor, ActorRef} +import akka.actor.{Transactor, Actor, ActorRef} import Actor._ diff --git a/akka-persistence/akka-persistence-couchdb/src/test/scala/CouchDBStorageBackendSpec.scala b/akka-persistence/akka-persistence-couchdb/src/test/scala/CouchDBStorageBackendSpec.scala index 554433b976..a28abd9601 100644 --- a/akka-persistence/akka-persistence-couchdb/src/test/scala/CouchDBStorageBackendSpec.scala +++ b/akka-persistence/akka-persistence-couchdb/src/test/scala/CouchDBStorageBackendSpec.scala @@ -1,11 +1,11 @@ -package se.scalablesolutions.akka.persistence.couchdb +package akka.persistence.couchdb import org.specs._ import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.serialization.Serializable -import se.scalablesolutions.akka.serialization.Serializer._ +import akka.serialization.Serializable +import akka.serialization.Serializer._ import CouchDBStorageBackend._ import sbinary._ diff --git a/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorage.scala b/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorage.scala index fc8114b7dd..131361922c 100644 --- a/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorage.scala +++ b/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorage.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.hbase +package akka.persistence.hbase -import se.scalablesolutions.akka.actor.{Uuid,newUuid} -import se.scalablesolutions.akka.stm._ -import se.scalablesolutions.akka.persistence.common._ +import akka.actor.{Uuid,newUuid} +import akka.stm._ +import akka.persistence.common._ object HbaseStorage extends Storage { type ElementType = Array[Byte] diff --git a/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorageBackend.scala b/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorageBackend.scala index 30873bf036..d5b5aba154 100644 --- a/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorageBackend.scala +++ b/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorageBackend.scala @@ -2,14 +2,14 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.hbase +package akka.persistence.hbase import scala.collection.mutable.ListBuffer -import se.scalablesolutions.akka.stm._ -import se.scalablesolutions.akka.persistence.common._ -import se.scalablesolutions.akka.util.Logging -import se.scalablesolutions.akka.util.Helpers._ -import se.scalablesolutions.akka.config.Config.config +import akka.stm._ +import akka.persistence.common._ +import akka.util.Logging +import akka.util.Helpers._ +import akka.config.Config.config import org.apache.hadoop.hbase.HBaseConfiguration import org.apache.hadoop.hbase.HColumnDescriptor import org.apache.hadoop.hbase.HTableDescriptor diff --git a/akka-persistence/akka-persistence-hbase/src/test/scala/HbasePersistentActorSpecTestIntegration.scala b/akka-persistence/akka-persistence-hbase/src/test/scala/HbasePersistentActorSpecTestIntegration.scala index fc496ed480..71d06d37e6 100644 --- a/akka-persistence/akka-persistence-hbase/src/test/scala/HbasePersistentActorSpecTestIntegration.scala +++ b/akka-persistence/akka-persistence-hbase/src/test/scala/HbasePersistentActorSpecTestIntegration.scala @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.persistence.hbase +package akka.persistence.hbase -import se.scalablesolutions.akka.actor.{ Actor, ActorRef, Transactor } +import akka.actor.{ Actor, ActorRef, Transactor } import Actor._ import org.junit.Test diff --git a/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseStorageSpecTestIntegration.scala b/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseStorageSpecTestIntegration.scala index 02f3c04172..f19fbe9271 100644 --- a/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseStorageSpecTestIntegration.scala +++ b/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseStorageSpecTestIntegration.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.persistence.hbase +package akka.persistence.hbase import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers diff --git a/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseTicket343SpecTestIntegration.scala b/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseTicket343SpecTestIntegration.scala index be5429e134..47fc87ee64 100644 --- a/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseTicket343SpecTestIntegration.scala +++ b/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseTicket343SpecTestIntegration.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.persistence.hbase +package akka.persistence.hbase import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -6,11 +6,11 @@ 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.Supervision.{OneForOneStrategy,Permanent} +import akka.actor.{Actor, ActorRef} +import akka.config.Supervision.{OneForOneStrategy,Permanent} import Actor._ -import se.scalablesolutions.akka.stm.global._ -import se.scalablesolutions.akka.util.Logging +import akka.stm.global._ +import akka.util.Logging import HbaseStorageBackend._ diff --git a/akka-persistence/akka-persistence-hbase/src/test/scala/SimpleHbaseSpecTestIntegration.scala b/akka-persistence/akka-persistence-hbase/src/test/scala/SimpleHbaseSpecTestIntegration.scala index 8df7bbc7c9..427163d634 100644 --- a/akka-persistence/akka-persistence-hbase/src/test/scala/SimpleHbaseSpecTestIntegration.scala +++ b/akka-persistence/akka-persistence-hbase/src/test/scala/SimpleHbaseSpecTestIntegration.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.persistence.hbase +package akka.persistence.hbase import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -43,7 +43,7 @@ class SimpleHbaseSpecTestIntegration extends Spec with BeforeAndAfterAll with Sh } it("should use the quorum read from the akka configuration and access the table") { - import se.scalablesolutions.akka.config.Config.config + import akka.config.Config.config import org.apache.hadoop.hbase.HBaseConfiguration import org.apache.hadoop.hbase.client.HBaseAdmin import org.apache.hadoop.hbase.client.HTable 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 6bd99ee3e1..07980a8e3c 100644 --- a/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorage.scala +++ b/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorage.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.mongo +package akka.persistence.mongo -import se.scalablesolutions.akka.stm._ -import se.scalablesolutions.akka.persistence.common._ -import se.scalablesolutions.akka.actor.{newUuid} +import akka.stm._ +import akka.persistence.common._ +import akka.actor.{newUuid} object MongoStorage extends Storage { type ElementType = Array[Byte] 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 6573100422..47cc44fdc8 100644 --- a/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorageBackend.scala +++ b/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorageBackend.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.mongo +package akka.persistence.mongo -import se.scalablesolutions.akka.stm._ -import se.scalablesolutions.akka.persistence.common._ -import se.scalablesolutions.akka.util.Logging -import se.scalablesolutions.akka.config.Config.config +import akka.stm._ +import akka.persistence.common._ +import akka.util.Logging +import akka.config.Config.config import com.novus.casbah.mongodb.Imports._ 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 01f735b254..8708d1b45e 100644 --- a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoPersistentActorSpec.scala +++ b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoPersistentActorSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.persistence.mongo +package akka.persistence.mongo import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -6,7 +6,7 @@ import org.scalatest.BeforeAndAfterEach import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.actor.{Transactor, Actor, ActorRef} +import akka.actor.{Transactor, Actor, ActorRef} import Actor._ 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 e9576cc152..a2b0a7aab5 100644 --- a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoStorageSpec.scala +++ b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoStorageSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.persistence.mongo +package akka.persistence.mongo import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers diff --git a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala index cd90620cef..ef70719f4f 100644 --- a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala +++ b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.persistence.mongo +package akka.persistence.mongo import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -6,11 +6,11 @@ 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.Supervision.{OneForOneStrategy,Permanent} +import akka.actor.{Actor, ActorRef} +import akka.config.Supervision.{OneForOneStrategy,Permanent} import Actor._ -import se.scalablesolutions.akka.stm.global._ -import se.scalablesolutions.akka.util.Logging +import akka.stm.global._ +import akka.util.Logging import MongoStorageBackend._ diff --git a/akka-persistence/akka-persistence-redis/src/main/scala/RedisPubSubServer.scala b/akka-persistence/akka-persistence-redis/src/main/scala/RedisPubSubServer.scala index afde9ebb1b..375bc60bae 100644 --- a/akka-persistence/akka-persistence-redis/src/main/scala/RedisPubSubServer.scala +++ b/akka-persistence/akka-persistence-redis/src/main/scala/RedisPubSubServer.scala @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.persistence.redis +package akka.persistence.redis -import se.scalablesolutions.akka.actor.Actor +import akka.actor.Actor import com.redis._ sealed trait Msg 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 9055b6717c..9dfc37770a 100644 --- a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorage.scala +++ b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorage.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.redis +package akka.persistence.redis -import se.scalablesolutions.akka.actor.{newUuid} -import se.scalablesolutions.akka.stm._ -import se.scalablesolutions.akka.persistence.common._ +import akka.actor.{newUuid} +import akka.stm._ +import akka.persistence.common._ object RedisStorage extends Storage { type ElementType = Array[Byte] 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 dbb63f972b..f255e469a2 100644 --- a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala +++ b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.redis +package akka.persistence.redis -import se.scalablesolutions.akka.stm._ -import se.scalablesolutions.akka.persistence.common._ -import se.scalablesolutions.akka.util.Logging -import se.scalablesolutions.akka.config.Config.config +import akka.stm._ +import akka.persistence.common._ +import akka.util.Logging +import akka.config.Config.config import com.redis._ diff --git a/akka-persistence/akka-persistence-redis/src/test/scala/RedisInconsistentSizeBugTest.scala b/akka-persistence/akka-persistence-redis/src/test/scala/RedisInconsistentSizeBugTest.scala index 78c21f2082..f365924f2f 100644 --- a/akka-persistence/akka-persistence-redis/src/test/scala/RedisInconsistentSizeBugTest.scala +++ b/akka-persistence/akka-persistence-redis/src/test/scala/RedisInconsistentSizeBugTest.scala @@ -1,15 +1,15 @@ -package se.scalablesolutions.akka.persistence.redis +package akka.persistence.redis import sbinary._ import sbinary.Operations._ import sbinary.DefaultProtocol._ -import se.scalablesolutions.akka.actor.{Actor, ActorRef} -import se.scalablesolutions.akka.config.Supervision.{OneForOneStrategy, Permanent} +import akka.actor.{Actor, ActorRef} +import akka.config.Supervision.{OneForOneStrategy, Permanent} import Actor._ -import se.scalablesolutions.akka.persistence.common.PersistentVector -import se.scalablesolutions.akka.stm.global._ -import se.scalablesolutions.akka.util.Logging +import akka.persistence.common.PersistentVector +import akka.stm.global._ +import akka.util.Logging import java.util.{Calendar, Date} diff --git a/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentActorSpec.scala b/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentActorSpec.scala index 2856d9f7f8..e0ab63abe8 100644 --- a/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentActorSpec.scala +++ b/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentActorSpec.scala @@ -1,12 +1,12 @@ -package se.scalablesolutions.akka.persistence.redis +package akka.persistence.redis import org.scalatest.junit.JUnitSuite import org.junit.{Test, Before} import org.junit.Assert._ -import se.scalablesolutions.akka.actor.{ActorRef, Transactor} -import se.scalablesolutions.akka.actor.Actor._ +import akka.actor.{ActorRef, Transactor} +import akka.actor.Actor._ /** * A persistent actor based on Redis storage. diff --git a/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentQSpec.scala b/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentQSpec.scala index 5522b00d45..f21b29064f 100644 --- a/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentQSpec.scala +++ b/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentQSpec.scala @@ -1,9 +1,9 @@ -package se.scalablesolutions.akka.persistence.redis +package akka.persistence.redis import org.junit.{Test, Before} import org.junit.Assert._ -import se.scalablesolutions.akka.actor.{Actor, ActorRef, Transactor} +import akka.actor.{Actor, ActorRef, Transactor} import Actor._ /** diff --git a/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentSortedSetSpec.scala b/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentSortedSetSpec.scala index eb17dd0d08..8e25dbf4d6 100644 --- a/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentSortedSetSpec.scala +++ b/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentSortedSetSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.persistence.redis +package akka.persistence.redis import org.scalatest.Spec import org.scalatest.Assertions @@ -7,7 +7,7 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.actor.{Actor, ActorRef, Transactor} +import akka.actor.{Actor, ActorRef, Transactor} import Actor._ /** diff --git a/akka-persistence/akka-persistence-redis/src/test/scala/RedisStorageBackendSpec.scala b/akka-persistence/akka-persistence-redis/src/test/scala/RedisStorageBackendSpec.scala index d7a7d579bf..37195cbbea 100644 --- a/akka-persistence/akka-persistence-redis/src/test/scala/RedisStorageBackendSpec.scala +++ b/akka-persistence/akka-persistence-redis/src/test/scala/RedisStorageBackendSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.persistence.redis +package akka.persistence.redis import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -6,8 +6,8 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.serialization.Serializable -import se.scalablesolutions.akka.serialization.Serializer._ +import akka.serialization.Serializable +import akka.serialization.Serializer._ import sbinary._ import sbinary.Operations._ diff --git a/akka-persistence/akka-persistence-redis/src/test/scala/RedisTicket343Spec.scala b/akka-persistence/akka-persistence-redis/src/test/scala/RedisTicket343Spec.scala index f1167ef84d..d5accca626 100644 --- a/akka-persistence/akka-persistence-redis/src/test/scala/RedisTicket343Spec.scala +++ b/akka-persistence/akka-persistence-redis/src/test/scala/RedisTicket343Spec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.persistence.redis +package akka.persistence.redis import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -6,12 +6,12 @@ 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.Supervision.{OneForOneStrategy,Permanent} +import akka.actor.{Actor} +import akka.config.Supervision.{OneForOneStrategy,Permanent} import Actor._ -import se.scalablesolutions.akka.persistence.common.PersistentVector -import se.scalablesolutions.akka.stm.global._ -import se.scalablesolutions.akka.util.Logging +import akka.persistence.common.PersistentVector +import akka.stm.global._ +import akka.util.Logging import RedisStorageBackend._ diff --git a/akka-persistence/akka-persistence-riak/src/main/scala/RiakStorage.scala b/akka-persistence/akka-persistence-riak/src/main/scala/RiakStorage.scala index 446311f715..a155144b30 100644 --- a/akka-persistence/akka-persistence-riak/src/main/scala/RiakStorage.scala +++ b/akka-persistence/akka-persistence-riak/src/main/scala/RiakStorage.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.riak +package akka.persistence.riak -import se.scalablesolutions.akka.actor.{newUuid} -import se.scalablesolutions.akka.stm._ -import se.scalablesolutions.akka.persistence.common._ +import akka.actor.{newUuid} +import akka.stm._ +import akka.persistence.common._ object RiakStorage extends Storage { diff --git a/akka-persistence/akka-persistence-riak/src/main/scala/RiakStorageBackend.scala b/akka-persistence/akka-persistence-riak/src/main/scala/RiakStorageBackend.scala index 0e77451a58..0967860851 100644 --- a/akka-persistence/akka-persistence-riak/src/main/scala/RiakStorageBackend.scala +++ b/akka-persistence/akka-persistence-riak/src/main/scala/RiakStorageBackend.scala @@ -2,22 +2,22 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.riak +package akka.persistence.riak -import se.scalablesolutions.akka.persistence.common._ -import se.scalablesolutions.akka.config.Config.config +import akka.persistence.common._ +import akka.config.Config.config import java.lang.String import collection.JavaConversions import collection.Map import java.util.{Map => JMap} -import se.scalablesolutions.akka.persistence.common.PersistentMapBinary.COrdering._ +import akka.persistence.common.PersistentMapBinary.COrdering._ import collection.immutable._ import com.google.protobuf.ByteString import com.trifork.riak.{RequestMeta, RiakObject, RiakClient} -private[akka] object RiakStorageBackend extends KVStorageBackend { +private[akka] object RiakStorageBackend extends CommonStorageBackend { val refBucket = config.getString("akka.storage.riak.bucket.ref", "Refs") val mapBucket = config.getString("akka.storage.riak.bucket.map", "Maps") val vectorBucket = config.getString("akka.storage.riak.bucket.vector", "Vectors") @@ -26,7 +26,8 @@ private[akka] object RiakStorageBackend extends KVStorageBackend { val clientPort = config.getInt("akka.storage.riak.client.port", 8087) val riakClient: RiakClient = new RiakClient(clientHost, clientPort); - import KVAccess._ + import CommonStorageBackendAccess._ + import KVStorageBackend._ import RiakAccess._ @@ -59,7 +60,7 @@ private[akka] object RiakStorageBackend extends KVStorageBackend { } - class RiakAccess(val bucket: String) extends KVAccess { + class RiakAccess(val bucket: String) extends KVStorageBackendAccess { //http://www.mail-archive.com/riak-users@lists.basho.com/msg01013.html val quorum: Int = 0xfffffffd val one: Int = 0xfffffffe diff --git a/akka-persistence/akka-persistence-riak/src/test/scala/RiakStorageBackendCompatibilityTest.scala b/akka-persistence/akka-persistence-riak/src/test/scala/RiakStorageBackendCompatibilityTest.scala index e7f149f98f..66cf126bb6 100644 --- a/akka-persistence/akka-persistence-riak/src/test/scala/RiakStorageBackendCompatibilityTest.scala +++ b/akka-persistence/akka-persistence-riak/src/test/scala/RiakStorageBackendCompatibilityTest.scala @@ -1,9 +1,9 @@ -package se.scalablesolutions.akka.persistence.riak +package akka.persistence.riak import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner -import se.scalablesolutions.akka.persistence.common.{QueueStorageBackendTest, VectorStorageBackendTest, MapStorageBackendTest, RefStorageBackendTest} +import akka.persistence.common.{QueueStorageBackendTest, VectorStorageBackendTest, MapStorageBackendTest, RefStorageBackendTest} @RunWith(classOf[JUnitRunner]) class RiakRefStorageBackendTestIntegration extends RefStorageBackendTest { diff --git a/akka-persistence/akka-persistence-riak/src/test/scala/RiakTicket343TestIntegration.scala b/akka-persistence/akka-persistence-riak/src/test/scala/RiakTicket343TestIntegration.scala index ae9e3e388f..3400f0bddb 100644 --- a/akka-persistence/akka-persistence-riak/src/test/scala/RiakTicket343TestIntegration.scala +++ b/akka-persistence/akka-persistence-riak/src/test/scala/RiakTicket343TestIntegration.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.riak +package akka.persistence.riak import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner -import se.scalablesolutions.akka.persistence.common._ +import akka.persistence.common._ @RunWith(classOf[JUnitRunner]) class RiakTicket343TestIntegration extends Ticket343Test { @@ -19,4 +19,5 @@ class RiakTicket343TestIntegration extends Ticket343Test { def getVector: (String) => PersistentVector[Array[Byte]] = RiakStorage.getVector def getMap: (String) => PersistentMap[Array[Byte], Array[Byte]] = RiakStorage.getMap + } \ No newline at end of file diff --git a/akka-persistence/akka-persistence-voldemort/src/main/scala/VoldemortStorage.scala b/akka-persistence/akka-persistence-voldemort/src/main/scala/VoldemortStorage.scala index 2a9c3c5717..f6d252df89 100644 --- a/akka-persistence/akka-persistence-voldemort/src/main/scala/VoldemortStorage.scala +++ b/akka-persistence/akka-persistence-voldemort/src/main/scala/VoldemortStorage.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.voldemort +package akka.persistence.voldemort -import se.scalablesolutions.akka.actor.{newUuid} -import se.scalablesolutions.akka.stm._ -import se.scalablesolutions.akka.persistence.common._ +import akka.actor.{newUuid} +import akka.stm._ +import akka.persistence.common._ object VoldemortStorage extends Storage { diff --git a/akka-persistence/akka-persistence-voldemort/src/main/scala/VoldemortStorageBackend.scala b/akka-persistence/akka-persistence-voldemort/src/main/scala/VoldemortStorageBackend.scala index 348f8a9f7a..87fd07cbe5 100644 --- a/akka-persistence/akka-persistence-voldemort/src/main/scala/VoldemortStorageBackend.scala +++ b/akka-persistence/akka-persistence-voldemort/src/main/scala/VoldemortStorageBackend.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.voldemort +package akka.persistence.voldemort -import se.scalablesolutions.akka.persistence.common._ -import se.scalablesolutions.akka.config.Config.config +import akka.persistence.common._ +import akka.config.Config.config import voldemort.client._ import java.lang.String @@ -21,9 +21,10 @@ import voldemort.client.protocol.admin.{AdminClientConfig, AdminClient} In this case all VoldemortBackend operations can be retried until successful, and data should remain consistent */ -private[akka] object VoldemortStorageBackend extends KVStorageBackend { +private[akka] object VoldemortStorageBackend extends CommonStorageBackend { - import KVAccess._ + import CommonStorageBackendAccess._ + import KVStorageBackend._ import VoldemortAccess._ val bootstrapUrlsProp = "bootstrap_urls" @@ -38,10 +39,10 @@ private[akka] object VoldemortStorageBackend extends KVStorageBackend { var storeClientFactory: StoreClientFactory = null - var refs: KVAccess = null - var maps: KVAccess = null - var vectors: KVAccess = null - var queues: KVAccess = null + var refs: KVStorageBackendAccess = null + var maps: KVStorageBackendAccess = null + var vectors: KVStorageBackendAccess = null + var queues: KVStorageBackendAccess = null resetAccess def refAccess = refs @@ -57,8 +58,8 @@ private[akka] object VoldemortStorageBackend extends KVStorageBackend { var admin: AdminClient = null } - class VoldemortAccess(val store: String) extends KVAccess { - + class VoldemortAccess(val store: String) extends KVStorageBackendAccess { + import KVStorageBackend._ import VoldemortAccess._ val client: StoreClient[Array[Byte], Array[Byte]] = VoldemortStorageBackend.storeClientFactory.getStoreClient(store) diff --git a/akka-persistence/akka-persistence-voldemort/src/test/scala/EmbeddedVoldemort.scala b/akka-persistence/akka-persistence-voldemort/src/test/scala/EmbeddedVoldemort.scala index e23c3002c6..867c9f8267 100644 --- a/akka-persistence/akka-persistence-voldemort/src/test/scala/EmbeddedVoldemort.scala +++ b/akka-persistence/akka-persistence-voldemort/src/test/scala/EmbeddedVoldemort.scala @@ -1,11 +1,11 @@ -package se.scalablesolutions.akka.persistence.voldemort +package akka.persistence.voldemort import voldemort.server.{VoldemortServer, VoldemortConfig} import org.scalatest.{Suite, BeforeAndAfterAll} import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import java.io.File -import se.scalablesolutions.akka.util.{Logging} +import akka.util.{Logging} import collection.JavaConversions import voldemort.store.memory.InMemoryStorageConfiguration import voldemort.client.protocol.admin.{AdminClientConfig, AdminClient} diff --git a/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortPersistentActorSuite.scala b/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortPersistentActorSuite.scala index fde1d0125c..7a14503335 100644 --- a/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortPersistentActorSuite.scala +++ b/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortPersistentActorSuite.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.persistence.voldemort +package akka.persistence.voldemort import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -6,7 +6,7 @@ import org.scalatest.BeforeAndAfterEach import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.actor.{Transactor, Actor, ActorRef} +import akka.actor.{Transactor, Actor, ActorRef} import Actor._ import BankAccountActor._ diff --git a/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortStorageBackendCompatibilityTest.scala b/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortStorageBackendCompatibilityTest.scala index baccc356f7..058c0f1385 100644 --- a/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortStorageBackendCompatibilityTest.scala +++ b/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortStorageBackendCompatibilityTest.scala @@ -1,9 +1,9 @@ -package se.scalablesolutions.akka.persistence.voldemort +package akka.persistence.voldemort import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner -import se.scalablesolutions.akka.persistence.common.{QueueStorageBackendTest, VectorStorageBackendTest, MapStorageBackendTest, RefStorageBackendTest} +import akka.persistence.common.{QueueStorageBackendTest, VectorStorageBackendTest, MapStorageBackendTest, RefStorageBackendTest} @RunWith(classOf[JUnitRunner]) class VoldemortRefStorageBackendTest extends RefStorageBackendTest with EmbeddedVoldemort { diff --git a/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortStorageBackendSuite.scala b/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortStorageBackendSuite.scala index 5b03e013ff..bd1116f56f 100644 --- a/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortStorageBackendSuite.scala +++ b/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortStorageBackendSuite.scala @@ -1,11 +1,13 @@ -package se.scalablesolutions.akka.persistence.voldemort +package akka.persistence.voldemort import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner -import se.scalablesolutions.akka.persistence.voldemort.VoldemortStorageBackend._ -import se.scalablesolutions.akka.util.{Logging} +import akka.persistence.voldemort.VoldemortStorageBackend._ +import akka.persistence.common.CommonStorageBackend._ +import akka.persistence.common.KVStorageBackend._ +import akka.util.Logging import collection.immutable.TreeSet import VoldemortStorageBackendSuite._ import scala.None @@ -100,10 +102,11 @@ class VoldemortStorageBackendSuite extends FunSuite with ShouldMatchers with Emb val key = "vectorApiKey" val value = bytes("Some bytes we want to store in a vector") val updatedValue = bytes("Some updated bytes we want to store in a vector") - vectorAccess.delete(getKey(key, vectorSizeIndex)) + vectorAccess.delete(getKey(key, vectorHeadIndex)) + vectorAccess.delete(getKey(key, vectorTailIndex)) vectorAccess.delete(getIndexedKey(key, 0)) vectorAccess.delete(getIndexedKey(key, 1)) - + insertVectorStorageEntryFor(key, value) //again insertVectorStorageEntryFor(key, value) @@ -128,7 +131,7 @@ class VoldemortStorageBackendSuite extends FunSuite with ShouldMatchers with Emb val key = "queueApiKey" val value = bytes("some bytes even") val valueOdd = bytes("some bytes odd") - + remove(key) VoldemortStorageBackend.size(key) should be(0) enqueue(key, value) should be(Some(1)) @@ -160,6 +163,11 @@ class VoldemortStorageBackendSuite extends FunSuite with ShouldMatchers with Emb } + def getIndexFromVectorValueKey(owner: String, key: Array[Byte]): Int = { + val indexBytes = new Array[Byte](IntSerializer.bytesPerInt) + System.arraycopy(key, key.length - IntSerializer.bytesPerInt, indexBytes, 0, IntSerializer.bytesPerInt) + IntSerializer.fromBytes(indexBytes) + } } diff --git a/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortTicket343Test.scala b/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortTicket343Test.scala index f63f354f4a..dce9d7452d 100644 --- a/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortTicket343Test.scala +++ b/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortTicket343Test.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.persistence.voldemort +package akka.persistence.voldemort import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner -import se.scalablesolutions.akka.persistence.common._ +import akka.persistence.common._ @RunWith(classOf[JUnitRunner]) class VoldemortTicket343Test extends Ticket343Test with EmbeddedVoldemort { @@ -19,4 +19,5 @@ class VoldemortTicket343Test extends Ticket343Test with EmbeddedVoldemort { def getVector: (String) => PersistentVector[Array[Byte]] = VoldemortStorage.getVector def getMap: (String) => PersistentMap[Array[Byte], Array[Byte]] = VoldemortStorage.getMap + } \ No newline at end of file diff --git a/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java b/akka-remote/src/main/java/akka/remote/protocol/RemoteProtocol.java similarity index 69% rename from akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java rename to akka-remote/src/main/java/akka/remote/protocol/RemoteProtocol.java index 60f5004861..1be793a2aa 100644 --- a/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java +++ b/akka-remote/src/main/java/akka/remote/protocol/RemoteProtocol.java @@ -1,7 +1,7 @@ // Generated by the protocol buffer compiler. DO NOT EDIT! // source: RemoteProtocol.proto -package se.scalablesolutions.akka.remote.protocol; +package akka.remote.protocol; public final class RemoteProtocol { private RemoteProtocol() {} @@ -49,7 +49,7 @@ public final class RemoteProtocol { } public static final com.google.protobuf.Descriptors.EnumDescriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(0); + return akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(0); } private static final ActorType[] VALUES = { @@ -71,7 +71,7 @@ public final class RemoteProtocol { } static { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor(); + akka.remote.protocol.RemoteProtocol.getDescriptor(); } // @@protoc_insertion_point(enum_scope:ActorType) @@ -122,7 +122,7 @@ public final class RemoteProtocol { } public static final com.google.protobuf.Descriptors.EnumDescriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(1); + return akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(1); } private static final SerializationSchemeType[] VALUES = { @@ -144,7 +144,7 @@ public final class RemoteProtocol { } static { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor(); + akka.remote.protocol.RemoteProtocol.getDescriptor(); } // @@protoc_insertion_point(enum_scope:SerializationSchemeType) @@ -189,7 +189,7 @@ public final class RemoteProtocol { } public static final com.google.protobuf.Descriptors.EnumDescriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(2); + return akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(2); } private static final LifeCycleType[] VALUES = { @@ -211,7 +211,7 @@ public final class RemoteProtocol { } static { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor(); + akka.remote.protocol.RemoteProtocol.getDescriptor(); } // @@protoc_insertion_point(enum_scope:LifeCycleType) @@ -236,12 +236,12 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteActorRefProtocol_descriptor; + return 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; + return akka.remote.protocol.RemoteProtocol.internal_static_RemoteActorRefProtocol_fieldAccessorTable; } // required string classOrServiceName = 1; @@ -261,9 +261,9 @@ public final class RemoteProtocol { // required .AddressProtocol homeAddress = 3; public static final int HOMEADDRESS_FIELD_NUMBER = 3; private boolean hasHomeAddress; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol homeAddress_; + private akka.remote.protocol.RemoteProtocol.AddressProtocol homeAddress_; public boolean hasHomeAddress() { return hasHomeAddress; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getHomeAddress() { return homeAddress_; } + public akka.remote.protocol.RemoteProtocol.AddressProtocol getHomeAddress() { return homeAddress_; } // optional uint64 timeout = 4; public static final int TIMEOUT_FIELD_NUMBER = 4; @@ -273,7 +273,7 @@ public final class RemoteProtocol { public long getTimeout() { return timeout_; } private void initFields() { - homeAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); + homeAddress_ = akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); } public final boolean isInitialized() { if (!hasClassOrServiceName) return false; @@ -328,41 +328,41 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom( + public static 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( + public static 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) + public static 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( + public static 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) + public static 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( + public static 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) + public static akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -371,7 +371,7 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseDelimitedFrom( + public static akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -382,12 +382,12 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom( + public static 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( + public static akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -397,25 +397,25 @@ public final class RemoteProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol prototype) { + public static Builder newBuilder(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; + private akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol result; - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder() + // Construct using 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(); + builder.result = new akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol(); return builder; } - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol internalGetResult() { + protected akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol internalGetResult() { return result; } @@ -424,7 +424,7 @@ public final class RemoteProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol(); + result = new akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol(); return this; } @@ -434,24 +434,24 @@ public final class RemoteProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDescriptor(); + return 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 akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getDefaultInstanceForType() { + return akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol build() { + public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol buildParsed() + private akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -460,27 +460,27 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol buildPartial() { + public 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; + 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); + if (other instanceof akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol) { + return mergeFrom((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; + public Builder mergeFrom(akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol other) { + if (other == akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) return this; if (other.hasClassOrServiceName()) { setClassOrServiceName(other.getClassOrServiceName()); } @@ -527,7 +527,7 @@ public final class RemoteProtocol { break; } case 26: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(); if (hasHomeAddress()) { subBuilder.mergeFrom(getHomeAddress()); } @@ -590,10 +590,10 @@ public final class RemoteProtocol { public boolean hasHomeAddress() { return result.hasHomeAddress(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getHomeAddress() { + public akka.remote.protocol.RemoteProtocol.AddressProtocol getHomeAddress() { return result.getHomeAddress(); } - public Builder setHomeAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol value) { + public Builder setHomeAddress(akka.remote.protocol.RemoteProtocol.AddressProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -601,16 +601,16 @@ public final class RemoteProtocol { result.homeAddress_ = value; return this; } - public Builder setHomeAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder builderForValue) { + public Builder setHomeAddress(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) { + public Builder mergeHomeAddress(akka.remote.protocol.RemoteProtocol.AddressProtocol value) { if (result.hasHomeAddress() && - result.homeAddress_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) { + result.homeAddress_ != akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) { result.homeAddress_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(result.homeAddress_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(result.homeAddress_).mergeFrom(value).buildPartial(); } else { result.homeAddress_ = value; } @@ -619,7 +619,7 @@ public final class RemoteProtocol { } public Builder clearHomeAddress() { result.hasHomeAddress = false; - result.homeAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); + result.homeAddress_ = akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); return this; } @@ -646,7 +646,7 @@ public final class RemoteProtocol { static { defaultInstance = new RemoteActorRefProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -672,20 +672,20 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteTypedActorRefProtocol_descriptor; + return akka.remote.protocol.RemoteProtocol.internal_static_RemoteTypedActorRefProtocol_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteTypedActorRefProtocol_fieldAccessorTable; + return akka.remote.protocol.RemoteProtocol.internal_static_RemoteTypedActorRefProtocol_fieldAccessorTable; } // required .RemoteActorRefProtocol actorRef = 1; public static final int ACTORREF_FIELD_NUMBER = 1; private boolean hasActorRef; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol actorRef_; + private akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol actorRef_; public boolean hasActorRef() { return hasActorRef; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getActorRef() { return actorRef_; } + public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getActorRef() { return actorRef_; } // required string interfaceName = 2; public static final int INTERFACENAME_FIELD_NUMBER = 2; @@ -695,7 +695,7 @@ public final class RemoteProtocol { public java.lang.String getInterfaceName() { return interfaceName_; } private void initFields() { - actorRef_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + actorRef_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); } public final boolean isInitialized() { if (!hasActorRef) return false; @@ -735,41 +735,41 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol parseFrom( + public static akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol parseFrom( + public static akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol 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.RemoteTypedActorRefProtocol parseFrom(byte[] data) + public static akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol parseFrom( + public static akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol 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.RemoteTypedActorRefProtocol parseFrom(java.io.InputStream input) + public static akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol parseFrom( + public static akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol 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.RemoteTypedActorRefProtocol parseDelimitedFrom(java.io.InputStream input) + public static akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -778,7 +778,7 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol parseDelimitedFrom( + public static akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -789,12 +789,12 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol parseFrom( + public static akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol parseFrom( + public static akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -804,25 +804,25 @@ public final class RemoteProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol prototype) { + public static Builder newBuilder(akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol 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.RemoteTypedActorRefProtocol result; + private akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol result; - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.newBuilder() + // Construct using akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.newBuilder() private Builder() {} private static Builder create() { Builder builder = new Builder(); - builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol(); + builder.result = new akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol(); return builder; } - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol internalGetResult() { + protected akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol internalGetResult() { return result; } @@ -831,7 +831,7 @@ public final class RemoteProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol(); + result = new akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol(); return this; } @@ -841,24 +841,24 @@ public final class RemoteProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.getDescriptor(); + return akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.getDescriptor(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol getDefaultInstanceForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.getDefaultInstance(); + public akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol getDefaultInstanceForType() { + return akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol build() { + public akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol buildParsed() + private akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -867,27 +867,27 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol buildPartial() { + public akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol buildPartial() { if (result == null) { throw new IllegalStateException( "build() has already been called on this Builder."); } - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol returnMe = result; + akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol returnMe = result; result = null; return returnMe; } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol) { - return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol)other); + if (other instanceof akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol) { + return mergeFrom((akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol other) { - if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.getDefaultInstance()) return this; + public Builder mergeFrom(akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol other) { + if (other == akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.getDefaultInstance()) return this; if (other.hasActorRef()) { mergeActorRef(other.getActorRef()); } @@ -920,7 +920,7 @@ public final class RemoteProtocol { break; } case 10: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(); if (hasActorRef()) { subBuilder.mergeFrom(getActorRef()); } @@ -941,10 +941,10 @@ public final class RemoteProtocol { public boolean hasActorRef() { return result.hasActorRef(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getActorRef() { + public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getActorRef() { return result.getActorRef(); } - public Builder setActorRef(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { + public Builder setActorRef(akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -952,16 +952,16 @@ public final class RemoteProtocol { result.actorRef_ = value; return this; } - public Builder setActorRef(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder builderForValue) { + public Builder setActorRef(akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder builderForValue) { result.hasActorRef = true; result.actorRef_ = builderForValue.build(); return this; } - public Builder mergeActorRef(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { + public Builder mergeActorRef(akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { if (result.hasActorRef() && - result.actorRef_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) { + result.actorRef_ != akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) { result.actorRef_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(result.actorRef_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(result.actorRef_).mergeFrom(value).buildPartial(); } else { result.actorRef_ = value; } @@ -970,7 +970,7 @@ public final class RemoteProtocol { } public Builder clearActorRef() { result.hasActorRef = false; - result.actorRef_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + result.actorRef_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); return this; } @@ -1000,7 +1000,7 @@ public final class RemoteProtocol { static { defaultInstance = new RemoteTypedActorRefProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -1026,20 +1026,20 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_descriptor; + return 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; + return akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_fieldAccessorTable; } // required .UuidProtocol uuid = 1; public static final int UUID_FIELD_NUMBER = 1; private boolean hasUuid; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_; + private akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_; public boolean hasUuid() { return hasUuid; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return uuid_; } + public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return uuid_; } // required string id = 2; public static final int ID_FIELD_NUMBER = 2; @@ -1058,9 +1058,9 @@ public final class RemoteProtocol { // required .AddressProtocol originalAddress = 4; public static final int ORIGINALADDRESS_FIELD_NUMBER = 4; private boolean hasOriginalAddress; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol originalAddress_; + private akka.remote.protocol.RemoteProtocol.AddressProtocol originalAddress_; public boolean hasOriginalAddress() { return hasOriginalAddress; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getOriginalAddress() { return originalAddress_; } + public akka.remote.protocol.RemoteProtocol.AddressProtocol getOriginalAddress() { return originalAddress_; } // optional bytes actorInstance = 5; public static final int ACTORINSTANCE_FIELD_NUMBER = 5; @@ -1100,16 +1100,16 @@ public final class RemoteProtocol { // optional .LifeCycleProtocol lifeCycle = 10; public static final int LIFECYCLE_FIELD_NUMBER = 10; private boolean hasLifeCycle; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol lifeCycle_; + private akka.remote.protocol.RemoteProtocol.LifeCycleProtocol lifeCycle_; public boolean hasLifeCycle() { return hasLifeCycle; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getLifeCycle() { return lifeCycle_; } + public 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_; + private akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol supervisor_; public boolean hasSupervisor() { return hasSupervisor; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor() { return supervisor_; } + public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor() { return supervisor_; } // optional bytes hotswapStack = 12; public static final int HOTSWAPSTACK_FIELD_NUMBER = 12; @@ -1120,21 +1120,21 @@ public final class RemoteProtocol { // repeated .RemoteRequestProtocol messages = 13; public static final int MESSAGES_FIELD_NUMBER = 13; - private java.util.List messages_ = + private java.util.List messages_ = java.util.Collections.emptyList(); - public java.util.List getMessagesList() { + public java.util.List getMessagesList() { return messages_; } public int getMessagesCount() { return messages_.size(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getMessages(int index) { + public akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getMessages(int index) { return messages_.get(index); } private void initFields() { - uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); - originalAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); - lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); - supervisor_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + originalAddress_ = akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); + lifeCycle_ = akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); + supervisor_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); } public final boolean isInitialized() { if (!hasUuid) return false; @@ -1149,7 +1149,7 @@ public final class RemoteProtocol { if (hasSupervisor()) { if (!getSupervisor().isInitialized()) return false; } - for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol element : getMessagesList()) { + for (akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol element : getMessagesList()) { if (!element.isInitialized()) return false; } return true; @@ -1194,7 +1194,7 @@ public final class RemoteProtocol { if (hasHotswapStack()) { output.writeBytes(12, getHotswapStack()); } - for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol element : getMessagesList()) { + for (akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol element : getMessagesList()) { output.writeMessage(13, element); } getUnknownFields().writeTo(output); @@ -1254,7 +1254,7 @@ public final class RemoteProtocol { size += com.google.protobuf.CodedOutputStream .computeBytesSize(12, getHotswapStack()); } - for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol element : getMessagesList()) { + for (akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol element : getMessagesList()) { size += com.google.protobuf.CodedOutputStream .computeMessageSize(13, element); } @@ -1263,41 +1263,41 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom( + public static 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( + public static 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) + public static 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( + public static 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) + public static 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( + public static 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) + public static akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -1306,7 +1306,7 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseDelimitedFrom( + public static akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -1317,12 +1317,12 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom( + public static 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( + public static akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -1332,25 +1332,25 @@ public final class RemoteProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol prototype) { + public static Builder newBuilder(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; + private akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol result; - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder() + // Construct using 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(); + builder.result = new akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol(); return builder; } - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol internalGetResult() { + protected akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol internalGetResult() { return result; } @@ -1359,7 +1359,7 @@ public final class RemoteProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol(); + result = new akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol(); return this; } @@ -1369,24 +1369,24 @@ public final class RemoteProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDescriptor(); + return 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 akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getDefaultInstanceForType() { + return akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol build() { + public akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildParsed() + private akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -1395,7 +1395,7 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildPartial() { + public akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildPartial() { if (result == null) { throw new IllegalStateException( "build() has already been called on this Builder."); @@ -1404,22 +1404,22 @@ public final class RemoteProtocol { result.messages_ = java.util.Collections.unmodifiableList(result.messages_); } - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol returnMe = result; + 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); + if (other instanceof akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol) { + return mergeFrom((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; + public Builder mergeFrom(akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol other) { + if (other == akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance()) return this; if (other.hasUuid()) { mergeUuid(other.getUuid()); } @@ -1458,7 +1458,7 @@ public final class RemoteProtocol { } if (!other.messages_.isEmpty()) { if (result.messages_.isEmpty()) { - result.messages_ = new java.util.ArrayList(); + result.messages_ = new java.util.ArrayList(); } result.messages_.addAll(other.messages_); } @@ -1488,7 +1488,7 @@ public final class RemoteProtocol { break; } case 10: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(); if (hasUuid()) { subBuilder.mergeFrom(getUuid()); } @@ -1505,7 +1505,7 @@ public final class RemoteProtocol { break; } case 34: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(); if (hasOriginalAddress()) { subBuilder.mergeFrom(getOriginalAddress()); } @@ -1534,7 +1534,7 @@ public final class RemoteProtocol { break; } case 82: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder(); if (hasLifeCycle()) { subBuilder.mergeFrom(getLifeCycle()); } @@ -1543,7 +1543,7 @@ public final class RemoteProtocol { break; } case 90: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(); if (hasSupervisor()) { subBuilder.mergeFrom(getSupervisor()); } @@ -1556,7 +1556,7 @@ public final class RemoteProtocol { break; } case 106: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.newBuilder(); input.readMessage(subBuilder, extensionRegistry); addMessages(subBuilder.buildPartial()); break; @@ -1570,10 +1570,10 @@ public final class RemoteProtocol { public boolean hasUuid() { return result.hasUuid(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { + public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return result.getUuid(); } - public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) { + public Builder setUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -1581,16 +1581,16 @@ public final class RemoteProtocol { result.uuid_ = value; return this; } - public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { + public Builder setUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { result.hasUuid = true; result.uuid_ = builderForValue.build(); return this; } - public Builder mergeUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) { + public Builder mergeUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { if (result.hasUuid() && - result.uuid_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { + result.uuid_ != akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { result.uuid_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.uuid_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.uuid_).mergeFrom(value).buildPartial(); } else { result.uuid_ = value; } @@ -1599,7 +1599,7 @@ public final class RemoteProtocol { } public Builder clearUuid() { result.hasUuid = false; - result.uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + result.uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); return this; } @@ -1649,10 +1649,10 @@ public final class RemoteProtocol { public boolean hasOriginalAddress() { return result.hasOriginalAddress(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getOriginalAddress() { + public akka.remote.protocol.RemoteProtocol.AddressProtocol getOriginalAddress() { return result.getOriginalAddress(); } - public Builder setOriginalAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol value) { + public Builder setOriginalAddress(akka.remote.protocol.RemoteProtocol.AddressProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -1660,16 +1660,16 @@ public final class RemoteProtocol { result.originalAddress_ = value; return this; } - public Builder setOriginalAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder builderForValue) { + public Builder setOriginalAddress(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) { + public Builder mergeOriginalAddress(akka.remote.protocol.RemoteProtocol.AddressProtocol value) { if (result.hasOriginalAddress() && - result.originalAddress_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) { + result.originalAddress_ != akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) { result.originalAddress_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(result.originalAddress_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(result.originalAddress_).mergeFrom(value).buildPartial(); } else { result.originalAddress_ = value; } @@ -1678,7 +1678,7 @@ public final class RemoteProtocol { } public Builder clearOriginalAddress() { result.hasOriginalAddress = false; - result.originalAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); + result.originalAddress_ = akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); return this; } @@ -1782,10 +1782,10 @@ public final class RemoteProtocol { public boolean hasLifeCycle() { return result.hasLifeCycle(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getLifeCycle() { + public akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getLifeCycle() { return result.getLifeCycle(); } - public Builder setLifeCycle(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol value) { + public Builder setLifeCycle(akka.remote.protocol.RemoteProtocol.LifeCycleProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -1793,16 +1793,16 @@ public final class RemoteProtocol { result.lifeCycle_ = value; return this; } - public Builder setLifeCycle(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder builderForValue) { + public Builder setLifeCycle(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) { + public Builder mergeLifeCycle(akka.remote.protocol.RemoteProtocol.LifeCycleProtocol value) { if (result.hasLifeCycle() && - result.lifeCycle_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance()) { + result.lifeCycle_ != akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance()) { result.lifeCycle_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder(result.lifeCycle_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder(result.lifeCycle_).mergeFrom(value).buildPartial(); } else { result.lifeCycle_ = value; } @@ -1811,7 +1811,7 @@ public final class RemoteProtocol { } public Builder clearLifeCycle() { result.hasLifeCycle = false; - result.lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); + result.lifeCycle_ = akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); return this; } @@ -1819,10 +1819,10 @@ public final class RemoteProtocol { public boolean hasSupervisor() { return result.hasSupervisor(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor() { + public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor() { return result.getSupervisor(); } - public Builder setSupervisor(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { + public Builder setSupervisor(akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -1830,16 +1830,16 @@ public final class RemoteProtocol { result.supervisor_ = value; return this; } - public Builder setSupervisor(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder builderForValue) { + public Builder setSupervisor(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) { + public Builder mergeSupervisor(akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { if (result.hasSupervisor() && - result.supervisor_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) { + result.supervisor_ != akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) { result.supervisor_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(result.supervisor_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(result.supervisor_).mergeFrom(value).buildPartial(); } else { result.supervisor_ = value; } @@ -1848,7 +1848,7 @@ public final class RemoteProtocol { } public Builder clearSupervisor() { result.hasSupervisor = false; - result.supervisor_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + result.supervisor_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); return this; } @@ -1874,47 +1874,47 @@ public final class RemoteProtocol { } // repeated .RemoteRequestProtocol messages = 13; - public java.util.List getMessagesList() { + 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) { + public 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) { + public Builder setMessages(int index, 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) { + public Builder setMessages(int index, 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) { + public Builder addMessages(akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol value) { if (value == null) { throw new NullPointerException(); } if (result.messages_.isEmpty()) { - result.messages_ = new java.util.ArrayList(); + result.messages_ = new java.util.ArrayList(); } result.messages_.add(value); return this; } - public Builder addMessages(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder builderForValue) { + public Builder addMessages(akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder builderForValue) { if (result.messages_.isEmpty()) { - result.messages_ = new java.util.ArrayList(); + result.messages_ = new java.util.ArrayList(); } result.messages_.add(builderForValue.build()); return this; } public Builder addAllMessages( - java.lang.Iterable values) { + java.lang.Iterable values) { if (result.messages_.isEmpty()) { - result.messages_ = new java.util.ArrayList(); + result.messages_ = new java.util.ArrayList(); } super.addAll(values, result.messages_); return this; @@ -1929,7 +1929,7 @@ public final class RemoteProtocol { static { defaultInstance = new SerializedActorRefProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -1955,20 +1955,20 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedTypedActorRefProtocol_descriptor; + return akka.remote.protocol.RemoteProtocol.internal_static_SerializedTypedActorRefProtocol_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedTypedActorRefProtocol_fieldAccessorTable; + return akka.remote.protocol.RemoteProtocol.internal_static_SerializedTypedActorRefProtocol_fieldAccessorTable; } // required .SerializedActorRefProtocol actorRef = 1; public static final int ACTORREF_FIELD_NUMBER = 1; private boolean hasActorRef; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol actorRef_; + private akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol actorRef_; public boolean hasActorRef() { return hasActorRef; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getActorRef() { return actorRef_; } + public akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getActorRef() { return actorRef_; } // required string interfaceName = 2; public static final int INTERFACENAME_FIELD_NUMBER = 2; @@ -1978,7 +1978,7 @@ public final class RemoteProtocol { public java.lang.String getInterfaceName() { return interfaceName_; } private void initFields() { - actorRef_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance(); + actorRef_ = akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance(); } public final boolean isInitialized() { if (!hasActorRef) return false; @@ -2018,41 +2018,41 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseFrom( + public static akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseFrom( + public static akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol 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.SerializedTypedActorRefProtocol parseFrom(byte[] data) + public static akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseFrom( + public static akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol 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.SerializedTypedActorRefProtocol parseFrom(java.io.InputStream input) + public static akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseFrom( + public static akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol 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.SerializedTypedActorRefProtocol parseDelimitedFrom(java.io.InputStream input) + public static akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -2061,7 +2061,7 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseDelimitedFrom( + public static akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -2072,12 +2072,12 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseFrom( + public static akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseFrom( + public static akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -2087,25 +2087,25 @@ public final class RemoteProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol prototype) { + public static Builder newBuilder(akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol 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.SerializedTypedActorRefProtocol result; + private akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol result; - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.newBuilder() + // Construct using akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.newBuilder() private Builder() {} private static Builder create() { Builder builder = new Builder(); - builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol(); + builder.result = new akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol(); return builder; } - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol internalGetResult() { + protected akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol internalGetResult() { return result; } @@ -2114,7 +2114,7 @@ public final class RemoteProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol(); + result = new akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol(); return this; } @@ -2124,24 +2124,24 @@ public final class RemoteProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.getDescriptor(); + return akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.getDescriptor(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol getDefaultInstanceForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.getDefaultInstance(); + public akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol getDefaultInstanceForType() { + return akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol build() { + public akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol buildParsed() + private akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -2150,27 +2150,27 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol buildPartial() { + public akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol buildPartial() { if (result == null) { throw new IllegalStateException( "build() has already been called on this Builder."); } - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol returnMe = result; + akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol returnMe = result; result = null; return returnMe; } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol) { - return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol)other); + if (other instanceof akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol) { + return mergeFrom((akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol other) { - if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.getDefaultInstance()) return this; + public Builder mergeFrom(akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol other) { + if (other == akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.getDefaultInstance()) return this; if (other.hasActorRef()) { mergeActorRef(other.getActorRef()); } @@ -2203,7 +2203,7 @@ public final class RemoteProtocol { break; } case 10: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder(); if (hasActorRef()) { subBuilder.mergeFrom(getActorRef()); } @@ -2224,10 +2224,10 @@ public final class RemoteProtocol { public boolean hasActorRef() { return result.hasActorRef(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getActorRef() { + public akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getActorRef() { return result.getActorRef(); } - public Builder setActorRef(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol value) { + public Builder setActorRef(akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -2235,16 +2235,16 @@ public final class RemoteProtocol { result.actorRef_ = value; return this; } - public Builder setActorRef(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder builderForValue) { + public Builder setActorRef(akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder builderForValue) { result.hasActorRef = true; result.actorRef_ = builderForValue.build(); return this; } - public Builder mergeActorRef(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol value) { + public Builder mergeActorRef(akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol value) { if (result.hasActorRef() && - result.actorRef_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance()) { + result.actorRef_ != akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance()) { result.actorRef_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder(result.actorRef_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder(result.actorRef_).mergeFrom(value).buildPartial(); } else { result.actorRef_ = value; } @@ -2253,7 +2253,7 @@ public final class RemoteProtocol { } public Builder clearActorRef() { result.hasActorRef = false; - result.actorRef_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance(); + result.actorRef_ = akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance(); return this; } @@ -2283,7 +2283,7 @@ public final class RemoteProtocol { static { defaultInstance = new SerializedTypedActorRefProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -2309,20 +2309,20 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MessageProtocol_descriptor; + return 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; + return 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_; + private akka.remote.protocol.RemoteProtocol.SerializationSchemeType serializationScheme_; public boolean hasSerializationScheme() { return hasSerializationScheme; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType getSerializationScheme() { return serializationScheme_; } + public akka.remote.protocol.RemoteProtocol.SerializationSchemeType getSerializationScheme() { return serializationScheme_; } // required bytes message = 2; public static final int MESSAGE_FIELD_NUMBER = 2; @@ -2339,7 +2339,7 @@ public final class RemoteProtocol { public com.google.protobuf.ByteString getMessageManifest() { return messageManifest_; } private void initFields() { - serializationScheme_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA; + serializationScheme_ = akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA; } public final boolean isInitialized() { if (!hasSerializationScheme) return false; @@ -2385,41 +2385,41 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom( + public static 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( + public static 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) + public static 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( + public static 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) + public static 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( + public static 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) + public static akka.remote.protocol.RemoteProtocol.MessageProtocol parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -2428,7 +2428,7 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseDelimitedFrom( + public static akka.remote.protocol.RemoteProtocol.MessageProtocol parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -2439,12 +2439,12 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom( + public static 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( + public static akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -2454,25 +2454,25 @@ public final class RemoteProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol prototype) { + public static Builder newBuilder(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; + private akka.remote.protocol.RemoteProtocol.MessageProtocol result; - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder() + // Construct using 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(); + builder.result = new akka.remote.protocol.RemoteProtocol.MessageProtocol(); return builder; } - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol internalGetResult() { + protected akka.remote.protocol.RemoteProtocol.MessageProtocol internalGetResult() { return result; } @@ -2481,7 +2481,7 @@ public final class RemoteProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol(); + result = new akka.remote.protocol.RemoteProtocol.MessageProtocol(); return this; } @@ -2491,24 +2491,24 @@ public final class RemoteProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDescriptor(); + return 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 akka.remote.protocol.RemoteProtocol.MessageProtocol getDefaultInstanceForType() { + return akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol build() { + public akka.remote.protocol.RemoteProtocol.MessageProtocol build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol buildParsed() + private akka.remote.protocol.RemoteProtocol.MessageProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -2517,27 +2517,27 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol buildPartial() { + public 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; + 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); + if (other instanceof akka.remote.protocol.RemoteProtocol.MessageProtocol) { + return mergeFrom((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; + public Builder mergeFrom(akka.remote.protocol.RemoteProtocol.MessageProtocol other) { + if (other == akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) return this; if (other.hasSerializationScheme()) { setSerializationScheme(other.getSerializationScheme()); } @@ -2574,7 +2574,7 @@ public final class RemoteProtocol { } case 8: { int rawValue = input.readEnum(); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType value = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.valueOf(rawValue); + akka.remote.protocol.RemoteProtocol.SerializationSchemeType value = akka.remote.protocol.RemoteProtocol.SerializationSchemeType.valueOf(rawValue); if (value == null) { unknownFields.mergeVarintField(1, rawValue); } else { @@ -2599,10 +2599,10 @@ public final class RemoteProtocol { public boolean hasSerializationScheme() { return result.hasSerializationScheme(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType getSerializationScheme() { + public akka.remote.protocol.RemoteProtocol.SerializationSchemeType getSerializationScheme() { return result.getSerializationScheme(); } - public Builder setSerializationScheme(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType value) { + public Builder setSerializationScheme(akka.remote.protocol.RemoteProtocol.SerializationSchemeType value) { if (value == null) { throw new NullPointerException(); } @@ -2612,7 +2612,7 @@ public final class RemoteProtocol { } public Builder clearSerializationScheme() { result.hasSerializationScheme = false; - result.serializationScheme_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA; + result.serializationScheme_ = akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA; return this; } @@ -2663,7 +2663,7 @@ public final class RemoteProtocol { static { defaultInstance = new MessageProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -2689,20 +2689,20 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_descriptor; + return 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; + return akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_fieldAccessorTable; } // required .UuidProtocol uuid = 1; public static final int UUID_FIELD_NUMBER = 1; private boolean hasUuid; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_; + private akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_; public boolean hasUuid() { return hasUuid; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return uuid_; } + public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return uuid_; } // required string target = 2; public static final int TARGET_FIELD_NUMBER = 2; @@ -2721,16 +2721,16 @@ public final class RemoteProtocol { // required .ActorType actorType = 4; public static final int ACTORTYPE_FIELD_NUMBER = 4; private boolean hasActorType; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType actorType_; + private akka.remote.protocol.RemoteProtocol.ActorType actorType_; public boolean hasActorType() { return hasActorType; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType getActorType() { return actorType_; } + public 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_; + private akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol typedActorInfo_; public boolean hasTypedActorInfo() { return hasTypedActorInfo; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getTypedActorInfo() { return typedActorInfo_; } + public akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getTypedActorInfo() { return typedActorInfo_; } // optional string id = 6; public static final int ID_FIELD_NUMBER = 6; @@ -2740,9 +2740,9 @@ public final class RemoteProtocol { public java.lang.String getId() { return id_; } private void initFields() { - uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); - actorType_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.SCALA_ACTOR; - typedActorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance(); + uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + actorType_ = akka.remote.protocol.RemoteProtocol.ActorType.SCALA_ACTOR; + typedActorInfo_ = akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance(); } public final boolean isInitialized() { if (!hasUuid) return false; @@ -2815,41 +2815,41 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom( + public static 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( + public static 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) + public static 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( + public static 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) + public static 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( + public static 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) + public static akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -2858,7 +2858,7 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseDelimitedFrom( + public static akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -2869,12 +2869,12 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom( + public static 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( + public static akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -2884,25 +2884,25 @@ public final class RemoteProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol prototype) { + public static Builder newBuilder(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; + private akka.remote.protocol.RemoteProtocol.ActorInfoProtocol result; - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder() + // Construct using 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(); + builder.result = new akka.remote.protocol.RemoteProtocol.ActorInfoProtocol(); return builder; } - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol internalGetResult() { + protected akka.remote.protocol.RemoteProtocol.ActorInfoProtocol internalGetResult() { return result; } @@ -2911,7 +2911,7 @@ public final class RemoteProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol(); + result = new akka.remote.protocol.RemoteProtocol.ActorInfoProtocol(); return this; } @@ -2921,24 +2921,24 @@ public final class RemoteProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDescriptor(); + return 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 akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getDefaultInstanceForType() { + return akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol build() { + public akka.remote.protocol.RemoteProtocol.ActorInfoProtocol build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol buildParsed() + private akka.remote.protocol.RemoteProtocol.ActorInfoProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -2947,27 +2947,27 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol buildPartial() { + public 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; + 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); + if (other instanceof akka.remote.protocol.RemoteProtocol.ActorInfoProtocol) { + return mergeFrom((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; + public Builder mergeFrom(akka.remote.protocol.RemoteProtocol.ActorInfoProtocol other) { + if (other == akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance()) return this; if (other.hasUuid()) { mergeUuid(other.getUuid()); } @@ -3012,7 +3012,7 @@ public final class RemoteProtocol { break; } case 10: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(); if (hasUuid()) { subBuilder.mergeFrom(getUuid()); } @@ -3030,7 +3030,7 @@ public final class RemoteProtocol { } case 32: { int rawValue = input.readEnum(); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType value = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.valueOf(rawValue); + akka.remote.protocol.RemoteProtocol.ActorType value = akka.remote.protocol.RemoteProtocol.ActorType.valueOf(rawValue); if (value == null) { unknownFields.mergeVarintField(4, rawValue); } else { @@ -3039,7 +3039,7 @@ public final class RemoteProtocol { break; } case 42: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder(); if (hasTypedActorInfo()) { subBuilder.mergeFrom(getTypedActorInfo()); } @@ -3060,10 +3060,10 @@ public final class RemoteProtocol { public boolean hasUuid() { return result.hasUuid(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { + public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return result.getUuid(); } - public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) { + public Builder setUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -3071,16 +3071,16 @@ public final class RemoteProtocol { result.uuid_ = value; return this; } - public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { + public Builder setUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { result.hasUuid = true; result.uuid_ = builderForValue.build(); return this; } - public Builder mergeUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) { + public Builder mergeUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { if (result.hasUuid() && - result.uuid_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { + result.uuid_ != akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { result.uuid_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.uuid_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.uuid_).mergeFrom(value).buildPartial(); } else { result.uuid_ = value; } @@ -3089,7 +3089,7 @@ public final class RemoteProtocol { } public Builder clearUuid() { result.hasUuid = false; - result.uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + result.uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); return this; } @@ -3136,10 +3136,10 @@ public final class RemoteProtocol { public boolean hasActorType() { return result.hasActorType(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType getActorType() { + public akka.remote.protocol.RemoteProtocol.ActorType getActorType() { return result.getActorType(); } - public Builder setActorType(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType value) { + public Builder setActorType(akka.remote.protocol.RemoteProtocol.ActorType value) { if (value == null) { throw new NullPointerException(); } @@ -3149,7 +3149,7 @@ public final class RemoteProtocol { } public Builder clearActorType() { result.hasActorType = false; - result.actorType_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.SCALA_ACTOR; + result.actorType_ = akka.remote.protocol.RemoteProtocol.ActorType.SCALA_ACTOR; return this; } @@ -3157,10 +3157,10 @@ public final class RemoteProtocol { public boolean hasTypedActorInfo() { return result.hasTypedActorInfo(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getTypedActorInfo() { + public akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getTypedActorInfo() { return result.getTypedActorInfo(); } - public Builder setTypedActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol value) { + public Builder setTypedActorInfo(akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -3168,16 +3168,16 @@ public final class RemoteProtocol { result.typedActorInfo_ = value; return this; } - public Builder setTypedActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.Builder builderForValue) { + public Builder setTypedActorInfo(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) { + public Builder mergeTypedActorInfo(akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol value) { if (result.hasTypedActorInfo() && - result.typedActorInfo_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance()) { + result.typedActorInfo_ != akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance()) { result.typedActorInfo_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder(result.typedActorInfo_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder(result.typedActorInfo_).mergeFrom(value).buildPartial(); } else { result.typedActorInfo_ = value; } @@ -3186,7 +3186,7 @@ public final class RemoteProtocol { } public Builder clearTypedActorInfo() { result.hasTypedActorInfo = false; - result.typedActorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance(); + result.typedActorInfo_ = akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance(); return this; } @@ -3216,7 +3216,7 @@ public final class RemoteProtocol { static { defaultInstance = new ActorInfoProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -3242,12 +3242,12 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_TypedActorInfoProtocol_descriptor; + return 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; + return akka.remote.protocol.RemoteProtocol.internal_static_TypedActorInfoProtocol_fieldAccessorTable; } // required string interface = 1; @@ -3303,41 +3303,41 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom( + public static 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( + public static 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) + public static 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( + public static 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) + public static 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( + public static 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) + public static akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -3346,7 +3346,7 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseDelimitedFrom( + public static akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -3357,12 +3357,12 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom( + public static 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( + public static akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -3372,25 +3372,25 @@ public final class RemoteProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol prototype) { + public static Builder newBuilder(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; + private akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol result; - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder() + // Construct using 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(); + builder.result = new akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol(); return builder; } - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol internalGetResult() { + protected akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol internalGetResult() { return result; } @@ -3399,7 +3399,7 @@ public final class RemoteProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol(); + result = new akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol(); return this; } @@ -3409,24 +3409,24 @@ public final class RemoteProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDescriptor(); + return 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 akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getDefaultInstanceForType() { + return akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol build() { + public akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol buildParsed() + private akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -3435,27 +3435,27 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol buildPartial() { + public 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; + 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); + if (other instanceof akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol) { + return mergeFrom((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; + public Builder mergeFrom(akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol other) { + if (other == akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance()) return this; if (other.hasInterface()) { setInterface(other.getInterface()); } @@ -3547,7 +3547,7 @@ public final class RemoteProtocol { static { defaultInstance = new TypedActorInfoProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -3573,34 +3573,34 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteRequestProtocol_descriptor; + return 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; + return akka.remote.protocol.RemoteProtocol.internal_static_RemoteRequestProtocol_fieldAccessorTable; } // required .UuidProtocol uuid = 1; public static final int UUID_FIELD_NUMBER = 1; private boolean hasUuid; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_; + private akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_; public boolean hasUuid() { return hasUuid; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return uuid_; } + public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return uuid_; } // required .MessageProtocol message = 2; public static final int MESSAGE_FIELD_NUMBER = 2; private boolean hasMessage; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol message_; + private akka.remote.protocol.RemoteProtocol.MessageProtocol message_; public boolean hasMessage() { return hasMessage; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { return message_; } + public 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_; + private akka.remote.protocol.RemoteProtocol.ActorInfoProtocol actorInfo_; public boolean hasActorInfo() { return hasActorInfo; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo() { return actorInfo_; } + public akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo() { return actorInfo_; } // required bool isOneWay = 4; public static final int ISONEWAY_FIELD_NUMBER = 4; @@ -3612,26 +3612,26 @@ public final class RemoteProtocol { // optional .UuidProtocol supervisorUuid = 5; public static final int SUPERVISORUUID_FIELD_NUMBER = 5; private boolean hasSupervisorUuid; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol supervisorUuid_; + private akka.remote.protocol.RemoteProtocol.UuidProtocol supervisorUuid_; public boolean hasSupervisorUuid() { return hasSupervisorUuid; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getSupervisorUuid() { return supervisorUuid_; } + public akka.remote.protocol.RemoteProtocol.UuidProtocol 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_; + private akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol sender_; public boolean hasSender() { return hasSender; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSender() { return sender_; } + public 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_ = + private java.util.List metadata_ = java.util.Collections.emptyList(); - public java.util.List getMetadataList() { + public java.util.List getMetadataList() { return metadata_; } public int getMetadataCount() { return metadata_.size(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getMetadata(int index) { + public akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getMetadata(int index) { return metadata_.get(index); } @@ -3643,11 +3643,11 @@ public final class RemoteProtocol { public java.lang.String getCookie() { return cookie_; } private void initFields() { - uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); - message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); - actorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); - supervisorUuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); - sender_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + message_ = akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); + actorInfo_ = akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); + supervisorUuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + sender_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); } public final boolean isInitialized() { if (!hasUuid) return false; @@ -3663,7 +3663,7 @@ public final class RemoteProtocol { if (hasSender()) { if (!getSender().isInitialized()) return false; } - for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { + for (akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { if (!element.isInitialized()) return false; } return true; @@ -3690,7 +3690,7 @@ public final class RemoteProtocol { if (hasSender()) { output.writeMessage(6, getSender()); } - for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { + for (akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { output.writeMessage(7, element); } if (hasCookie()) { @@ -3729,7 +3729,7 @@ public final class RemoteProtocol { size += com.google.protobuf.CodedOutputStream .computeMessageSize(6, getSender()); } - for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { + for (akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { size += com.google.protobuf.CodedOutputStream .computeMessageSize(7, element); } @@ -3742,41 +3742,41 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom( + public static 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( + public static 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) + public static 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( + public static 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) + public static 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( + public static 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) + public static akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -3785,7 +3785,7 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseDelimitedFrom( + public static akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -3796,12 +3796,12 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom( + public static 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( + public static akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -3811,25 +3811,25 @@ public final class RemoteProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol prototype) { + public static Builder newBuilder(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; + private akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol result; - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.newBuilder() + // Construct using 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(); + builder.result = new akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol(); return builder; } - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol internalGetResult() { + protected akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol internalGetResult() { return result; } @@ -3838,7 +3838,7 @@ public final class RemoteProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol(); + result = new akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol(); return this; } @@ -3848,24 +3848,24 @@ public final class RemoteProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDescriptor(); + return 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 akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getDefaultInstanceForType() { + return akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol build() { + public akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol buildParsed() + private akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -3874,7 +3874,7 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol buildPartial() { + public akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol buildPartial() { if (result == null) { throw new IllegalStateException( "build() has already been called on this Builder."); @@ -3883,22 +3883,22 @@ public final class RemoteProtocol { result.metadata_ = java.util.Collections.unmodifiableList(result.metadata_); } - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol returnMe = result; + 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); + if (other instanceof akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol) { + return mergeFrom((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; + public Builder mergeFrom(akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol other) { + if (other == akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDefaultInstance()) return this; if (other.hasUuid()) { mergeUuid(other.getUuid()); } @@ -3919,7 +3919,7 @@ public final class RemoteProtocol { } if (!other.metadata_.isEmpty()) { if (result.metadata_.isEmpty()) { - result.metadata_ = new java.util.ArrayList(); + result.metadata_ = new java.util.ArrayList(); } result.metadata_.addAll(other.metadata_); } @@ -3952,7 +3952,7 @@ public final class RemoteProtocol { break; } case 10: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(); if (hasUuid()) { subBuilder.mergeFrom(getUuid()); } @@ -3961,7 +3961,7 @@ public final class RemoteProtocol { break; } case 18: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(); if (hasMessage()) { subBuilder.mergeFrom(getMessage()); } @@ -3970,7 +3970,7 @@ public final class RemoteProtocol { break; } case 26: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder(); if (hasActorInfo()) { subBuilder.mergeFrom(getActorInfo()); } @@ -3983,7 +3983,7 @@ public final class RemoteProtocol { break; } case 42: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(); if (hasSupervisorUuid()) { subBuilder.mergeFrom(getSupervisorUuid()); } @@ -3992,7 +3992,7 @@ public final class RemoteProtocol { break; } case 50: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(); if (hasSender()) { subBuilder.mergeFrom(getSender()); } @@ -4001,7 +4001,7 @@ public final class RemoteProtocol { break; } case 58: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.newBuilder(); input.readMessage(subBuilder, extensionRegistry); addMetadata(subBuilder.buildPartial()); break; @@ -4019,10 +4019,10 @@ public final class RemoteProtocol { public boolean hasUuid() { return result.hasUuid(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { + public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return result.getUuid(); } - public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) { + public Builder setUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -4030,16 +4030,16 @@ public final class RemoteProtocol { result.uuid_ = value; return this; } - public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { + public Builder setUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { result.hasUuid = true; result.uuid_ = builderForValue.build(); return this; } - public Builder mergeUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) { + public Builder mergeUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { if (result.hasUuid() && - result.uuid_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { + result.uuid_ != akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { result.uuid_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.uuid_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.uuid_).mergeFrom(value).buildPartial(); } else { result.uuid_ = value; } @@ -4048,7 +4048,7 @@ public final class RemoteProtocol { } public Builder clearUuid() { result.hasUuid = false; - result.uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + result.uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); return this; } @@ -4056,10 +4056,10 @@ public final class RemoteProtocol { public boolean hasMessage() { return result.hasMessage(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { + public akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { return result.getMessage(); } - public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol value) { + public Builder setMessage(akka.remote.protocol.RemoteProtocol.MessageProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -4067,16 +4067,16 @@ public final class RemoteProtocol { result.message_ = value; return this; } - public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder builderForValue) { + public Builder setMessage(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) { + public Builder mergeMessage(akka.remote.protocol.RemoteProtocol.MessageProtocol value) { if (result.hasMessage() && - result.message_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) { + result.message_ != akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) { result.message_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(result.message_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(result.message_).mergeFrom(value).buildPartial(); } else { result.message_ = value; } @@ -4085,7 +4085,7 @@ public final class RemoteProtocol { } public Builder clearMessage() { result.hasMessage = false; - result.message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); + result.message_ = akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); return this; } @@ -4093,10 +4093,10 @@ public final class RemoteProtocol { public boolean hasActorInfo() { return result.hasActorInfo(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo() { + public akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo() { return result.getActorInfo(); } - public Builder setActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol value) { + public Builder setActorInfo(akka.remote.protocol.RemoteProtocol.ActorInfoProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -4104,16 +4104,16 @@ public final class RemoteProtocol { result.actorInfo_ = value; return this; } - public Builder setActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder builderForValue) { + public Builder setActorInfo(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) { + public Builder mergeActorInfo(akka.remote.protocol.RemoteProtocol.ActorInfoProtocol value) { if (result.hasActorInfo() && - result.actorInfo_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance()) { + result.actorInfo_ != akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance()) { result.actorInfo_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder(result.actorInfo_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder(result.actorInfo_).mergeFrom(value).buildPartial(); } else { result.actorInfo_ = value; } @@ -4122,7 +4122,7 @@ public final class RemoteProtocol { } public Builder clearActorInfo() { result.hasActorInfo = false; - result.actorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); + result.actorInfo_ = akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); return this; } @@ -4148,10 +4148,10 @@ public final class RemoteProtocol { public boolean hasSupervisorUuid() { return result.hasSupervisorUuid(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getSupervisorUuid() { + public akka.remote.protocol.RemoteProtocol.UuidProtocol getSupervisorUuid() { return result.getSupervisorUuid(); } - public Builder setSupervisorUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) { + public Builder setSupervisorUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -4159,16 +4159,16 @@ public final class RemoteProtocol { result.supervisorUuid_ = value; return this; } - public Builder setSupervisorUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { + public Builder setSupervisorUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { result.hasSupervisorUuid = true; result.supervisorUuid_ = builderForValue.build(); return this; } - public Builder mergeSupervisorUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) { + public Builder mergeSupervisorUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { if (result.hasSupervisorUuid() && - result.supervisorUuid_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { + result.supervisorUuid_ != akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { result.supervisorUuid_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.supervisorUuid_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.supervisorUuid_).mergeFrom(value).buildPartial(); } else { result.supervisorUuid_ = value; } @@ -4177,7 +4177,7 @@ public final class RemoteProtocol { } public Builder clearSupervisorUuid() { result.hasSupervisorUuid = false; - result.supervisorUuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + result.supervisorUuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); return this; } @@ -4185,10 +4185,10 @@ public final class RemoteProtocol { public boolean hasSender() { return result.hasSender(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSender() { + public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSender() { return result.getSender(); } - public Builder setSender(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { + public Builder setSender(akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -4196,16 +4196,16 @@ public final class RemoteProtocol { result.sender_ = value; return this; } - public Builder setSender(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder builderForValue) { + public Builder setSender(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) { + public Builder mergeSender(akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { if (result.hasSender() && - result.sender_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) { + result.sender_ != akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) { result.sender_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(result.sender_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(result.sender_).mergeFrom(value).buildPartial(); } else { result.sender_ = value; } @@ -4214,52 +4214,52 @@ public final class RemoteProtocol { } public Builder clearSender() { result.hasSender = false; - result.sender_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + result.sender_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); return this; } // repeated .MetadataEntryProtocol metadata = 7; - public java.util.List getMetadataList() { + 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) { + public 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) { + public Builder setMetadata(int index, 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) { + public Builder setMetadata(int index, 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) { + public Builder addMetadata(akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol value) { if (value == null) { throw new NullPointerException(); } if (result.metadata_.isEmpty()) { - result.metadata_ = new java.util.ArrayList(); + result.metadata_ = new java.util.ArrayList(); } result.metadata_.add(value); return this; } - public Builder addMetadata(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) { + public Builder addMetadata(akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) { if (result.metadata_.isEmpty()) { - result.metadata_ = new java.util.ArrayList(); + result.metadata_ = new java.util.ArrayList(); } result.metadata_.add(builderForValue.build()); return this; } public Builder addAllMetadata( - java.lang.Iterable values) { + java.lang.Iterable values) { if (result.metadata_.isEmpty()) { - result.metadata_ = new java.util.ArrayList(); + result.metadata_ = new java.util.ArrayList(); } super.addAll(values, result.metadata_); return this; @@ -4295,7 +4295,7 @@ public final class RemoteProtocol { static { defaultInstance = new RemoteRequestProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -4321,41 +4321,41 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteReplyProtocol_descriptor; + return 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; + return akka.remote.protocol.RemoteProtocol.internal_static_RemoteReplyProtocol_fieldAccessorTable; } // required .UuidProtocol uuid = 1; public static final int UUID_FIELD_NUMBER = 1; private boolean hasUuid; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_; + private akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_; public boolean hasUuid() { return hasUuid; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return uuid_; } + public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return uuid_; } // optional .MessageProtocol message = 2; public static final int MESSAGE_FIELD_NUMBER = 2; private boolean hasMessage; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol message_; + private akka.remote.protocol.RemoteProtocol.MessageProtocol message_; public boolean hasMessage() { return hasMessage; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { return message_; } + public 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_; + private akka.remote.protocol.RemoteProtocol.ExceptionProtocol exception_; public boolean hasException() { return hasException; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() { return exception_; } + public akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() { return exception_; } // optional .UuidProtocol supervisorUuid = 4; public static final int SUPERVISORUUID_FIELD_NUMBER = 4; private boolean hasSupervisorUuid; - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol supervisorUuid_; + private akka.remote.protocol.RemoteProtocol.UuidProtocol supervisorUuid_; public boolean hasSupervisorUuid() { return hasSupervisorUuid; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getSupervisorUuid() { return supervisorUuid_; } + public akka.remote.protocol.RemoteProtocol.UuidProtocol getSupervisorUuid() { return supervisorUuid_; } // required bool isActor = 5; public static final int ISACTOR_FIELD_NUMBER = 5; @@ -4373,13 +4373,13 @@ public final class RemoteProtocol { // repeated .MetadataEntryProtocol metadata = 7; public static final int METADATA_FIELD_NUMBER = 7; - private java.util.List metadata_ = + private java.util.List metadata_ = java.util.Collections.emptyList(); - public java.util.List getMetadataList() { + public java.util.List getMetadataList() { return metadata_; } public int getMetadataCount() { return metadata_.size(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getMetadata(int index) { + public akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getMetadata(int index) { return metadata_.get(index); } @@ -4391,10 +4391,10 @@ public final class RemoteProtocol { public java.lang.String getCookie() { return cookie_; } private void initFields() { - uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); - message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); - exception_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); - supervisorUuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + message_ = akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); + exception_ = akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); + supervisorUuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); } public final boolean isInitialized() { if (!hasUuid) return false; @@ -4410,7 +4410,7 @@ public final class RemoteProtocol { if (hasSupervisorUuid()) { if (!getSupervisorUuid().isInitialized()) return false; } - for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { + for (akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { if (!element.isInitialized()) return false; } return true; @@ -4437,7 +4437,7 @@ public final class RemoteProtocol { if (hasIsSuccessful()) { output.writeBool(6, getIsSuccessful()); } - for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { + for (akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { output.writeMessage(7, element); } if (hasCookie()) { @@ -4476,7 +4476,7 @@ public final class RemoteProtocol { size += com.google.protobuf.CodedOutputStream .computeBoolSize(6, getIsSuccessful()); } - for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { + for (akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { size += com.google.protobuf.CodedOutputStream .computeMessageSize(7, element); } @@ -4489,41 +4489,41 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom( + public static 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( + public static 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) + public static 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( + public static 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) + public static 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( + public static 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) + public static akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -4532,7 +4532,7 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseDelimitedFrom( + public static akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -4543,12 +4543,12 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom( + public static 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( + public static akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -4558,25 +4558,25 @@ public final class RemoteProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol prototype) { + public static Builder newBuilder(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; + private akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol result; - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.newBuilder() + // Construct using 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(); + builder.result = new akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol(); return builder; } - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol internalGetResult() { + protected akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol internalGetResult() { return result; } @@ -4585,7 +4585,7 @@ public final class RemoteProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol(); + result = new akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol(); return this; } @@ -4595,24 +4595,24 @@ public final class RemoteProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDescriptor(); + return 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 akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol getDefaultInstanceForType() { + return akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol build() { + public akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol buildParsed() + private akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -4621,7 +4621,7 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol buildPartial() { + public akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol buildPartial() { if (result == null) { throw new IllegalStateException( "build() has already been called on this Builder."); @@ -4630,22 +4630,22 @@ public final class RemoteProtocol { result.metadata_ = java.util.Collections.unmodifiableList(result.metadata_); } - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol returnMe = result; + 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); + if (other instanceof akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol) { + return mergeFrom((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; + public Builder mergeFrom(akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol other) { + if (other == akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDefaultInstance()) return this; if (other.hasUuid()) { mergeUuid(other.getUuid()); } @@ -4666,7 +4666,7 @@ public final class RemoteProtocol { } if (!other.metadata_.isEmpty()) { if (result.metadata_.isEmpty()) { - result.metadata_ = new java.util.ArrayList(); + result.metadata_ = new java.util.ArrayList(); } result.metadata_.addAll(other.metadata_); } @@ -4699,7 +4699,7 @@ public final class RemoteProtocol { break; } case 10: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(); if (hasUuid()) { subBuilder.mergeFrom(getUuid()); } @@ -4708,7 +4708,7 @@ public final class RemoteProtocol { break; } case 18: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(); if (hasMessage()) { subBuilder.mergeFrom(getMessage()); } @@ -4717,7 +4717,7 @@ public final class RemoteProtocol { break; } case 26: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder(); if (hasException()) { subBuilder.mergeFrom(getException()); } @@ -4726,7 +4726,7 @@ public final class RemoteProtocol { break; } case 34: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(); if (hasSupervisorUuid()) { subBuilder.mergeFrom(getSupervisorUuid()); } @@ -4743,7 +4743,7 @@ public final class RemoteProtocol { break; } case 58: { - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.newBuilder(); + akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.newBuilder(); input.readMessage(subBuilder, extensionRegistry); addMetadata(subBuilder.buildPartial()); break; @@ -4761,10 +4761,10 @@ public final class RemoteProtocol { public boolean hasUuid() { return result.hasUuid(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { + public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return result.getUuid(); } - public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) { + public Builder setUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -4772,16 +4772,16 @@ public final class RemoteProtocol { result.uuid_ = value; return this; } - public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { + public Builder setUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { result.hasUuid = true; result.uuid_ = builderForValue.build(); return this; } - public Builder mergeUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) { + public Builder mergeUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { if (result.hasUuid() && - result.uuid_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { + result.uuid_ != akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { result.uuid_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.uuid_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.uuid_).mergeFrom(value).buildPartial(); } else { result.uuid_ = value; } @@ -4790,7 +4790,7 @@ public final class RemoteProtocol { } public Builder clearUuid() { result.hasUuid = false; - result.uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + result.uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); return this; } @@ -4798,10 +4798,10 @@ public final class RemoteProtocol { public boolean hasMessage() { return result.hasMessage(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { + public akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { return result.getMessage(); } - public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol value) { + public Builder setMessage(akka.remote.protocol.RemoteProtocol.MessageProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -4809,16 +4809,16 @@ public final class RemoteProtocol { result.message_ = value; return this; } - public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder builderForValue) { + public Builder setMessage(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) { + public Builder mergeMessage(akka.remote.protocol.RemoteProtocol.MessageProtocol value) { if (result.hasMessage() && - result.message_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) { + result.message_ != akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) { result.message_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(result.message_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(result.message_).mergeFrom(value).buildPartial(); } else { result.message_ = value; } @@ -4827,7 +4827,7 @@ public final class RemoteProtocol { } public Builder clearMessage() { result.hasMessage = false; - result.message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); + result.message_ = akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); return this; } @@ -4835,10 +4835,10 @@ public final class RemoteProtocol { public boolean hasException() { return result.hasException(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() { + public akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() { return result.getException(); } - public Builder setException(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol value) { + public Builder setException(akka.remote.protocol.RemoteProtocol.ExceptionProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -4846,16 +4846,16 @@ public final class RemoteProtocol { result.exception_ = value; return this; } - public Builder setException(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder builderForValue) { + public Builder setException(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) { + public Builder mergeException(akka.remote.protocol.RemoteProtocol.ExceptionProtocol value) { if (result.hasException() && - result.exception_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance()) { + result.exception_ != akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance()) { result.exception_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder(result.exception_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder(result.exception_).mergeFrom(value).buildPartial(); } else { result.exception_ = value; } @@ -4864,7 +4864,7 @@ public final class RemoteProtocol { } public Builder clearException() { result.hasException = false; - result.exception_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); + result.exception_ = akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); return this; } @@ -4872,10 +4872,10 @@ public final class RemoteProtocol { public boolean hasSupervisorUuid() { return result.hasSupervisorUuid(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getSupervisorUuid() { + public akka.remote.protocol.RemoteProtocol.UuidProtocol getSupervisorUuid() { return result.getSupervisorUuid(); } - public Builder setSupervisorUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) { + public Builder setSupervisorUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { if (value == null) { throw new NullPointerException(); } @@ -4883,16 +4883,16 @@ public final class RemoteProtocol { result.supervisorUuid_ = value; return this; } - public Builder setSupervisorUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { + public Builder setSupervisorUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { result.hasSupervisorUuid = true; result.supervisorUuid_ = builderForValue.build(); return this; } - public Builder mergeSupervisorUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) { + public Builder mergeSupervisorUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { if (result.hasSupervisorUuid() && - result.supervisorUuid_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { + result.supervisorUuid_ != akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { result.supervisorUuid_ = - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.supervisorUuid_).mergeFrom(value).buildPartial(); + akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.supervisorUuid_).mergeFrom(value).buildPartial(); } else { result.supervisorUuid_ = value; } @@ -4901,7 +4901,7 @@ public final class RemoteProtocol { } public Builder clearSupervisorUuid() { result.hasSupervisorUuid = false; - result.supervisorUuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + result.supervisorUuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); return this; } @@ -4942,47 +4942,47 @@ public final class RemoteProtocol { } // repeated .MetadataEntryProtocol metadata = 7; - public java.util.List getMetadataList() { + 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) { + public 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) { + public Builder setMetadata(int index, 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) { + public Builder setMetadata(int index, 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) { + public Builder addMetadata(akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol value) { if (value == null) { throw new NullPointerException(); } if (result.metadata_.isEmpty()) { - result.metadata_ = new java.util.ArrayList(); + result.metadata_ = new java.util.ArrayList(); } result.metadata_.add(value); return this; } - public Builder addMetadata(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) { + public Builder addMetadata(akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) { if (result.metadata_.isEmpty()) { - result.metadata_ = new java.util.ArrayList(); + result.metadata_ = new java.util.ArrayList(); } result.metadata_.add(builderForValue.build()); return this; } public Builder addAllMetadata( - java.lang.Iterable values) { + java.lang.Iterable values) { if (result.metadata_.isEmpty()) { - result.metadata_ = new java.util.ArrayList(); + result.metadata_ = new java.util.ArrayList(); } super.addAll(values, result.metadata_); return this; @@ -5018,7 +5018,7 @@ public final class RemoteProtocol { static { defaultInstance = new RemoteReplyProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -5044,12 +5044,12 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_UuidProtocol_descriptor; + return 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; + return akka.remote.protocol.RemoteProtocol.internal_static_UuidProtocol_fieldAccessorTable; } // required uint64 high = 1; @@ -5105,41 +5105,41 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom( + public static 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( + public static 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) + public static 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( + public static 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) + public static 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( + public static 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) + public static akka.remote.protocol.RemoteProtocol.UuidProtocol parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -5148,7 +5148,7 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseDelimitedFrom( + public static akka.remote.protocol.RemoteProtocol.UuidProtocol parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -5159,12 +5159,12 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom( + public static 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( + public static akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -5174,25 +5174,25 @@ public final class RemoteProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol prototype) { + public static Builder newBuilder(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; + private akka.remote.protocol.RemoteProtocol.UuidProtocol result; - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder() + // Construct using 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(); + builder.result = new akka.remote.protocol.RemoteProtocol.UuidProtocol(); return builder; } - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol internalGetResult() { + protected akka.remote.protocol.RemoteProtocol.UuidProtocol internalGetResult() { return result; } @@ -5201,7 +5201,7 @@ public final class RemoteProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol(); + result = new akka.remote.protocol.RemoteProtocol.UuidProtocol(); return this; } @@ -5211,24 +5211,24 @@ public final class RemoteProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDescriptor(); + return 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 akka.remote.protocol.RemoteProtocol.UuidProtocol getDefaultInstanceForType() { + return akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol build() { + public akka.remote.protocol.RemoteProtocol.UuidProtocol build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol buildParsed() + private akka.remote.protocol.RemoteProtocol.UuidProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -5237,27 +5237,27 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol buildPartial() { + public 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; + 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); + if (other instanceof akka.remote.protocol.RemoteProtocol.UuidProtocol) { + return mergeFrom((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; + public Builder mergeFrom(akka.remote.protocol.RemoteProtocol.UuidProtocol other) { + if (other == akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) return this; if (other.hasHigh()) { setHigh(other.getHigh()); } @@ -5343,7 +5343,7 @@ public final class RemoteProtocol { static { defaultInstance = new UuidProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -5369,12 +5369,12 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MetadataEntryProtocol_descriptor; + return 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; + return akka.remote.protocol.RemoteProtocol.internal_static_MetadataEntryProtocol_fieldAccessorTable; } // required string key = 1; @@ -5430,41 +5430,41 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom( + public static 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( + public static 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) + public static 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( + public static 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) + public static 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( + public static 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) + public static akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -5473,7 +5473,7 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseDelimitedFrom( + public static akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -5484,12 +5484,12 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom( + public static 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( + public static akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -5499,25 +5499,25 @@ public final class RemoteProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol prototype) { + public static Builder newBuilder(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; + private akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol result; - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.newBuilder() + // Construct using 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(); + builder.result = new akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol(); return builder; } - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol internalGetResult() { + protected akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol internalGetResult() { return result; } @@ -5526,7 +5526,7 @@ public final class RemoteProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol(); + result = new akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol(); return this; } @@ -5536,24 +5536,24 @@ public final class RemoteProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.getDescriptor(); + return 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 akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getDefaultInstanceForType() { + return akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol build() { + public akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol buildParsed() + private akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -5562,27 +5562,27 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol buildPartial() { + public 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; + 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); + if (other instanceof akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol) { + return mergeFrom((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; + public Builder mergeFrom(akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol other) { + if (other == akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.getDefaultInstance()) return this; if (other.hasKey()) { setKey(other.getKey()); } @@ -5674,7 +5674,7 @@ public final class RemoteProtocol { static { defaultInstance = new MetadataEntryProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -5700,23 +5700,23 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_LifeCycleProtocol_descriptor; + return 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; + return 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_; + private akka.remote.protocol.RemoteProtocol.LifeCycleType lifeCycle_; public boolean hasLifeCycle() { return hasLifeCycle; } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle() { return lifeCycle_; } + public akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle() { return lifeCycle_; } private void initFields() { - lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT; + lifeCycle_ = akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT; } public final boolean isInitialized() { if (!hasLifeCycle) return false; @@ -5747,41 +5747,41 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom( + public static 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( + public static 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) + public static 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( + public static 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) + public static 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( + public static 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) + public static akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -5790,7 +5790,7 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseDelimitedFrom( + public static akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -5801,12 +5801,12 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom( + public static 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( + public static akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -5816,25 +5816,25 @@ public final class RemoteProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol prototype) { + public static Builder newBuilder(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; + private akka.remote.protocol.RemoteProtocol.LifeCycleProtocol result; - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder() + // Construct using 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(); + builder.result = new akka.remote.protocol.RemoteProtocol.LifeCycleProtocol(); return builder; } - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol internalGetResult() { + protected akka.remote.protocol.RemoteProtocol.LifeCycleProtocol internalGetResult() { return result; } @@ -5843,7 +5843,7 @@ public final class RemoteProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol(); + result = new akka.remote.protocol.RemoteProtocol.LifeCycleProtocol(); return this; } @@ -5853,24 +5853,24 @@ public final class RemoteProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDescriptor(); + return 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 akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getDefaultInstanceForType() { + return akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol build() { + public akka.remote.protocol.RemoteProtocol.LifeCycleProtocol build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol buildParsed() + private akka.remote.protocol.RemoteProtocol.LifeCycleProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -5879,27 +5879,27 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol buildPartial() { + public 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; + 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); + if (other instanceof akka.remote.protocol.RemoteProtocol.LifeCycleProtocol) { + return mergeFrom((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; + public Builder mergeFrom(akka.remote.protocol.RemoteProtocol.LifeCycleProtocol other) { + if (other == akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance()) return this; if (other.hasLifeCycle()) { setLifeCycle(other.getLifeCycle()); } @@ -5930,7 +5930,7 @@ public final class RemoteProtocol { } case 8: { int rawValue = input.readEnum(); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType value = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.valueOf(rawValue); + akka.remote.protocol.RemoteProtocol.LifeCycleType value = akka.remote.protocol.RemoteProtocol.LifeCycleType.valueOf(rawValue); if (value == null) { unknownFields.mergeVarintField(1, rawValue); } else { @@ -5947,10 +5947,10 @@ public final class RemoteProtocol { public boolean hasLifeCycle() { return result.hasLifeCycle(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle() { + public akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle() { return result.getLifeCycle(); } - public Builder setLifeCycle(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType value) { + public Builder setLifeCycle(akka.remote.protocol.RemoteProtocol.LifeCycleType value) { if (value == null) { throw new NullPointerException(); } @@ -5960,7 +5960,7 @@ public final class RemoteProtocol { } public Builder clearLifeCycle() { result.hasLifeCycle = false; - result.lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT; + result.lifeCycle_ = akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT; return this; } @@ -5969,7 +5969,7 @@ public final class RemoteProtocol { static { defaultInstance = new LifeCycleProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -5995,12 +5995,12 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_AddressProtocol_descriptor; + return 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; + return akka.remote.protocol.RemoteProtocol.internal_static_AddressProtocol_fieldAccessorTable; } // required string hostname = 1; @@ -6056,41 +6056,41 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom( + public static 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( + public static 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) + public static 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( + public static 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) + public static 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( + public static 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) + public static akka.remote.protocol.RemoteProtocol.AddressProtocol parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -6099,7 +6099,7 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseDelimitedFrom( + public static akka.remote.protocol.RemoteProtocol.AddressProtocol parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -6110,12 +6110,12 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom( + public static 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( + public static akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -6125,25 +6125,25 @@ public final class RemoteProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol prototype) { + public static Builder newBuilder(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; + private akka.remote.protocol.RemoteProtocol.AddressProtocol result; - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder() + // Construct using 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(); + builder.result = new akka.remote.protocol.RemoteProtocol.AddressProtocol(); return builder; } - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol internalGetResult() { + protected akka.remote.protocol.RemoteProtocol.AddressProtocol internalGetResult() { return result; } @@ -6152,7 +6152,7 @@ public final class RemoteProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol(); + result = new akka.remote.protocol.RemoteProtocol.AddressProtocol(); return this; } @@ -6162,24 +6162,24 @@ public final class RemoteProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDescriptor(); + return 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 akka.remote.protocol.RemoteProtocol.AddressProtocol getDefaultInstanceForType() { + return akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol build() { + public akka.remote.protocol.RemoteProtocol.AddressProtocol build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol buildParsed() + private akka.remote.protocol.RemoteProtocol.AddressProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -6188,27 +6188,27 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol buildPartial() { + public 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; + 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); + if (other instanceof akka.remote.protocol.RemoteProtocol.AddressProtocol) { + return mergeFrom((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; + public Builder mergeFrom(akka.remote.protocol.RemoteProtocol.AddressProtocol other) { + if (other == akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) return this; if (other.hasHostname()) { setHostname(other.getHostname()); } @@ -6297,7 +6297,7 @@ public final class RemoteProtocol { static { defaultInstance = new AddressProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -6323,12 +6323,12 @@ public final class RemoteProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ExceptionProtocol_descriptor; + return 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; + return akka.remote.protocol.RemoteProtocol.internal_static_ExceptionProtocol_fieldAccessorTable; } // required string classname = 1; @@ -6384,41 +6384,41 @@ public final class RemoteProtocol { return size; } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom( + public static 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( + public static 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) + public static 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( + public static 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) + public static 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( + public static 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) + public static akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -6427,7 +6427,7 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseDelimitedFrom( + public static akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -6438,12 +6438,12 @@ public final class RemoteProtocol { return null; } } - public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom( + public static 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( + public static akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -6453,25 +6453,25 @@ public final class RemoteProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol prototype) { + public static Builder newBuilder(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; + private akka.remote.protocol.RemoteProtocol.ExceptionProtocol result; - // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder() + // Construct using 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(); + builder.result = new akka.remote.protocol.RemoteProtocol.ExceptionProtocol(); return builder; } - protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol internalGetResult() { + protected akka.remote.protocol.RemoteProtocol.ExceptionProtocol internalGetResult() { return result; } @@ -6480,7 +6480,7 @@ public final class RemoteProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol(); + result = new akka.remote.protocol.RemoteProtocol.ExceptionProtocol(); return this; } @@ -6490,24 +6490,24 @@ public final class RemoteProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDescriptor(); + return 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 akka.remote.protocol.RemoteProtocol.ExceptionProtocol getDefaultInstanceForType() { + return akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol build() { + public akka.remote.protocol.RemoteProtocol.ExceptionProtocol build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol buildParsed() + private akka.remote.protocol.RemoteProtocol.ExceptionProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -6516,27 +6516,27 @@ public final class RemoteProtocol { return buildPartial(); } - public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol buildPartial() { + public 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; + 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); + if (other instanceof akka.remote.protocol.RemoteProtocol.ExceptionProtocol) { + return mergeFrom((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; + public Builder mergeFrom(akka.remote.protocol.RemoteProtocol.ExceptionProtocol other) { + if (other == akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance()) return this; if (other.hasClassname()) { setClassname(other.getClassname()); } @@ -6628,7 +6628,7 @@ public final class RemoteProtocol { static { defaultInstance = new ExceptionProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -6767,8 +6767,7 @@ public final class RemoteProtocol { "onSchemeType\022\010\n\004JAVA\020\001\022\013\n\007SBINARY\020\002\022\016\n\nS" + "CALA_JSON\020\003\022\r\n\tJAVA_JSON\020\004\022\014\n\010PROTOBUF\020\005" + "*-\n\rLifeCycleType\022\r\n\tPERMANENT\020\001\022\r\n\tTEMP" + - "ORARY\020\002B-\n)se.scalablesolutions.akka.rem" + - "ote.protocolH\001" + "ORARY\020\002B\030\n\024akka.remote.protocolH\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -6781,112 +6780,112 @@ public final class RemoteProtocol { com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RemoteActorRefProtocol_descriptor, new java.lang.String[] { "ClassOrServiceName", "ActorClassname", "HomeAddress", "Timeout", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder.class); + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.class, + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder.class); internal_static_RemoteTypedActorRefProtocol_descriptor = getDescriptor().getMessageTypes().get(1); internal_static_RemoteTypedActorRefProtocol_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RemoteTypedActorRefProtocol_descriptor, new java.lang.String[] { "ActorRef", "InterfaceName", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.Builder.class); + akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.class, + akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.Builder.class); internal_static_SerializedActorRefProtocol_descriptor = getDescriptor().getMessageTypes().get(2); 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); + akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.class, + akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder.class); internal_static_SerializedTypedActorRefProtocol_descriptor = getDescriptor().getMessageTypes().get(3); internal_static_SerializedTypedActorRefProtocol_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SerializedTypedActorRefProtocol_descriptor, new java.lang.String[] { "ActorRef", "InterfaceName", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.Builder.class); + akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.class, + akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.Builder.class); internal_static_MessageProtocol_descriptor = getDescriptor().getMessageTypes().get(4); 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); + akka.remote.protocol.RemoteProtocol.MessageProtocol.class, + akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder.class); internal_static_ActorInfoProtocol_descriptor = getDescriptor().getMessageTypes().get(5); internal_static_ActorInfoProtocol_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ActorInfoProtocol_descriptor, 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); + akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.class, + akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder.class); internal_static_TypedActorInfoProtocol_descriptor = getDescriptor().getMessageTypes().get(6); 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); + akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.class, + akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.Builder.class); internal_static_RemoteRequestProtocol_descriptor = getDescriptor().getMessageTypes().get(7); internal_static_RemoteRequestProtocol_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RemoteRequestProtocol_descriptor, new java.lang.String[] { "Uuid", "Message", "ActorInfo", "IsOneWay", "SupervisorUuid", "Sender", "Metadata", "Cookie", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder.class); + akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.class, + akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder.class); internal_static_RemoteReplyProtocol_descriptor = getDescriptor().getMessageTypes().get(8); internal_static_RemoteReplyProtocol_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RemoteReplyProtocol_descriptor, new java.lang.String[] { "Uuid", "Message", "Exception", "SupervisorUuid", "IsActor", "IsSuccessful", "Metadata", "Cookie", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.Builder.class); + akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.class, + akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.Builder.class); internal_static_UuidProtocol_descriptor = getDescriptor().getMessageTypes().get(9); 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); + akka.remote.protocol.RemoteProtocol.UuidProtocol.class, + akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder.class); internal_static_MetadataEntryProtocol_descriptor = getDescriptor().getMessageTypes().get(10); 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); + akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.class, + akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder.class); internal_static_LifeCycleProtocol_descriptor = getDescriptor().getMessageTypes().get(11); internal_static_LifeCycleProtocol_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_LifeCycleProtocol_descriptor, new java.lang.String[] { "LifeCycle", }, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.class, - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder.class); + akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.class, + akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder.class); internal_static_AddressProtocol_descriptor = getDescriptor().getMessageTypes().get(12); 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); + akka.remote.protocol.RemoteProtocol.AddressProtocol.class, + akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder.class); internal_static_ExceptionProtocol_descriptor = getDescriptor().getMessageTypes().get(13); 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); + akka.remote.protocol.RemoteProtocol.ExceptionProtocol.class, + akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder.class); return null; } }; diff --git a/akka-remote/src/main/protocol/RemoteProtocol.proto b/akka-remote/src/main/protocol/RemoteProtocol.proto index 46e38fd158..e7ee0129d4 100644 --- a/akka-remote/src/main/protocol/RemoteProtocol.proto +++ b/akka-remote/src/main/protocol/RemoteProtocol.proto @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -option java_package = "se.scalablesolutions.akka.remote.protocol"; +option java_package = "akka.remote.protocol"; option optimize_for = SPEED; /****************************************** diff --git a/akka-remote/src/main/scala/remote/BootableRemoteActorService.scala b/akka-remote/src/main/scala/remote/BootableRemoteActorService.scala index 643dbd2f33..951b65ea7f 100644 --- a/akka-remote/src/main/scala/remote/BootableRemoteActorService.scala +++ b/akka-remote/src/main/scala/remote/BootableRemoteActorService.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.remote +package akka.remote -import se.scalablesolutions.akka.actor.BootableActorLoaderService -import se.scalablesolutions.akka.util.{Bootable, Logging} -import se.scalablesolutions.akka.config.Config.config +import akka.actor.BootableActorLoaderService +import akka.util.{Bootable, Logging} +import akka.config.Config.config /** * This bundle/service is responsible for booting up and shutting down the remote actors facility diff --git a/akka-remote/src/main/scala/remote/Cluster.scala b/akka-remote/src/main/scala/remote/Cluster.scala index f7ef779289..9fe5c46981 100644 --- a/akka-remote/src/main/scala/remote/Cluster.scala +++ b/akka-remote/src/main/scala/remote/Cluster.scala @@ -2,16 +2,16 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.remote +package akka.remote -import se.scalablesolutions.akka.config.Config.config -import se.scalablesolutions.akka.config.Supervision._ -import se.scalablesolutions.akka.serialization.Serializer -import se.scalablesolutions.akka.actor.{Supervisor, SupervisorFactory, Actor, ActorRef, ActorRegistry} -import se.scalablesolutions.akka.util.Logging +import akka.config.Config.config +import akka.config.Supervision._ +import akka.serialization.Serializer +import akka.actor.{Supervisor, SupervisorFactory, Actor, ActorRef, ActorRegistry} +import akka.util.Logging import scala.collection.immutable.{Map, HashMap} -import se.scalablesolutions.akka.config.Supervision.{Permanent} -import se.scalablesolutions.akka.config.{RemoteAddress} +import akka.config.Supervision.{Permanent} +import akka.config.{RemoteAddress} /** * Interface for interacting with the Cluster Membership API. diff --git a/akka-remote/src/main/scala/remote/JGroupsClusterActor.scala b/akka-remote/src/main/scala/remote/JGroupsClusterActor.scala index 07cbf4d65b..900c5e9785 100644 --- a/akka-remote/src/main/scala/remote/JGroupsClusterActor.scala +++ b/akka-remote/src/main/scala/remote/JGroupsClusterActor.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.remote +package akka.remote import org.jgroups.{JChannel, View => JG_VIEW, Address, Message => JG_MSG, ExtendedMembershipListener, Receiver} import org.jgroups.util.Util @@ -10,7 +10,7 @@ import org.jgroups.util.Util */ class JGroupsClusterActor extends BasicClusterActor { import scala.collection.JavaConversions._ - import se.scalablesolutions.akka.remote.ClusterActor._ + import akka.remote.ClusterActor._ type ADDR_T = Address diff --git a/akka-remote/src/main/scala/remote/MessageSerializer.scala b/akka-remote/src/main/scala/remote/MessageSerializer.scala index 7cda9e4b4c..b68bb0bca6 100644 --- a/akka-remote/src/main/scala/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/remote/MessageSerializer.scala @@ -2,11 +2,11 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.remote +package akka.remote -import se.scalablesolutions.akka.serialization.{Serializer, Serializable} -import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._ -import se.scalablesolutions.akka.util._ +import akka.serialization.{Serializer, Serializable} +import akka.remote.protocol.RemoteProtocol._ +import akka.util._ import com.google.protobuf.{Message, ByteString} diff --git a/akka-remote/src/main/scala/remote/RemoteClient.scala b/akka-remote/src/main/scala/remote/RemoteClient.scala index 3ecb64f77c..e42a38900c 100644 --- a/akka-remote/src/main/scala/remote/RemoteClient.scala +++ b/akka-remote/src/main/scala/remote/RemoteClient.scala @@ -2,16 +2,15 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.remote +package akka.remote -import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.{ ActorType => ActorTypeProtocol, _ } -import se.scalablesolutions.akka.actor._ -//import se.scalablesolutions.akka.actor.Uuid.{newUuid, uuidFrom} -import se.scalablesolutions.akka.dispatch.{ DefaultCompletableFuture, CompletableFuture } -import se.scalablesolutions.akka.util._ -import se.scalablesolutions.akka.config.Config._ -import se.scalablesolutions.akka.serialization.RemoteActorSerialization._ -import se.scalablesolutions.akka.AkkaException +import akka.remote.protocol.RemoteProtocol.{ActorType => ActorTypeProtocol, _} +import akka.actor.{Exit, Actor, ActorRef, ActorType, RemoteActorRef, IllegalActorStateException} +import akka.dispatch.{DefaultCompletableFuture, CompletableFuture} +import akka.actor.{Uuid,newUuid,uuidFrom} +import akka.config.Config._ +import akka.serialization.RemoteActorSerialization._ +import akka.AkkaException import Actor._ import org.jboss.netty.channel._ @@ -32,6 +31,10 @@ import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean } import scala.collection.mutable.{ HashSet, HashMap } import scala.reflect.BeanProperty +import akka.actor._ +import akka.util._ + + /** * Life-cycle events for RemoteClient. */ diff --git a/akka-remote/src/main/scala/remote/RemoteServer.scala b/akka-remote/src/main/scala/remote/RemoteServer.scala index 3d42ed0ad9..9bc85e2608 100644 --- a/akka-remote/src/main/scala/remote/RemoteServer.scala +++ b/akka-remote/src/main/scala/remote/RemoteServer.scala @@ -2,24 +2,23 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.remote +package akka.remote import java.lang.reflect.InvocationTargetException import java.net.InetSocketAddress import java.util.concurrent.{ConcurrentHashMap, Executors} import java.util.{Map => JMap} -import se.scalablesolutions.akka.actor.{ +import akka.actor.{ Actor, TypedActor, ActorRef, IllegalActorStateException, RemoteActorSystemMessage, uuidFrom, Uuid, ActorRegistry, LifeCycleMessage} -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.config.ConfigurationException -import se.scalablesolutions.akka.dispatch.{DefaultCompletableFuture, CompletableFuture} -import se.scalablesolutions.akka.serialization.RemoteActorSerialization -import se.scalablesolutions.akka.serialization.RemoteActorSerialization._ +import akka.actor.Actor._ +import akka.util._ +import akka.remote.protocol.RemoteProtocol._ +import akka.remote.protocol.RemoteProtocol.ActorType._ +import akka.config.Config._ +import akka.dispatch.{DefaultCompletableFuture, CompletableFuture} +import akka.serialization.RemoteActorSerialization +import akka.serialization.RemoteActorSerialization._ import org.jboss.netty.bootstrap.ServerBootstrap import org.jboss.netty.channel._ @@ -32,6 +31,7 @@ import org.jboss.netty.handler.ssl.SslHandler import scala.collection.mutable.Map import scala.reflect.BeanProperty +import akka.config.ConfigurationException /** * Use this object if you need a single remote server on a specific node. diff --git a/akka-remote/src/main/scala/serialization/Binary.scala b/akka-remote/src/main/scala/serialization/Binary.scala index af0505a574..1f93b7f093 100644 --- a/akka-remote/src/main/scala/serialization/Binary.scala +++ b/akka-remote/src/main/scala/serialization/Binary.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.serialization +package akka.serialization import sbinary._ import sbinary.Operations._ diff --git a/akka-remote/src/main/scala/serialization/Compression.scala b/akka-remote/src/main/scala/serialization/Compression.scala index bbb8d95421..5f8f2e5e94 100644 --- a/akka-remote/src/main/scala/serialization/Compression.scala +++ b/akka-remote/src/main/scala/serialization/Compression.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.serialization +package akka.serialization /** * @author Jonas Bonér diff --git a/akka-remote/src/main/scala/serialization/Serializable.scala b/akka-remote/src/main/scala/serialization/Serializable.scala index a939964420..79c87f6300 100644 --- a/akka-remote/src/main/scala/serialization/Serializable.scala +++ b/akka-remote/src/main/scala/serialization/Serializable.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.serialization +package akka.serialization import org.codehaus.jackson.map.ObjectMapper diff --git a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala index 9cbf2446aa..a5873f4dc3 100644 --- a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala +++ b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala @@ -2,18 +2,19 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.serialization +package akka.serialization + +import akka.stm.global._ +import akka.stm.TransactionManagement._ +import akka.stm.TransactionManagement +import akka.dispatch.MessageInvocation +import akka.remote.{RemoteServer, RemoteClient, MessageSerializer} +import akka.remote.protocol.RemoteProtocol.{ActorType => ActorTypeProtocol, _} -import se.scalablesolutions.akka.stm.global._ -import se.scalablesolutions.akka.stm.TransactionManagement._ -import se.scalablesolutions.akka.stm.TransactionManagement -import se.scalablesolutions.akka.dispatch.MessageInvocation -import se.scalablesolutions.akka.remote.{RemoteServer, RemoteClient, MessageSerializer} -import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.{ActorType => ActorTypeProtocol, _} import ActorTypeProtocol._ -import se.scalablesolutions.akka.config.Supervision._ -import se.scalablesolutions.akka.actor.{uuidFrom,newUuid} -import se.scalablesolutions.akka.actor._ +import akka.config.Supervision._ +import akka.actor.{uuidFrom,newUuid} +import akka.actor._ import scala.collection.immutable.Stack diff --git a/akka-remote/src/main/scala/serialization/Serializer.scala b/akka-remote/src/main/scala/serialization/Serializer.scala index 9c8941bca7..e30e615322 100644 --- a/akka-remote/src/main/scala/serialization/Serializer.scala +++ b/akka-remote/src/main/scala/serialization/Serializer.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.serialization +package akka.serialization import java.io.{ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream} diff --git a/akka-remote/src/main/scala/serialization/package.scala b/akka-remote/src/main/scala/serialization/package.scala index 1a3c83341f..f75560403a 100644 --- a/akka-remote/src/main/scala/serialization/package.scala +++ b/akka-remote/src/main/scala/serialization/package.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka +package akka package object serialization { type JsValue = _root_.dispatch.json.JsValue diff --git a/akka-remote/src/test/java/se/scalablesolutions/akka/actor/ProtobufProtocol.java b/akka-remote/src/test/java/akka/actor/ProtobufProtocol.java similarity index 71% rename from akka-remote/src/test/java/se/scalablesolutions/akka/actor/ProtobufProtocol.java rename to akka-remote/src/test/java/akka/actor/ProtobufProtocol.java index 183d2025d0..49d2e028c5 100644 --- a/akka-remote/src/test/java/se/scalablesolutions/akka/actor/ProtobufProtocol.java +++ b/akka-remote/src/test/java/akka/actor/ProtobufProtocol.java @@ -1,7 +1,7 @@ // Generated by the protocol buffer compiler. DO NOT EDIT! // source: ProtobufProtocol.proto -package se.scalablesolutions.akka.actor; +package akka.actor; public final class ProtobufProtocol { private ProtobufProtocol() {} @@ -27,12 +27,12 @@ public final class ProtobufProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.actor.ProtobufProtocol.internal_static_se_scalablesolutions_akka_actor_ProtobufPOJO_descriptor; + return akka.actor.ProtobufProtocol.internal_static_akka_actor_ProtobufPOJO_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return se.scalablesolutions.akka.actor.ProtobufProtocol.internal_static_se_scalablesolutions_akka_actor_ProtobufPOJO_fieldAccessorTable; + return akka.actor.ProtobufProtocol.internal_static_akka_actor_ProtobufPOJO_fieldAccessorTable; } // required uint64 id = 1; @@ -103,41 +103,41 @@ public final class ProtobufProtocol { return size; } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO parseFrom( + public static akka.actor.ProtobufProtocol.ProtobufPOJO parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO parseFrom( + public static akka.actor.ProtobufProtocol.ProtobufPOJO 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.actor.ProtobufProtocol.ProtobufPOJO parseFrom(byte[] data) + public static akka.actor.ProtobufProtocol.ProtobufPOJO parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO parseFrom( + public static akka.actor.ProtobufProtocol.ProtobufPOJO parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO parseFrom(java.io.InputStream input) + public static akka.actor.ProtobufProtocol.ProtobufPOJO parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO parseFrom( + public static akka.actor.ProtobufProtocol.ProtobufPOJO 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.actor.ProtobufProtocol.ProtobufPOJO parseDelimitedFrom(java.io.InputStream input) + public static akka.actor.ProtobufProtocol.ProtobufPOJO parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -146,7 +146,7 @@ public final class ProtobufProtocol { return null; } } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO parseDelimitedFrom( + public static akka.actor.ProtobufProtocol.ProtobufPOJO parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -157,12 +157,12 @@ public final class ProtobufProtocol { return null; } } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO parseFrom( + public static akka.actor.ProtobufProtocol.ProtobufPOJO parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO parseFrom( + public static akka.actor.ProtobufProtocol.ProtobufPOJO parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -172,25 +172,25 @@ public final class ProtobufProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO prototype) { + public static Builder newBuilder(akka.actor.ProtobufProtocol.ProtobufPOJO 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.actor.ProtobufProtocol.ProtobufPOJO result; + private akka.actor.ProtobufProtocol.ProtobufPOJO result; - // Construct using se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO.newBuilder() + // Construct using akka.actor.ProtobufProtocol.ProtobufPOJO.newBuilder() private Builder() {} private static Builder create() { Builder builder = new Builder(); - builder.result = new se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO(); + builder.result = new akka.actor.ProtobufProtocol.ProtobufPOJO(); return builder; } - protected se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO internalGetResult() { + protected akka.actor.ProtobufProtocol.ProtobufPOJO internalGetResult() { return result; } @@ -199,7 +199,7 @@ public final class ProtobufProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO(); + result = new akka.actor.ProtobufProtocol.ProtobufPOJO(); return this; } @@ -209,24 +209,24 @@ public final class ProtobufProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO.getDescriptor(); + return akka.actor.ProtobufProtocol.ProtobufPOJO.getDescriptor(); } - public se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO getDefaultInstanceForType() { - return se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO.getDefaultInstance(); + public akka.actor.ProtobufProtocol.ProtobufPOJO getDefaultInstanceForType() { + return akka.actor.ProtobufProtocol.ProtobufPOJO.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO build() { + public akka.actor.ProtobufProtocol.ProtobufPOJO build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO buildParsed() + private akka.actor.ProtobufProtocol.ProtobufPOJO buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -235,27 +235,27 @@ public final class ProtobufProtocol { return buildPartial(); } - public se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO buildPartial() { + public akka.actor.ProtobufProtocol.ProtobufPOJO buildPartial() { if (result == null) { throw new IllegalStateException( "build() has already been called on this Builder."); } - se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO returnMe = result; + akka.actor.ProtobufProtocol.ProtobufPOJO returnMe = result; result = null; return returnMe; } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO) { - return mergeFrom((se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO)other); + if (other instanceof akka.actor.ProtobufProtocol.ProtobufPOJO) { + return mergeFrom((akka.actor.ProtobufProtocol.ProtobufPOJO)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO other) { - if (other == se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO.getDefaultInstance()) return this; + public Builder mergeFrom(akka.actor.ProtobufProtocol.ProtobufPOJO other) { + if (other == akka.actor.ProtobufProtocol.ProtobufPOJO.getDefaultInstance()) return this; if (other.hasId()) { setId(other.getId()); } @@ -364,16 +364,16 @@ public final class ProtobufProtocol { return this; } - // @@protoc_insertion_point(builder_scope:se.scalablesolutions.akka.actor.ProtobufPOJO) + // @@protoc_insertion_point(builder_scope:akka.actor.ProtobufPOJO) } static { defaultInstance = new ProtobufPOJO(true); - se.scalablesolutions.akka.actor.ProtobufProtocol.internalForceInit(); + akka.actor.ProtobufProtocol.internalForceInit(); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:se.scalablesolutions.akka.actor.ProtobufPOJO) + // @@protoc_insertion_point(class_scope:akka.actor.ProtobufPOJO) } public static final class Counter extends @@ -395,12 +395,12 @@ public final class ProtobufProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.actor.ProtobufProtocol.internal_static_se_scalablesolutions_akka_actor_Counter_descriptor; + return akka.actor.ProtobufProtocol.internal_static_akka_actor_Counter_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return se.scalablesolutions.akka.actor.ProtobufProtocol.internal_static_se_scalablesolutions_akka_actor_Counter_fieldAccessorTable; + return akka.actor.ProtobufProtocol.internal_static_akka_actor_Counter_fieldAccessorTable; } // required uint32 count = 1; @@ -441,41 +441,41 @@ public final class ProtobufProtocol { return size; } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.Counter parseFrom( + public static akka.actor.ProtobufProtocol.Counter parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.Counter parseFrom( + public static akka.actor.ProtobufProtocol.Counter 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.actor.ProtobufProtocol.Counter parseFrom(byte[] data) + public static akka.actor.ProtobufProtocol.Counter parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.Counter parseFrom( + public static akka.actor.ProtobufProtocol.Counter parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.Counter parseFrom(java.io.InputStream input) + public static akka.actor.ProtobufProtocol.Counter parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.Counter parseFrom( + public static akka.actor.ProtobufProtocol.Counter 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.actor.ProtobufProtocol.Counter parseDelimitedFrom(java.io.InputStream input) + public static akka.actor.ProtobufProtocol.Counter parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -484,7 +484,7 @@ public final class ProtobufProtocol { return null; } } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.Counter parseDelimitedFrom( + public static akka.actor.ProtobufProtocol.Counter parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -495,12 +495,12 @@ public final class ProtobufProtocol { return null; } } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.Counter parseFrom( + public static akka.actor.ProtobufProtocol.Counter parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.Counter parseFrom( + public static akka.actor.ProtobufProtocol.Counter parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -510,25 +510,25 @@ public final class ProtobufProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.actor.ProtobufProtocol.Counter prototype) { + public static Builder newBuilder(akka.actor.ProtobufProtocol.Counter 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.actor.ProtobufProtocol.Counter result; + private akka.actor.ProtobufProtocol.Counter result; - // Construct using se.scalablesolutions.akka.actor.ProtobufProtocol.Counter.newBuilder() + // Construct using akka.actor.ProtobufProtocol.Counter.newBuilder() private Builder() {} private static Builder create() { Builder builder = new Builder(); - builder.result = new se.scalablesolutions.akka.actor.ProtobufProtocol.Counter(); + builder.result = new akka.actor.ProtobufProtocol.Counter(); return builder; } - protected se.scalablesolutions.akka.actor.ProtobufProtocol.Counter internalGetResult() { + protected akka.actor.ProtobufProtocol.Counter internalGetResult() { return result; } @@ -537,7 +537,7 @@ public final class ProtobufProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.actor.ProtobufProtocol.Counter(); + result = new akka.actor.ProtobufProtocol.Counter(); return this; } @@ -547,24 +547,24 @@ public final class ProtobufProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.actor.ProtobufProtocol.Counter.getDescriptor(); + return akka.actor.ProtobufProtocol.Counter.getDescriptor(); } - public se.scalablesolutions.akka.actor.ProtobufProtocol.Counter getDefaultInstanceForType() { - return se.scalablesolutions.akka.actor.ProtobufProtocol.Counter.getDefaultInstance(); + public akka.actor.ProtobufProtocol.Counter getDefaultInstanceForType() { + return akka.actor.ProtobufProtocol.Counter.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.actor.ProtobufProtocol.Counter build() { + public akka.actor.ProtobufProtocol.Counter build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.actor.ProtobufProtocol.Counter buildParsed() + private akka.actor.ProtobufProtocol.Counter buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -573,27 +573,27 @@ public final class ProtobufProtocol { return buildPartial(); } - public se.scalablesolutions.akka.actor.ProtobufProtocol.Counter buildPartial() { + public akka.actor.ProtobufProtocol.Counter buildPartial() { if (result == null) { throw new IllegalStateException( "build() has already been called on this Builder."); } - se.scalablesolutions.akka.actor.ProtobufProtocol.Counter returnMe = result; + akka.actor.ProtobufProtocol.Counter returnMe = result; result = null; return returnMe; } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.actor.ProtobufProtocol.Counter) { - return mergeFrom((se.scalablesolutions.akka.actor.ProtobufProtocol.Counter)other); + if (other instanceof akka.actor.ProtobufProtocol.Counter) { + return mergeFrom((akka.actor.ProtobufProtocol.Counter)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(se.scalablesolutions.akka.actor.ProtobufProtocol.Counter other) { - if (other == se.scalablesolutions.akka.actor.ProtobufProtocol.Counter.getDefaultInstance()) return this; + public Builder mergeFrom(akka.actor.ProtobufProtocol.Counter other) { + if (other == akka.actor.ProtobufProtocol.Counter.getDefaultInstance()) return this; if (other.hasCount()) { setCount(other.getCount()); } @@ -649,16 +649,16 @@ public final class ProtobufProtocol { return this; } - // @@protoc_insertion_point(builder_scope:se.scalablesolutions.akka.actor.Counter) + // @@protoc_insertion_point(builder_scope:akka.actor.Counter) } static { defaultInstance = new Counter(true); - se.scalablesolutions.akka.actor.ProtobufProtocol.internalForceInit(); + akka.actor.ProtobufProtocol.internalForceInit(); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:se.scalablesolutions.akka.actor.Counter) + // @@protoc_insertion_point(class_scope:akka.actor.Counter) } public static final class DualCounter extends @@ -680,12 +680,12 @@ public final class ProtobufProtocol { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return se.scalablesolutions.akka.actor.ProtobufProtocol.internal_static_se_scalablesolutions_akka_actor_DualCounter_descriptor; + return akka.actor.ProtobufProtocol.internal_static_akka_actor_DualCounter_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return se.scalablesolutions.akka.actor.ProtobufProtocol.internal_static_se_scalablesolutions_akka_actor_DualCounter_fieldAccessorTable; + return akka.actor.ProtobufProtocol.internal_static_akka_actor_DualCounter_fieldAccessorTable; } // required uint32 count1 = 1; @@ -741,41 +741,41 @@ public final class ProtobufProtocol { return size; } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter parseFrom( + public static akka.actor.ProtobufProtocol.DualCounter parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter parseFrom( + public static akka.actor.ProtobufProtocol.DualCounter 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.actor.ProtobufProtocol.DualCounter parseFrom(byte[] data) + public static akka.actor.ProtobufProtocol.DualCounter parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter parseFrom( + public static akka.actor.ProtobufProtocol.DualCounter parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter parseFrom(java.io.InputStream input) + public static akka.actor.ProtobufProtocol.DualCounter parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter parseFrom( + public static akka.actor.ProtobufProtocol.DualCounter 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.actor.ProtobufProtocol.DualCounter parseDelimitedFrom(java.io.InputStream input) + public static akka.actor.ProtobufProtocol.DualCounter parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -784,7 +784,7 @@ public final class ProtobufProtocol { return null; } } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter parseDelimitedFrom( + public static akka.actor.ProtobufProtocol.DualCounter parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -795,12 +795,12 @@ public final class ProtobufProtocol { return null; } } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter parseFrom( + public static akka.actor.ProtobufProtocol.DualCounter parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter parseFrom( + public static akka.actor.ProtobufProtocol.DualCounter parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -810,25 +810,25 @@ public final class ProtobufProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter prototype) { + public static Builder newBuilder(akka.actor.ProtobufProtocol.DualCounter 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.actor.ProtobufProtocol.DualCounter result; + private akka.actor.ProtobufProtocol.DualCounter result; - // Construct using se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter.newBuilder() + // Construct using akka.actor.ProtobufProtocol.DualCounter.newBuilder() private Builder() {} private static Builder create() { Builder builder = new Builder(); - builder.result = new se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter(); + builder.result = new akka.actor.ProtobufProtocol.DualCounter(); return builder; } - protected se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter internalGetResult() { + protected akka.actor.ProtobufProtocol.DualCounter internalGetResult() { return result; } @@ -837,7 +837,7 @@ public final class ProtobufProtocol { throw new IllegalStateException( "Cannot call clear() after build()."); } - result = new se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter(); + result = new akka.actor.ProtobufProtocol.DualCounter(); return this; } @@ -847,24 +847,24 @@ public final class ProtobufProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter.getDescriptor(); + return akka.actor.ProtobufProtocol.DualCounter.getDescriptor(); } - public se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter getDefaultInstanceForType() { - return se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter.getDefaultInstance(); + public akka.actor.ProtobufProtocol.DualCounter getDefaultInstanceForType() { + return akka.actor.ProtobufProtocol.DualCounter.getDefaultInstance(); } public boolean isInitialized() { return result.isInitialized(); } - public se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter build() { + public akka.actor.ProtobufProtocol.DualCounter build() { if (result != null && !isInitialized()) { throw newUninitializedMessageException(result); } return buildPartial(); } - private se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter buildParsed() + private akka.actor.ProtobufProtocol.DualCounter buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { if (!isInitialized()) { throw newUninitializedMessageException( @@ -873,27 +873,27 @@ public final class ProtobufProtocol { return buildPartial(); } - public se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter buildPartial() { + public akka.actor.ProtobufProtocol.DualCounter buildPartial() { if (result == null) { throw new IllegalStateException( "build() has already been called on this Builder."); } - se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter returnMe = result; + akka.actor.ProtobufProtocol.DualCounter returnMe = result; result = null; return returnMe; } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter) { - return mergeFrom((se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter)other); + if (other instanceof akka.actor.ProtobufProtocol.DualCounter) { + return mergeFrom((akka.actor.ProtobufProtocol.DualCounter)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter other) { - if (other == se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter.getDefaultInstance()) return this; + public Builder mergeFrom(akka.actor.ProtobufProtocol.DualCounter other) { + if (other == akka.actor.ProtobufProtocol.DualCounter.getDefaultInstance()) return this; if (other.hasCount1()) { setCount1(other.getCount1()); } @@ -974,33 +974,33 @@ public final class ProtobufProtocol { return this; } - // @@protoc_insertion_point(builder_scope:se.scalablesolutions.akka.actor.DualCounter) + // @@protoc_insertion_point(builder_scope:akka.actor.DualCounter) } static { defaultInstance = new DualCounter(true); - se.scalablesolutions.akka.actor.ProtobufProtocol.internalForceInit(); + akka.actor.ProtobufProtocol.internalForceInit(); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:se.scalablesolutions.akka.actor.DualCounter) + // @@protoc_insertion_point(class_scope:akka.actor.DualCounter) } private static com.google.protobuf.Descriptors.Descriptor - internal_static_se_scalablesolutions_akka_actor_ProtobufPOJO_descriptor; + internal_static_akka_actor_ProtobufPOJO_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_se_scalablesolutions_akka_actor_ProtobufPOJO_fieldAccessorTable; + internal_static_akka_actor_ProtobufPOJO_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_se_scalablesolutions_akka_actor_Counter_descriptor; + internal_static_akka_actor_Counter_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_se_scalablesolutions_akka_actor_Counter_fieldAccessorTable; + internal_static_akka_actor_Counter_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_se_scalablesolutions_akka_actor_DualCounter_descriptor; + internal_static_akka_actor_DualCounter_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_se_scalablesolutions_akka_actor_DualCounter_fieldAccessorTable; + internal_static_akka_actor_DualCounter_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -1010,41 +1010,41 @@ public final class ProtobufProtocol { descriptor; static { java.lang.String[] descriptorData = { - "\n\026ProtobufProtocol.proto\022\037se.scalablesol" + - "utions.akka.actor\"8\n\014ProtobufPOJO\022\n\n\002id\030" + - "\001 \002(\004\022\014\n\004name\030\002 \002(\t\022\016\n\006status\030\003 \002(\010\"\030\n\007C" + - "ounter\022\r\n\005count\030\001 \002(\r\"-\n\013DualCounter\022\016\n\006" + - "count1\030\001 \002(\r\022\016\n\006count2\030\002 \002(\r" + "\n\026ProtobufProtocol.proto\022\nakka.actor\"8\n\014" + + "ProtobufPOJO\022\n\n\002id\030\001 \002(\004\022\014\n\004name\030\002 \002(\t\022\016" + + "\n\006status\030\003 \002(\010\"\030\n\007Counter\022\r\n\005count\030\001 \002(\r" + + "\"-\n\013DualCounter\022\016\n\006count1\030\001 \002(\r\022\016\n\006count" + + "2\030\002 \002(\r" }; 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_se_scalablesolutions_akka_actor_ProtobufPOJO_descriptor = + internal_static_akka_actor_ProtobufPOJO_descriptor = getDescriptor().getMessageTypes().get(0); - internal_static_se_scalablesolutions_akka_actor_ProtobufPOJO_fieldAccessorTable = new + internal_static_akka_actor_ProtobufPOJO_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_se_scalablesolutions_akka_actor_ProtobufPOJO_descriptor, + internal_static_akka_actor_ProtobufPOJO_descriptor, new java.lang.String[] { "Id", "Name", "Status", }, - se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO.class, - se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO.Builder.class); - internal_static_se_scalablesolutions_akka_actor_Counter_descriptor = + akka.actor.ProtobufProtocol.ProtobufPOJO.class, + akka.actor.ProtobufProtocol.ProtobufPOJO.Builder.class); + internal_static_akka_actor_Counter_descriptor = getDescriptor().getMessageTypes().get(1); - internal_static_se_scalablesolutions_akka_actor_Counter_fieldAccessorTable = new + internal_static_akka_actor_Counter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_se_scalablesolutions_akka_actor_Counter_descriptor, + internal_static_akka_actor_Counter_descriptor, new java.lang.String[] { "Count", }, - se.scalablesolutions.akka.actor.ProtobufProtocol.Counter.class, - se.scalablesolutions.akka.actor.ProtobufProtocol.Counter.Builder.class); - internal_static_se_scalablesolutions_akka_actor_DualCounter_descriptor = + akka.actor.ProtobufProtocol.Counter.class, + akka.actor.ProtobufProtocol.Counter.Builder.class); + internal_static_akka_actor_DualCounter_descriptor = getDescriptor().getMessageTypes().get(2); - internal_static_se_scalablesolutions_akka_actor_DualCounter_fieldAccessorTable = new + internal_static_akka_actor_DualCounter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_se_scalablesolutions_akka_actor_DualCounter_descriptor, + internal_static_akka_actor_DualCounter_descriptor, new java.lang.String[] { "Count1", "Count2", }, - se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter.class, - se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter.Builder.class); + akka.actor.ProtobufProtocol.DualCounter.class, + akka.actor.ProtobufProtocol.DualCounter.Builder.class); return null; } }; diff --git a/akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOne.java b/akka-remote/src/test/java/akka/actor/RemoteTypedActorOne.java similarity index 77% rename from akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOne.java rename to akka-remote/src/test/java/akka/actor/RemoteTypedActorOne.java index dd03a45d12..83303d031a 100644 --- a/akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOne.java +++ b/akka-remote/src/test/java/akka/actor/RemoteTypedActorOne.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.actor; +package akka.actor; public interface RemoteTypedActorOne { public String requestReply(String s) throws Exception; diff --git a/akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOneImpl.java b/akka-remote/src/test/java/akka/actor/RemoteTypedActorOneImpl.java similarity index 89% rename from akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOneImpl.java rename to akka-remote/src/test/java/akka/actor/RemoteTypedActorOneImpl.java index 715e5366a4..41d4bbe395 100644 --- a/akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOneImpl.java +++ b/akka-remote/src/test/java/akka/actor/RemoteTypedActorOneImpl.java @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.actor.remote; +package akka.actor.remote; -import se.scalablesolutions.akka.actor.*; +import akka.actor.*; import java.util.concurrent.CountDownLatch; diff --git a/akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwo.java b/akka-remote/src/test/java/akka/actor/RemoteTypedActorTwo.java similarity index 77% rename from akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwo.java rename to akka-remote/src/test/java/akka/actor/RemoteTypedActorTwo.java index 5fd289b8c2..25da478284 100644 --- a/akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwo.java +++ b/akka-remote/src/test/java/akka/actor/RemoteTypedActorTwo.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.actor; +package akka.actor; public interface RemoteTypedActorTwo { public String requestReply(String s) throws Exception; diff --git a/akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwoImpl.java b/akka-remote/src/test/java/akka/actor/RemoteTypedActorTwoImpl.java similarity index 89% rename from akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwoImpl.java rename to akka-remote/src/test/java/akka/actor/RemoteTypedActorTwoImpl.java index a5882fd4e6..d36deb8011 100644 --- a/akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwoImpl.java +++ b/akka-remote/src/test/java/akka/actor/RemoteTypedActorTwoImpl.java @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.actor.remote; +package akka.actor.remote; -import se.scalablesolutions.akka.actor.*; +import akka.actor.*; import java.util.concurrent.CountDownLatch; diff --git a/akka-remote/src/test/protocol/ProtobufProtocol.proto b/akka-remote/src/test/protocol/ProtobufProtocol.proto index a63ebdff62..84b3ab4990 100644 --- a/akka-remote/src/test/protocol/ProtobufProtocol.proto +++ b/akka-remote/src/test/protocol/ProtobufProtocol.proto @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor; +package akka.actor; /* Compile with: diff --git a/akka-remote/src/test/scala/Messages.scala b/akka-remote/src/test/scala/Messages.scala index 2b99155626..b2200885ac 100644 --- a/akka-remote/src/test/scala/Messages.scala +++ b/akka-remote/src/test/scala/Messages.scala @@ -2,9 +2,9 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka +package akka -import se.scalablesolutions.akka.serialization.Serializable +import akka.serialization.Serializable import sbinary._ import sbinary.Operations._ diff --git a/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala b/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala index ba550dc2aa..fb68efcb24 100644 --- a/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala +++ b/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala @@ -1,12 +1,12 @@ -package se.scalablesolutions.akka.actor.remote +package akka.actor.remote import java.util.concurrent.{CountDownLatch, TimeUnit} import org.scalatest.junit.JUnitSuite import org.junit.{Test, Before, After} -import se.scalablesolutions.akka.remote.{RemoteServer, RemoteClient} -import se.scalablesolutions.akka.dispatch.Dispatchers -import se.scalablesolutions.akka.actor.{ActorRef, Actor} +import akka.remote.{RemoteServer, RemoteClient} +import akka.dispatch.Dispatchers +import akka.actor.{ActorRef, Actor} import Actor._ object ClientInitiatedRemoteActorSpec { @@ -69,7 +69,7 @@ object ClientInitiatedRemoteActorSpec { class ClientInitiatedRemoteActorSpec extends JUnitSuite { import ClientInitiatedRemoteActorSpec._ - se.scalablesolutions.akka.config.Config.config + akka.config.Config.config val HOSTNAME = "localhost" val PORT1 = 9990 diff --git a/akka-remote/src/test/scala/remote/RemoteAgentSpec.scala b/akka-remote/src/test/scala/remote/RemoteAgentSpec.scala index 02a93e949b..df82850d0d 100644 --- a/akka-remote/src/test/scala/remote/RemoteAgentSpec.scala +++ b/akka-remote/src/test/scala/remote/RemoteAgentSpec.scala @@ -1,10 +1,10 @@ -package se.scalablesolutions.akka.actor.remote +package akka.actor.remote import org.scalatest.junit.JUnitSuite import org.junit.{Test, Before, After} -import se.scalablesolutions.akka.config.RemoteAddress -import se.scalablesolutions.akka.actor.Agent -import se.scalablesolutions.akka.remote. {RemoteClient, RemoteServer} +import akka.config.RemoteAddress +import akka.actor.Agent +import akka.remote. {RemoteClient, RemoteServer} class RemoteAgentSpec extends JUnitSuite { diff --git a/akka-remote/src/test/scala/remote/RemoteSupervisorSpec.scala b/akka-remote/src/test/scala/remote/RemoteSupervisorSpec.scala index 9834b035af..b340e89f45 100644 --- a/akka-remote/src/test/scala/remote/RemoteSupervisorSpec.scala +++ b/akka-remote/src/test/scala/remote/RemoteSupervisorSpec.scala @@ -2,16 +2,16 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor.remote +package akka.actor.remote import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, BlockingQueue} -import se.scalablesolutions.akka.serialization.BinaryString -import se.scalablesolutions.akka.config.Supervision._ -import se.scalablesolutions.akka.remote.{RemoteServer, RemoteClient} -import se.scalablesolutions.akka.OneWay +import akka.serialization.BinaryString +import akka.config.Supervision._ +import akka.remote.{RemoteServer, RemoteClient} +import akka.OneWay import org.scalatest.junit.JUnitSuite import org.junit.{Test, Before, After} -import se.scalablesolutions.akka.actor.{SupervisorFactory, Supervisor, ActorRef, Actor} +import akka.actor.{SupervisorFactory, Supervisor, ActorRef, Actor} import Actor._ object Log { diff --git a/akka-remote/src/test/scala/remote/RemoteTransactionalTypedActorSpec.scala b/akka-remote/src/test/scala/remote/RemoteTransactionalTypedActorSpec.scala index 71d44fd4bd..0e5d94aeee 100644 --- a/akka-remote/src/test/scala/remote/RemoteTransactionalTypedActorSpec.scala +++ b/akka-remote/src/test/scala/remote/RemoteTransactionalTypedActorSpec.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor import org.scalatest.Spec import org.scalatest.Assertions @@ -12,9 +12,9 @@ import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith import org.junit.{Test, Before, After} -import se.scalablesolutions.akka.config.Config -import se.scalablesolutions.akka.config.TypedActorConfigurator -import se.scalablesolutions.akka.remote.{RemoteNode, RemoteServer, RemoteClient} +import akka.config.Config +import akka.config.TypedActorConfigurator +import akka.remote.{RemoteNode, RemoteServer, RemoteClient} object RemoteTransactionalTypedActorSpec { val HOSTNAME = "localhost" diff --git a/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala b/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala index cd8f09a615..431c633102 100644 --- a/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala +++ b/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala @@ -2,19 +2,19 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor.remote +package akka.actor.remote import org.scalatest.matchers.ShouldMatchers import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.config.Supervision._ -import se.scalablesolutions.akka.actor._ -import se.scalablesolutions.akka.remote.{RemoteServer, RemoteClient} +import akka.config.Supervision._ +import akka.actor._ +import akka.remote.{RemoteServer, RemoteClient} import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, BlockingQueue} import org.scalatest.{BeforeAndAfterEach, Spec, Assertions, BeforeAndAfterAll} -import se.scalablesolutions.akka.config.{Config, TypedActorConfigurator, RemoteAddress} +import akka.config.{Config, TypedActorConfigurator, RemoteAddress} object RemoteTypedActorSpec { val HOSTNAME = "localhost" diff --git a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSample.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSample.scala index b9b8946bee..855d6cd723 100644 --- a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSample.scala +++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSample.scala @@ -1,8 +1,8 @@ -package se.scalablesolutions.akka.actor.remote +package akka.actor.remote -import se.scalablesolutions.akka.actor.Actor -import se.scalablesolutions.akka.remote.{RemoteClient, RemoteNode} -import se.scalablesolutions.akka.util.Logging +import akka.actor.Actor +import akka.remote.{RemoteClient, RemoteNode} +import akka.util.Logging import Actor._ diff --git a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala index 070682c794..c91b3745de 100644 --- a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala +++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala @@ -1,12 +1,12 @@ -package se.scalablesolutions.akka.actor.remote +package akka.actor.remote import java.util.concurrent.{CountDownLatch, TimeUnit} import org.scalatest.junit.JUnitSuite import org.junit.{Test, Before, After} -import se.scalablesolutions.akka.remote.{RemoteServer, RemoteClient} -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.actor.{ActorRegistry, ActorRef, Actor} +import akka.remote.{RemoteServer, RemoteClient} +import akka.actor.Actor._ +import akka.actor.{ActorRegistry, ActorRef, Actor} object ServerInitiatedRemoteActorSpec { val HOSTNAME = "localhost" @@ -84,7 +84,7 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite { @Test def shouldSendWithBang { val actor = RemoteClient.actorFor( - "se.scalablesolutions.akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", + "akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", 5000L, HOSTNAME, PORT) val result = actor ! "OneWay" @@ -95,7 +95,7 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite { @Test def shouldSendWithBangBangAndGetReply { val actor = RemoteClient.actorFor( - "se.scalablesolutions.akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorBidirectional", + "akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorBidirectional", 5000L, HOSTNAME, PORT) val result = actor !! "Hello" @@ -107,7 +107,7 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite { def shouldSendWithBangAndGetReplyThroughSenderRef { implicit val timeout = 500000000L val actor = RemoteClient.actorFor( - "se.scalablesolutions.akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorBidirectional", + "akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorBidirectional", timeout, HOSTNAME, PORT) val sender = actorOf[RemoteActorSpecActorAsyncSender] @@ -122,7 +122,7 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite { def shouldSendWithBangBangAndReplyWithException { implicit val timeout = 500000000L val actor = RemoteClient.actorFor( - "se.scalablesolutions.akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorBidirectional", + "akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorBidirectional", timeout, HOSTNAME, PORT) try { @@ -153,7 +153,7 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite { @Test def shouldNotRecreateRegisteredActor { server.register(actorOf[RemoteActorSpecActorUnidirectional]) - val actor = RemoteClient.actorFor("se.scalablesolutions.akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", HOSTNAME, PORT) + val actor = RemoteClient.actorFor("akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", HOSTNAME, PORT) val numberOfActorsInRegistry = ActorRegistry.actors.length actor ! "OneWay" assert(RemoteActorSpecActorUnidirectional.latch.await(1, TimeUnit.SECONDS)) @@ -165,7 +165,7 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite { 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 actor1 = RemoteClient.actorFor("akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", HOSTNAME, PORT) val actor2 = RemoteClient.actorFor("my-service", HOSTNAME, PORT) val actor3 = RemoteClient.actorFor("my-service", HOSTNAME, PORT) diff --git a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala index cdb8cf5cf2..c918c7e842 100644 --- a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala +++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor.remote +package akka.actor.remote import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -12,8 +12,8 @@ import org.junit.runner.RunWith import java.util.concurrent.TimeUnit -import se.scalablesolutions.akka.remote.{RemoteServer, RemoteClient} -import se.scalablesolutions.akka.actor._ +import akka.remote.{RemoteServer, RemoteClient} +import akka.actor._ import RemoteTypedActorLog._ object ServerInitiatedRemoteTypedActorSpec { diff --git a/akka-remote/src/test/scala/remote/ShutdownSpec.scala b/akka-remote/src/test/scala/remote/ShutdownSpec.scala index afc7610cb8..1f8787a1c7 100644 --- a/akka-remote/src/test/scala/remote/ShutdownSpec.scala +++ b/akka-remote/src/test/scala/remote/ShutdownSpec.scala @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.remote +package akka.remote -import se.scalablesolutions.akka.actor.Actor +import akka.actor.Actor import Actor._ diff --git a/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala b/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala index e05e4b0394..c6d8baed55 100644 --- a/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala +++ b/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala @@ -1,11 +1,11 @@ -package se.scalablesolutions.akka.actor.serialization +package akka.actor.serialization import java.util.concurrent.TimeUnit import org.scalatest.junit.JUnitSuite import org.junit.{Test, Before, After} -import se.scalablesolutions.akka.remote.{RemoteServer, RemoteClient} -import se.scalablesolutions.akka.actor.{ProtobufProtocol, Actor} +import akka.remote.{RemoteServer, RemoteClient} +import akka.actor.{ProtobufProtocol, Actor} import ProtobufProtocol.ProtobufPOJO import Actor._ diff --git a/akka-remote/src/test/scala/serialization/ScalaJSONSerializableSpec.scala b/akka-remote/src/test/scala/serialization/ScalaJSONSerializableSpec.scala index 68b2f171e4..7ccfc0d9b6 100644 --- a/akka-remote/src/test/scala/serialization/ScalaJSONSerializableSpec.scala +++ b/akka-remote/src/test/scala/serialization/ScalaJSONSerializableSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.serialization +package akka.serialization import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -6,7 +6,7 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.serialization.Serializable.ScalaJSON +import akka.serialization.Serializable.ScalaJSON object Serializables { import DefaultProtocol._ diff --git a/akka-remote/src/test/scala/serialization/ScalaJSONSerializerSpec.scala b/akka-remote/src/test/scala/serialization/ScalaJSONSerializerSpec.scala index 7d1ef4c7a0..45dc8d5090 100644 --- a/akka-remote/src/test/scala/serialization/ScalaJSONSerializerSpec.scala +++ b/akka-remote/src/test/scala/serialization/ScalaJSONSerializerSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.serialization +package akka.serialization import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -6,7 +6,7 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.serialization.Serializer.ScalaJSON +import akka.serialization.Serializer.ScalaJSON object Protocols { import sjson.json.DefaultProtocol._ diff --git a/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala b/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala index 681826a251..33e3a54cd0 100644 --- a/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala +++ b/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.actor.serialization +package akka.actor.serialization import org.scalatest.Spec @@ -7,9 +7,9 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.serialization._ +import akka.serialization._ // import dispatch.json._ -import se.scalablesolutions.akka.actor._ +import akka.actor._ import ActorSerialization._ import Actor._ diff --git a/akka-remote/src/test/scala/serialization/SerializerSpec.scala b/akka-remote/src/test/scala/serialization/SerializerSpec.scala index bff387ec99..9c503b3f97 100644 --- a/akka-remote/src/test/scala/serialization/SerializerSpec.scala +++ b/akka-remote/src/test/scala/serialization/SerializerSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.serialization +package akka.serialization import org.scalatest.junit.JUnitSuite import org.junit.Test diff --git a/akka-remote/src/test/scala/serialization/Ticket435Spec.scala b/akka-remote/src/test/scala/serialization/Ticket435Spec.scala index ed175ea0ad..f22c876808 100644 --- a/akka-remote/src/test/scala/serialization/Ticket435Spec.scala +++ b/akka-remote/src/test/scala/serialization/Ticket435Spec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.actor.serialization +package akka.actor.serialization import org.scalatest.Spec @@ -7,8 +7,8 @@ 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 akka.serialization._ +import akka.actor._ import ActorSerialization._ import Actor._ diff --git a/akka-remote/src/test/scala/serialization/TypedActorSerializationSpec.scala b/akka-remote/src/test/scala/serialization/TypedActorSerializationSpec.scala index ccf4d05f7f..15a7fa3601 100644 --- a/akka-remote/src/test/scala/serialization/TypedActorSerializationSpec.scala +++ b/akka-remote/src/test/scala/serialization/TypedActorSerializationSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor.serialization +package akka.actor.serialization import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -9,13 +9,13 @@ 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 akka.serialization._ +import akka.actor._ import TypedActorSerialization._ import Actor._ -import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer} -import se.scalablesolutions.akka.actor.remote.ServerInitiatedRemoteActorSpec.RemoteActorSpecActorUnidirectional +import akka.remote.{RemoteClient, RemoteServer} +import akka.actor.remote.ServerInitiatedRemoteActorSpec.RemoteActorSpecActorUnidirectional @RunWith(classOf[JUnitRunner]) class TypedActorSerializationSpec extends diff --git a/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala b/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala index d230bbaffe..e844ff0104 100644 --- a/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala +++ b/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.actor.serialization +package akka.actor.serialization import org.scalatest.Spec @@ -7,8 +7,8 @@ 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 akka.serialization._ +import akka.actor._ import ActorSerialization._ import Actor._ diff --git a/akka-remote/src/test/scala/ticket/Ticket001Spec.scala b/akka-remote/src/test/scala/ticket/Ticket001Spec.scala index b94796d9a3..d4de2675fb 100644 --- a/akka-remote/src/test/scala/ticket/Ticket001Spec.scala +++ b/akka-remote/src/test/scala/ticket/Ticket001Spec.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.actor.ticket +package akka.actor.ticket import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers diff --git a/akka-remote/src/test/scala/ticket/Ticket434Spec.scala b/akka-remote/src/test/scala/ticket/Ticket434Spec.scala index 29944d8e0b..75854b4e04 100644 --- a/akka-remote/src/test/scala/ticket/Ticket434Spec.scala +++ b/akka-remote/src/test/scala/ticket/Ticket434Spec.scala @@ -1,16 +1,16 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor.ticket +package akka.actor.ticket import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.actor.{Uuid,newUuid,uuidFrom} -import se.scalablesolutions.akka.actor.remote.ServerInitiatedRemoteActorSpec.RemoteActorSpecActorUnidirectional +import akka.actor.Actor._ +import akka.actor.{Uuid,newUuid,uuidFrom} +import akka.actor.remote.ServerInitiatedRemoteActorSpec.RemoteActorSpecActorUnidirectional import java.util.concurrent.TimeUnit -import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer} -import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._ +import akka.remote.{RemoteClient, RemoteServer} +import akka.remote.protocol.RemoteProtocol._ class Ticket434Spec extends Spec with ShouldMatchers { diff --git a/akka-samples/akka-sample-ants/src/main/scala/Ants.scala b/akka-samples/akka-sample-ants/src/main/scala/Ants.scala index 0bf8dc4fdf..5ff6e9ceb1 100644 --- a/akka-samples/akka-sample-ants/src/main/scala/Ants.scala +++ b/akka-samples/akka-sample-ants/src/main/scala/Ants.scala @@ -6,7 +6,6 @@ package sample.ants import java.util.concurrent.TimeUnit import scala.util.Random.{nextInt => randomInt} -import se.scalablesolutions.akka import akka.actor.{Actor, ActorRef, Scheduler} import akka.actor.Actor.actorOf import akka.stm.local._ diff --git a/akka-samples/akka-sample-ants/src/main/spde/Ants.spde b/akka-samples/akka-sample-ants/src/main/spde/Ants.spde index 05565673d6..ad7dce1239 100644 --- a/akka-samples/akka-sample-ants/src/main/spde/Ants.spde +++ b/akka-samples/akka-sample-ants/src/main/spde/Ants.spde @@ -1,6 +1,6 @@ import sample.ants._ import sample.ants.Config._ -import se.scalablesolutions.akka.stm.local._ +import akka.stm.local._ val scale = 5 diff --git a/akka-samples/akka-sample-camel/src/main/java/sample/camel/BeanImpl.java b/akka-samples/akka-sample-camel/src/main/java/sample/camel/BeanImpl.java index e1b4520f59..9ceba85d64 100644 --- a/akka-samples/akka-sample-camel/src/main/java/sample/camel/BeanImpl.java +++ b/akka-samples/akka-sample-camel/src/main/java/sample/camel/BeanImpl.java @@ -1,6 +1,6 @@ package sample.camel; -import se.scalablesolutions.akka.actor.TypedActor; +import akka.actor.TypedActor; /** * @author Martin Krasser */ diff --git a/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer1.java b/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer1.java index 3e451af524..3e8ce1e20f 100644 --- a/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer1.java +++ b/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer1.java @@ -3,7 +3,7 @@ package sample.camel; import org.apache.camel.Body; import org.apache.camel.Header; -import se.scalablesolutions.akka.camel.consume; +import akka.camel.consume; /** * @author Martin Krasser 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 0e05d5036c..522db0e4a7 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,6 +1,6 @@ package sample.camel; -import se.scalablesolutions.akka.actor.TypedActor; +import akka.actor.TypedActor; /** * @author Martin Krasser 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 index 3b2494ee4e..ba093a1d96 100644 --- 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 @@ -2,7 +2,7 @@ package sample.camel; import org.apache.camel.Body; import org.apache.camel.Header; -import se.scalablesolutions.akka.camel.consume; +import akka.camel.consume; /** * @author Martin Krasser 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 5c76485aa7..b3475ad2d6 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,6 +1,6 @@ package sample.camel; -import se.scalablesolutions.akka.actor.TypedActor; +import akka.actor.TypedActor; /** * @author Martin Krasser diff --git a/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer1.java b/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer1.java index 78a8430880..6213fb8f09 100644 --- a/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer1.java +++ b/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer1.java @@ -3,7 +3,7 @@ package sample.camel; import org.apache.camel.Body; import org.apache.camel.Header; -import se.scalablesolutions.akka.camel.consume; +import akka.camel.consume; /** * @author Martin Krasser diff --git a/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer1Impl.java b/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer1Impl.java index dc4014fb1f..bd735fe14b 100644 --- a/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer1Impl.java +++ b/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer1Impl.java @@ -3,7 +3,7 @@ package sample.camel; import org.apache.camel.Body; import org.apache.camel.Header; -import se.scalablesolutions.akka.actor.TypedActor; +import akka.actor.TypedActor; /** * @author Martin Krasser diff --git a/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer2.java b/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer2.java index e8fe43c167..9a39b534b5 100644 --- a/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer2.java +++ b/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer2.java @@ -2,7 +2,7 @@ package sample.camel; import org.apache.camel.Body; import org.apache.camel.Header; -import se.scalablesolutions.akka.camel.consume; +import akka.camel.consume; /** * @author Martin Krasser diff --git a/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer2Impl.java b/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer2Impl.java index 9e61a78804..ed82810c10 100644 --- a/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer2Impl.java +++ b/akka-samples/akka-sample-camel/src/main/java/sample/camel/TypedConsumer2Impl.java @@ -1,6 +1,6 @@ package sample.camel; -import se.scalablesolutions.akka.actor.TypedActor; +import akka.actor.TypedActor; /** * @author Martin Krasser diff --git a/akka-samples/akka-sample-camel/src/main/scala/Actors.scala b/akka-samples/akka-sample-camel/src/main/scala/Actors.scala index db2aab1729..15010c4ed6 100644 --- a/akka-samples/akka-sample-camel/src/main/scala/Actors.scala +++ b/akka-samples/akka-sample-camel/src/main/scala/Actors.scala @@ -2,9 +2,9 @@ package sample.camel import org.apache.camel.Exchange -import se.scalablesolutions.akka.actor.{Actor, ActorRef, RemoteActor} -import se.scalablesolutions.akka.camel.{Failure, Producer, Message, Consumer} -import se.scalablesolutions.akka.util.Logging +import akka.actor.{Actor, ActorRef, RemoteActor} +import akka.camel.{Failure, Producer, Message, Consumer} +import akka.util.Logging /** * Client-initiated remote actor. diff --git a/akka-samples/akka-sample-camel/src/main/scala/Boot.scala b/akka-samples/akka-sample-camel/src/main/scala/Boot.scala index 3892583026..d8fe43a7a9 100644 --- a/akka-samples/akka-sample-camel/src/main/scala/Boot.scala +++ b/akka-samples/akka-sample-camel/src/main/scala/Boot.scala @@ -6,10 +6,10 @@ import org.apache.camel.impl.DefaultCamelContext import org.apache.camel.spring.spi.ApplicationContextRegistry import org.springframework.context.support.ClassPathXmlApplicationContext -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.actor.{TypedActor, Supervisor} -import se.scalablesolutions.akka.camel.CamelContextManager -import se.scalablesolutions.akka.config.Supervision._ +import akka.actor.Actor._ +import akka.actor.{TypedActor, Supervisor} +import akka.camel.CamelContextManager +import akka.config.Supervision._ /** * @author Martin Krasser 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 9fab1b9649..7b3d70df80 100644 --- a/akka-samples/akka-sample-camel/src/main/scala/ClientApplication.scala +++ b/akka-samples/akka-sample-camel/src/main/scala/ClientApplication.scala @@ -1,9 +1,9 @@ package sample.camel -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.actor.TypedActor -import se.scalablesolutions.akka.camel.Message -import se.scalablesolutions.akka.remote.RemoteClient +import akka.actor.Actor._ +import akka.actor.TypedActor +import akka.camel.Message +import akka.remote.RemoteClient /** * @author Martin Krasser 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 b3ef404841..40f68e510b 100644 --- a/akka-samples/akka-sample-camel/src/main/scala/ServerApplication.scala +++ b/akka-samples/akka-sample-camel/src/main/scala/ServerApplication.scala @@ -1,9 +1,9 @@ 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 +import akka.actor.Actor._ +import akka.remote.RemoteNode +import akka.camel.CamelServiceManager +import akka.actor.TypedActor /** * @author Martin Krasser diff --git a/akka-samples/akka-sample-camel/src/main/scala/StandaloneApplication.scala b/akka-samples/akka-sample-camel/src/main/scala/StandaloneApplication.scala index 2ecccb1e02..b083509216 100644 --- a/akka-samples/akka-sample-camel/src/main/scala/StandaloneApplication.scala +++ b/akka-samples/akka-sample-camel/src/main/scala/StandaloneApplication.scala @@ -5,8 +5,8 @@ import org.apache.camel.builder.RouteBuilder import org.apache.camel.spring.spi.ApplicationContextRegistry import org.springframework.context.support.ClassPathXmlApplicationContext -import se.scalablesolutions.akka.actor.{Actor, ActorRegistry, TypedActor} -import se.scalablesolutions.akka.camel._ +import akka.actor.{Actor, ActorRegistry, TypedActor} +import akka.camel._ /** * @author Martin Krasser diff --git a/akka-samples/akka-sample-camel/src/test/scala/HttpConcurrencyTestStress.scala b/akka-samples/akka-sample-camel/src/test/scala/HttpConcurrencyTestStress.scala index 76cbc58a8b..bca3fed321 100644 --- a/akka-samples/akka-sample-camel/src/test/scala/HttpConcurrencyTestStress.scala +++ b/akka-samples/akka-sample-camel/src/test/scala/HttpConcurrencyTestStress.scala @@ -7,12 +7,12 @@ import java.util.concurrent.CountDownLatch import org.junit._ import org.scalatest.junit.JUnitSuite -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.actor.{ActorRegistry, ActorRef, Actor} -import se.scalablesolutions.akka.camel._ -import se.scalablesolutions.akka.camel.CamelServiceManager._ -import se.scalablesolutions.akka.routing.CyclicIterator -import se.scalablesolutions.akka.routing.Routing._ +import akka.actor.Actor._ +import akka.actor.{ActorRegistry, ActorRef, Actor} +import akka.camel._ +import akka.camel.CamelServiceManager._ +import akka.routing.CyclicIterator +import akka.routing.Routing._ /** * @author Martin Krasser diff --git a/akka-samples/akka-sample-chat/README b/akka-samples/akka-sample-chat/README index fec39724e1..dff045d6f8 100644 --- a/akka-samples/akka-sample-chat/README +++ b/akka-samples/akka-sample-chat/README @@ -18,7 +18,7 @@ Then to run the sample: - Run 'sbt console' to start up a REPL (interpreter). 4. In the first REPL you get execute: - scala> import sample.chat._ - - scala> import se.scalablesolutions.akka.actor.Actor._ + - scala> import akka.actor.Actor._ - scala> val chatService = actorOf[ChatService].start 5. In the second REPL you get execute: - scala> import sample.chat._ 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 aa70cabeb9..142ffc20a3 100644 --- a/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala +++ b/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala @@ -6,13 +6,13 @@ package sample.chat import scala.collection.mutable.HashMap -import se.scalablesolutions.akka.actor.{SupervisorFactory, Actor, ActorRef, RemoteActor} -import se.scalablesolutions.akka.remote.{RemoteNode, RemoteClient} -import se.scalablesolutions.akka.persistence.common.PersistentVector -import se.scalablesolutions.akka.persistence.redis.RedisStorage -import se.scalablesolutions.akka.stm.global._ -import se.scalablesolutions.akka.config.Supervision.{OneForOneStrategy,Permanent} -import se.scalablesolutions.akka.util.Logging +import akka.actor.{SupervisorFactory, Actor, ActorRef, RemoteActor} +import akka.remote.{RemoteNode, RemoteClient} +import akka.persistence.common.PersistentVector +import akka.persistence.redis.RedisStorage +import akka.stm.global._ +import akka.config.Supervision.{OneForOneStrategy,Permanent} +import akka.util.Logging import Actor._ /****************************************************************************** @@ -34,7 +34,7 @@ Then to run the sample: - Run 'sbt console' to start up a REPL (interpreter). 2. In the first REPL you get execute: - scala> import sample.chat._ - - scala> import se.scalablesolutions.akka.actor.Actor._ + - scala> import akka.actor.Actor._ - scala> val chatService = actorOf[ChatService].start 3. In the second REPL you get execute: - scala> import sample.chat._ diff --git a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala index 189ea2745c..c762c2da7d 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala @@ -3,8 +3,8 @@ package sample.fsm.dining.become //Akka adaptation of //http://www.dalnefre.com/wp/2010/08/dining-philosophers-in-humus/ -import se.scalablesolutions.akka.actor.{Scheduler, ActorRef, Actor} -import se.scalablesolutions.akka.actor.Actor._ +import akka.actor.{Scheduler, ActorRef, Actor} +import akka.actor.Actor._ import java.util.concurrent.TimeUnit /* diff --git a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala index ecb4d82ba0..bb90c981d5 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala @@ -1,6 +1,6 @@ package sample.fsm.dining.fsm -import se.scalablesolutions.akka.actor.{ActorRef, Actor, FSM} +import akka.actor.{ActorRef, Actor, FSM} import Actor._ /* @@ -31,7 +31,7 @@ class Chopstick(name: String) extends Actor with FSM[ChopstickState, TakenBy] { self.id = name // When a chopstick is available, it can be taken by a some hakker - inState(Available) { + when(Available) { case Event(Take, _) => goto(Taken) using TakenBy(self.sender) replying Taken(self) } @@ -39,7 +39,7 @@ class Chopstick(name: String) extends Actor with FSM[ChopstickState, TakenBy] { // When a chopstick is taken by a hakker // It will refuse to be taken by other hakkers // But the owning hakker can put it back - inState(Taken) { + when(Taken) { case Event(Take, currentState) => stay replying Busy(self) case Event(Put, TakenBy(hakker)) if self.sender == hakker => @@ -47,7 +47,7 @@ class Chopstick(name: String) extends Actor with FSM[ChopstickState, TakenBy] { } // A chopstick begins its existence as available and taken by no one - setInitialState(Available, TakenBy(None)) + startWith(Available, TakenBy(None)) } /** @@ -78,7 +78,7 @@ case class TakenChopsticks(left: Option[ActorRef], right: Option[ActorRef]) class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor with FSM[FSMHakkerState, TakenChopsticks] { self.id = name - inState(Waiting) { + when(Waiting) { case Event(Think, _) => log.info("%s starts to think", name) startThinking(5000) @@ -86,7 +86,7 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit //When a hakker is thinking it can become hungry //and try to pick up its chopsticks and eat - inState(Thinking) { + when(Thinking) { case Event(StateTimeout, _) => left ! Take right ! Take @@ -97,7 +97,7 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit // When it picks one up, it goes into wait for the other // If the hakkers first attempt at grabbing a chopstick fails, // it starts to wait for the response of the other grab - inState(Hungry) { + when(Hungry) { case Event(Taken(`left`), _) => goto(WaitForOtherChopstick) using TakenChopsticks(Some(left), None) case Event(Taken(`right`), _) => @@ -109,7 +109,7 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit // When a hakker is waiting for the last chopstick it can either obtain it // and start eating, or the other chopstick was busy, and the hakker goes // back to think about how he should obtain his chopsticks :-) - inState(WaitForOtherChopstick) { + when(WaitForOtherChopstick) { case Event(Taken(`left`), TakenChopsticks(None, Some(right))) => startEating(left, right) case Event(Taken(`right`), TakenChopsticks(Some(left), None)) => startEating(left, right) case Event(Busy(chopstick), TakenChopsticks(leftOption, rightOption)) => @@ -126,7 +126,7 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit // When the results of the other grab comes back, // he needs to put it back if he got the other one. // Then go back and think and try to grab the chopsticks again - inState(FirstChopstickDenied) { + when(FirstChopstickDenied) { case Event(Taken(secondChopstick), _) => secondChopstick ! Put startThinking(10) @@ -136,7 +136,7 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit // When a hakker is eating, he can decide to start to think, // then he puts down his chopsticks and starts to think - inState(Eating) { + when(Eating) { case Event(StateTimeout, _) => log.info("%s puts down his chopsticks and starts to think", name) left ! Put @@ -149,7 +149,7 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit } //All hakkers start waiting - setInitialState(Waiting, TakenChopsticks(None, None)) + startWith(Waiting, TakenChopsticks(None, None)) } /* diff --git a/akka-samples/akka-sample-osgi/src/main/scala/osgiExample.scala b/akka-samples/akka-sample-osgi/src/main/scala/osgiExample.scala index 18323d04d3..276c0033fb 100644 --- a/akka-samples/akka-sample-osgi/src/main/scala/osgiExample.scala +++ b/akka-samples/akka-sample-osgi/src/main/scala/osgiExample.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka +package akka package sample.osgi import actor.{ Actor, ActorRegistry } diff --git a/akka-samples/akka-sample-pubsub/src/main/scala/RedisPubSub.scala b/akka-samples/akka-sample-pubsub/src/main/scala/RedisPubSub.scala index ca3148aab7..c6b6c57da9 100644 --- a/akka-samples/akka-sample-pubsub/src/main/scala/RedisPubSub.scala +++ b/akka-samples/akka-sample-pubsub/src/main/scala/RedisPubSub.scala @@ -5,8 +5,8 @@ package sample.pubsub import com.redis.{RedisClient, PubSubMessage, S, U, M} -import se.scalablesolutions.akka.persistence.redis._ -import se.scalablesolutions.akka.actor.Actor._ +import akka.persistence.redis._ +import akka.actor.Actor._ /** * Sample Akka application for Redis PubSub diff --git a/akka-samples/akka-sample-remote/src/main/scala/ClientManagedRemoteActorSample.scala b/akka-samples/akka-sample-remote/src/main/scala/ClientManagedRemoteActorSample.scala index 24f81872f7..692c35de8a 100644 --- a/akka-samples/akka-sample-remote/src/main/scala/ClientManagedRemoteActorSample.scala +++ b/akka-samples/akka-sample-remote/src/main/scala/ClientManagedRemoteActorSample.scala @@ -4,10 +4,10 @@ package sample.remote -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.actor.RemoteActor -import se.scalablesolutions.akka.remote.RemoteNode -import se.scalablesolutions.akka.util.Logging +import akka.actor.Actor._ +import akka.actor.RemoteActor +import akka.remote.RemoteNode +import akka.util.Logging class RemoteHelloWorldActor extends RemoteActor("localhost", 9999) { def receive = { diff --git a/akka-samples/akka-sample-remote/src/main/scala/ServerManagedRemoteActorSample.scala b/akka-samples/akka-sample-remote/src/main/scala/ServerManagedRemoteActorSample.scala index 2671d9e25e..3ee0934a13 100644 --- a/akka-samples/akka-sample-remote/src/main/scala/ServerManagedRemoteActorSample.scala +++ b/akka-samples/akka-sample-remote/src/main/scala/ServerManagedRemoteActorSample.scala @@ -4,10 +4,10 @@ package sample.remote -import se.scalablesolutions.akka.actor.Actor -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.remote.{RemoteClient, RemoteNode} -import se.scalablesolutions.akka.util.Logging +import akka.actor.Actor +import akka.actor.Actor._ +import akka.remote.{RemoteClient, RemoteNode} +import akka.util.Logging class HelloWorldActor extends Actor { def receive = { diff --git a/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/Boot.java b/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/Boot.java index d9b33cf559..530d396abb 100644 --- a/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/Boot.java +++ b/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/Boot.java @@ -4,8 +4,8 @@ package sample.rest.java; -import se.scalablesolutions.akka.config.TypedActorConfigurator; -import static se.scalablesolutions.akka.config.Supervision.*; +import akka.config.TypedActorConfigurator; +import static akka.config.Supervision.*; public class Boot { public final static TypedActorConfigurator configurator = new TypedActorConfigurator(); diff --git a/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/PersistentSimpleServiceImpl.java b/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/PersistentSimpleServiceImpl.java index 28a332abfe..f59262d693 100644 --- a/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/PersistentSimpleServiceImpl.java +++ b/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/PersistentSimpleServiceImpl.java @@ -4,9 +4,9 @@ package sample.rest.java; -import se.scalablesolutions.akka.actor.TypedTransactor; -import se.scalablesolutions.akka.persistence.common.PersistentMap; -import se.scalablesolutions.akka.persistence.cassandra.CassandraStorage; +import akka.actor.TypedTransactor; +import akka.persistence.common.PersistentMap; +import akka.persistence.cassandra.CassandraStorage; import java.nio.ByteBuffer; diff --git a/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/ReceiverImpl.java b/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/ReceiverImpl.java index 4a200dfa77..4e00f831ab 100644 --- a/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/ReceiverImpl.java +++ b/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/ReceiverImpl.java @@ -4,7 +4,7 @@ package sample.rest.java; -import se.scalablesolutions.akka.actor.TypedActor; +import akka.actor.TypedActor; public class ReceiverImpl extends TypedActor implements Receiver { public SimpleService get() { diff --git a/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/SimpleServiceImpl.java b/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/SimpleServiceImpl.java index 9de40a1a04..0eb7be096a 100644 --- a/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/SimpleServiceImpl.java +++ b/akka-samples/akka-sample-rest-java/src/main/java/sample/rest/java/SimpleServiceImpl.java @@ -4,9 +4,9 @@ package sample.rest.java; -import se.scalablesolutions.akka.actor.TypedActor; -import se.scalablesolutions.akka.actor.TypedTransactor; -import se.scalablesolutions.akka.stm.TransactionalMap; +import akka.actor.TypedActor; +import akka.actor.TypedTransactor; +import akka.stm.TransactionalMap; public class SimpleServiceImpl extends TypedTransactor implements SimpleService { private String KEY = "COUNTER"; diff --git a/akka-samples/akka-sample-rest-scala/src/main/scala/SimpleService.scala b/akka-samples/akka-sample-rest-scala/src/main/scala/SimpleService.scala index a316c54fcc..ccfd6fd286 100644 --- a/akka-samples/akka-sample-rest-scala/src/main/scala/SimpleService.scala +++ b/akka-samples/akka-sample-rest-scala/src/main/scala/SimpleService.scala @@ -4,19 +4,19 @@ package sample.rest.scala -import se.scalablesolutions.akka.actor.{Transactor, SupervisorFactory, Actor} -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.stm.TransactionalMap -import se.scalablesolutions.akka.persistence.cassandra.CassandraStorage -import se.scalablesolutions.akka.config.Supervision._ -import se.scalablesolutions.akka.util.Logging -import se.scalablesolutions.akka.comet.AkkaClusterBroadcastFilter +import akka.actor.{Transactor, SupervisorFactory, Actor} +import akka.actor.Actor._ +import akka.stm.TransactionalMap +import akka.persistence.cassandra.CassandraStorage +import akka.config.Supervision._ +import akka.util.Logging +import akka.comet.AkkaClusterBroadcastFilter import scala.xml.NodeSeq import java.lang.Integer import java.nio.ByteBuffer import javax.ws.rs.core.MultivaluedMap import javax.ws.rs.{GET, POST, Path, Produces, WebApplicationException, Consumes,PathParam} -import se.scalablesolutions.akka.actor.ActorRegistry.actorFor +import akka.actor.ActorRegistry.actorFor import org.atmosphere.annotation.{Broadcast, Suspend,Cluster} import org.atmosphere.util.XSSHtmlFilter import org.atmosphere.cpr.{Broadcaster, BroadcastFilter} diff --git a/akka-samples/akka-sample-security/src/main/scala/SimpleService.scala b/akka-samples/akka-sample-security/src/main/scala/SimpleService.scala index 0f2dd0e59a..d42ae12f2e 100644 --- a/akka-samples/akka-sample-security/src/main/scala/SimpleService.scala +++ b/akka-samples/akka-sample-security/src/main/scala/SimpleService.scala @@ -4,13 +4,13 @@ package sample.security -import se.scalablesolutions.akka.actor.{SupervisorFactory, Transactor, Actor} -import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.config.Supervision._ -import se.scalablesolutions.akka.util.Logging -import se.scalablesolutions.akka.security.{BasicAuthenticationActor,BasicCredentials,SpnegoAuthenticationActor,DigestAuthenticationActor, UserInfo} -import se.scalablesolutions.akka.stm.TransactionalMap -import se.scalablesolutions.akka.actor.ActorRegistry.actorFor +import akka.actor.{SupervisorFactory, Transactor, Actor} +import akka.actor.Actor._ +import akka.config.Supervision._ +import akka.util.Logging +import akka.security.{BasicAuthenticationActor,BasicCredentials,SpnegoAuthenticationActor,DigestAuthenticationActor, UserInfo} +import akka.stm.TransactionalMap +import akka.actor.ActorRegistry.actorFor class Boot { val factory = SupervisorFactory( diff --git a/akka-samples/akka-sample-security/src/main/webapp/WEB-INF/web.xml b/akka-samples/akka-sample-security/src/main/webapp/WEB-INF/web.xml index b81f94c9e6..e9d5bbb4db 100644 --- a/akka-samples/akka-sample-security/src/main/webapp/WEB-INF/web.xml +++ b/akka-samples/akka-sample-security/src/main/webapp/WEB-INF/web.xml @@ -8,7 +8,7 @@ AkkaServlet - se.scalablesolutions.akka.rest.AkkaServlet + akka.rest.AkkaServlet diff --git a/akka-sbt-plugin/project/build.properties b/akka-sbt-plugin/project/build.properties index 0fe93cd2c6..984cdaa83b 100644 --- a/akka-sbt-plugin/project/build.properties +++ b/akka-sbt-plugin/project/build.properties @@ -1,5 +1,5 @@ project.name=Akka SBT Plugin -project.organization=se.scalablesolutions.akka +project.organization=akka # mirrors akka version project.version=1.0-SNAPSHOT sbt.version=0.7.4 diff --git a/akka-sbt-plugin/src/main/scala/AkkaProject.scala b/akka-sbt-plugin/src/main/scala/AkkaProject.scala index 82ccbe401a..b89b159c41 100644 --- a/akka-sbt-plugin/src/main/scala/AkkaProject.scala +++ b/akka-sbt-plugin/src/main/scala/AkkaProject.scala @@ -17,7 +17,7 @@ trait AkkaBaseProject extends BasicScalaProject { // is resolved from a ModuleConfiguration. This will result in a significant acceleration of the update action. // for development version resolve to .ivy2/local - // val akkaModuleConfig = ModuleConfiguration("se.scalablesolutions.akka", AkkaRepo) + // val akkaModuleConfig = ModuleConfiguration("akka", AkkaRepo) val aspectwerkzModuleConfig = ModuleConfiguration("org.codehaus.aspectwerkz", AkkaRepo) val cassandraModuleConfig = ModuleConfiguration("org.apache.cassandra", AkkaRepo) @@ -48,7 +48,7 @@ trait AkkaProject extends AkkaBaseProject { val akkaVersion = "1.0-SNAPSHOT" // convenience method - def akkaModule(module: String) = "se.scalablesolutions.akka" %% ("akka-" + module) % akkaVersion + def akkaModule(module: String) = "akka" %% ("akka-" + module) % akkaVersion // akka remote dependency by default val akkaRemote = akkaModule("remote") diff --git a/akka-spring/src/main/resources/META-INF/spring.handlers b/akka-spring/src/main/resources/META-INF/spring.handlers index 9e8b198496..0812c4dd2e 100644 --- a/akka-spring/src/main/resources/META-INF/spring.handlers +++ b/akka-spring/src/main/resources/META-INF/spring.handlers @@ -1 +1 @@ -http\://www.akkasource.org/schema/akka=se.scalablesolutions.akka.spring.AkkaNamespaceHandler +http\://www.akkasource.org/schema/akka=akka.spring.AkkaNamespaceHandler diff --git a/akka-spring/src/main/resources/META-INF/spring.schemas b/akka-spring/src/main/resources/META-INF/spring.schemas index 4879dc82a4..37ec717596 100644 --- a/akka-spring/src/main/resources/META-INF/spring.schemas +++ b/akka-spring/src/main/resources/META-INF/spring.schemas @@ -1 +1 @@ -http\://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd=se/scalablesolutions/akka/spring/akka-1.0-SNAPSHOT.xsd +http\://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd=akka/spring/akka-1.0-SNAPSHOT.xsd diff --git a/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-0.10.xsd b/akka-spring/src/main/resources/akka/spring/akka-0.10.xsd similarity index 100% rename from akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-0.10.xsd rename to akka-spring/src/main/resources/akka/spring/akka-0.10.xsd diff --git a/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-1.0-SNAPSHOT.xsd b/akka-spring/src/main/resources/akka/spring/akka-1.0-SNAPSHOT.xsd similarity index 100% rename from akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-1.0-SNAPSHOT.xsd rename to akka-spring/src/main/resources/akka/spring/akka-1.0-SNAPSHOT.xsd diff --git a/akka-spring/src/main/scala/ActorBeanDefinitionParser.scala b/akka-spring/src/main/scala/ActorBeanDefinitionParser.scala index 6c69886e2e..f0ca235615 100644 --- a/akka-spring/src/main/scala/ActorBeanDefinitionParser.scala +++ b/akka-spring/src/main/scala/ActorBeanDefinitionParser.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.springframework.beans.factory.support.BeanDefinitionBuilder import org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser diff --git a/akka-spring/src/main/scala/ActorFactoryBean.scala b/akka-spring/src/main/scala/ActorFactoryBean.scala index 87233ab451..c587f12609 100644 --- a/akka-spring/src/main/scala/ActorFactoryBean.scala +++ b/akka-spring/src/main/scala/ActorFactoryBean.scala @@ -2,17 +2,17 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.springframework.beans.{BeanUtils,BeansException,BeanWrapper,BeanWrapperImpl} -import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer} +import akka.remote.{RemoteClient, RemoteServer} import org.springframework.beans.factory.config.AbstractFactoryBean import org.springframework.context.{ApplicationContext,ApplicationContextAware} 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 akka.actor.{ActorRef, AspectInitRegistry, TypedActorConfiguration, TypedActor,Actor} +import akka.dispatch.MessageDispatcher +import akka.util.{Logging, Duration} import scala.reflect.BeanProperty import java.net.InetSocketAddress diff --git a/akka-spring/src/main/scala/ActorParser.scala b/akka-spring/src/main/scala/ActorParser.scala index e8048d1cd2..3a0c756f20 100644 --- a/akka-spring/src/main/scala/ActorParser.scala +++ b/akka-spring/src/main/scala/ActorParser.scala @@ -1,12 +1,12 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.springframework.util.xml.DomUtils import org.w3c.dom.Element import scala.collection.JavaConversions._ -import se.scalablesolutions.akka.util.Logging +import akka.util.Logging /** * Parser trait for custom namespace configuration for typed-actor. diff --git a/akka-spring/src/main/scala/ActorProperties.scala b/akka-spring/src/main/scala/ActorProperties.scala index 3f811644c7..487c3530da 100644 --- a/akka-spring/src/main/scala/ActorProperties.scala +++ b/akka-spring/src/main/scala/ActorProperties.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.springframework.beans.factory.support.BeanDefinitionBuilder import AkkaSpringConfigurationTags._ diff --git a/akka-spring/src/main/scala/AkkaNamespaceHandler.scala b/akka-spring/src/main/scala/AkkaNamespaceHandler.scala index 91c46c99e8..38041a3ea4 100644 --- a/akka-spring/src/main/scala/AkkaNamespaceHandler.scala +++ b/akka-spring/src/main/scala/AkkaNamespaceHandler.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.springframework.beans.factory.xml.NamespaceHandlerSupport import AkkaSpringConfigurationTags._ diff --git a/akka-spring/src/main/scala/AkkaSpringConfigurationTags.scala b/akka-spring/src/main/scala/AkkaSpringConfigurationTags.scala index 1eef274df6..0871797810 100644 --- a/akka-spring/src/main/scala/AkkaSpringConfigurationTags.scala +++ b/akka-spring/src/main/scala/AkkaSpringConfigurationTags.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring /** * XML configuration tags. diff --git a/akka-spring/src/main/scala/CamelServiceBeanDefinitionParser.scala b/akka-spring/src/main/scala/CamelServiceBeanDefinitionParser.scala index c7f95d8a3a..4025a831a8 100644 --- a/akka-spring/src/main/scala/CamelServiceBeanDefinitionParser.scala +++ b/akka-spring/src/main/scala/CamelServiceBeanDefinitionParser.scala @@ -1,14 +1,14 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.springframework.beans.factory.support.BeanDefinitionBuilder import org.springframework.beans.factory.xml.{ParserContext, AbstractSingleBeanDefinitionParser} import org.springframework.util.xml.DomUtils import org.w3c.dom.Element -import se.scalablesolutions.akka.spring.AkkaSpringConfigurationTags._ +import akka.spring.AkkaSpringConfigurationTags._ /** diff --git a/akka-spring/src/main/scala/CamelServiceFactoryBean.scala b/akka-spring/src/main/scala/CamelServiceFactoryBean.scala index b491a38072..337413f0eb 100644 --- a/akka-spring/src/main/scala/CamelServiceFactoryBean.scala +++ b/akka-spring/src/main/scala/CamelServiceFactoryBean.scala @@ -1,12 +1,12 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.apache.camel.CamelContext import org.springframework.beans.factory.{DisposableBean, InitializingBean, FactoryBean} -import se.scalablesolutions.akka.camel.{CamelContextManager, CamelService, CamelServiceFactory} +import akka.camel.{CamelContextManager, CamelService, CamelServiceFactory} /** * Factory bean for a {@link CamelService}. diff --git a/akka-spring/src/main/scala/ConfiggyPropertyPlaceholderConfigurer.scala b/akka-spring/src/main/scala/ConfiggyPropertyPlaceholderConfigurer.scala index 1360b62d9c..d6ac5e5d03 100644 --- a/akka-spring/src/main/scala/ConfiggyPropertyPlaceholderConfigurer.scala +++ b/akka-spring/src/main/scala/ConfiggyPropertyPlaceholderConfigurer.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.springframework.beans.factory.config.PropertyPlaceholderConfigurer import org.springframework.core.io.Resource diff --git a/akka-spring/src/main/scala/DispatcherBeanDefinitionParser.scala b/akka-spring/src/main/scala/DispatcherBeanDefinitionParser.scala index 90c56b0b5b..4f2a40469f 100644 --- a/akka-spring/src/main/scala/DispatcherBeanDefinitionParser.scala +++ b/akka-spring/src/main/scala/DispatcherBeanDefinitionParser.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.w3c.dom.Element import org.springframework.beans.factory.support.BeanDefinitionBuilder diff --git a/akka-spring/src/main/scala/DispatcherFactoryBean.scala b/akka-spring/src/main/scala/DispatcherFactoryBean.scala index cfe0e05930..bdcfca5d33 100644 --- a/akka-spring/src/main/scala/DispatcherFactoryBean.scala +++ b/akka-spring/src/main/scala/DispatcherFactoryBean.scala @@ -1,17 +1,17 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.springframework.beans.factory.config.AbstractFactoryBean -import se.scalablesolutions.akka.config.Supervision._ +import akka.config.Supervision._ import AkkaSpringConfigurationTags._ import reflect.BeanProperty -import se.scalablesolutions.akka.actor.ActorRef +import akka.actor.ActorRef import java.util.concurrent.RejectedExecutionHandler import java.util.concurrent.ThreadPoolExecutor.{DiscardPolicy, DiscardOldestPolicy, CallerRunsPolicy, AbortPolicy} -import se.scalablesolutions.akka.dispatch._ -import se.scalablesolutions.akka.util.Duration +import akka.dispatch._ +import akka.util.Duration /** * Reusable factory method for dispatchers. diff --git a/akka-spring/src/main/scala/DispatcherProperties.scala b/akka-spring/src/main/scala/DispatcherProperties.scala index 89d97670ca..b19309a1a8 100644 --- a/akka-spring/src/main/scala/DispatcherProperties.scala +++ b/akka-spring/src/main/scala/DispatcherProperties.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.springframework.beans.factory.support.BeanDefinitionBuilder diff --git a/akka-spring/src/main/scala/PropertyEntries.scala b/akka-spring/src/main/scala/PropertyEntries.scala index 9a7dc098de..9f6493bbb3 100644 --- a/akka-spring/src/main/scala/PropertyEntries.scala +++ b/akka-spring/src/main/scala/PropertyEntries.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.springframework.beans.factory.support.BeanDefinitionBuilder diff --git a/akka-spring/src/main/scala/StringReflect.scala b/akka-spring/src/main/scala/StringReflect.scala index c0c8aab9ff..2b77f8caa6 100644 --- a/akka-spring/src/main/scala/StringReflect.scala +++ b/akka-spring/src/main/scala/StringReflect.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring object StringReflect { diff --git a/akka-spring/src/main/scala/SupervisionBeanDefinitionParser.scala b/akka-spring/src/main/scala/SupervisionBeanDefinitionParser.scala index 8eb2d6ef37..c4753d1d5b 100644 --- a/akka-spring/src/main/scala/SupervisionBeanDefinitionParser.scala +++ b/akka-spring/src/main/scala/SupervisionBeanDefinitionParser.scala @@ -1,12 +1,12 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring -import se.scalablesolutions.akka.util.Logging +import akka.util.Logging import org.springframework.beans.factory.support.BeanDefinitionBuilder import org.springframework.beans.factory.xml.{ParserContext, AbstractSingleBeanDefinitionParser} -import se.scalablesolutions.akka.config.Supervision._ +import akka.config.Supervision._ import AkkaSpringConfigurationTags._ diff --git a/akka-spring/src/main/scala/SupervisionFactoryBean.scala b/akka-spring/src/main/scala/SupervisionFactoryBean.scala index b4559304b5..4fd842f755 100644 --- a/akka-spring/src/main/scala/SupervisionFactoryBean.scala +++ b/akka-spring/src/main/scala/SupervisionFactoryBean.scala @@ -1,14 +1,14 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.springframework.beans.factory.config.AbstractFactoryBean -import se.scalablesolutions.akka.config.Supervision._ -import se.scalablesolutions.akka.actor.{Supervisor, SupervisorFactory, Actor} +import akka.config.Supervision._ +import akka.actor.{Supervisor, SupervisorFactory, Actor} import AkkaSpringConfigurationTags._ import reflect.BeanProperty -import se.scalablesolutions.akka.config.{TypedActorConfigurator, RemoteAddress} +import akka.config.{TypedActorConfigurator, RemoteAddress} /** * Factory bean for supervisor configuration. diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/Pojo.java b/akka-spring/src/test/java/akka/spring/Pojo.java similarity index 92% rename from akka-spring/src/test/java/se/scalablesolutions/akka/spring/Pojo.java rename to akka-spring/src/test/java/akka/spring/Pojo.java index 6046f2bb5d..618adc8cc3 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/Pojo.java +++ b/akka-spring/src/test/java/akka/spring/Pojo.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.spring; +package akka.spring; import org.springframework.context.ApplicationContext; import org.springframework.context.ApplicationContextAware; @@ -6,7 +6,7 @@ import org.springframework.context.ApplicationContextAware; import javax.annotation.PreDestroy; import javax.annotation.PostConstruct; -import se.scalablesolutions.akka.actor.*; +import akka.actor.*; public class Pojo extends TypedActor implements PojoInf, ApplicationContextAware { diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/PojoInf.java b/akka-spring/src/test/java/akka/spring/PojoInf.java similarity index 86% rename from akka-spring/src/test/java/se/scalablesolutions/akka/spring/PojoInf.java rename to akka-spring/src/test/java/akka/spring/PojoInf.java index 0a313ceb18..f73ce35814 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/PojoInf.java +++ b/akka-spring/src/test/java/akka/spring/PojoInf.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.spring; +package akka.spring; import javax.annotation.PreDestroy; import javax.annotation.PostConstruct; diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleBean.java b/akka-spring/src/test/java/akka/spring/SampleBean.java similarity index 80% rename from akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleBean.java rename to akka-spring/src/test/java/akka/spring/SampleBean.java index 29e80d1c65..e23672d060 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleBean.java +++ b/akka-spring/src/test/java/akka/spring/SampleBean.java @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.spring; +package akka.spring; -import se.scalablesolutions.akka.actor.*; +import akka.actor.*; public class SampleBean extends TypedActor implements SampleBeanIntf { diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleBeanIntf.java b/akka-spring/src/test/java/akka/spring/SampleBeanIntf.java similarity index 70% rename from akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleBeanIntf.java rename to akka-spring/src/test/java/akka/spring/SampleBeanIntf.java index ec189ecd5f..365275f193 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleBeanIntf.java +++ b/akka-spring/src/test/java/akka/spring/SampleBeanIntf.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.spring; +package akka.spring; public interface SampleBeanIntf { public boolean down(); diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleRoute.java b/akka-spring/src/test/java/akka/spring/SampleRoute.java similarity index 84% rename from akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleRoute.java rename to akka-spring/src/test/java/akka/spring/SampleRoute.java index 5e7e5ea126..fb3565661d 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleRoute.java +++ b/akka-spring/src/test/java/akka/spring/SampleRoute.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.spring; +package akka.spring; import org.apache.camel.builder.RouteBuilder; diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/Bar.java b/akka-spring/src/test/java/akka/spring/foo/Bar.java similarity index 77% rename from akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/Bar.java rename to akka-spring/src/test/java/akka/spring/foo/Bar.java index bc86766c90..36276ff108 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/Bar.java +++ b/akka-spring/src/test/java/akka/spring/foo/Bar.java @@ -1,7 +1,7 @@ -package se.scalablesolutions.akka.spring.foo; +package akka.spring.foo; import java.io.IOException; -import se.scalablesolutions.akka.actor.*; +import akka.actor.*; public class Bar extends TypedActor implements IBar { diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/Foo.java b/akka-spring/src/test/java/akka/spring/foo/Foo.java similarity index 54% rename from akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/Foo.java rename to akka-spring/src/test/java/akka/spring/foo/Foo.java index 00e4b0df2e..189f146e51 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/Foo.java +++ b/akka-spring/src/test/java/akka/spring/foo/Foo.java @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.spring.foo; +package akka.spring.foo; -import se.scalablesolutions.akka.actor.*; +import akka.actor.*; public class Foo extends TypedActor implements IFoo{ diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IBar.java b/akka-spring/src/test/java/akka/spring/foo/IBar.java similarity index 54% rename from akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IBar.java rename to akka-spring/src/test/java/akka/spring/foo/IBar.java index 99bd7e3cf5..803b4ab50a 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IBar.java +++ b/akka-spring/src/test/java/akka/spring/foo/IBar.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.spring.foo; +package akka.spring.foo; public interface IBar { diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IFoo.java b/akka-spring/src/test/java/akka/spring/foo/IFoo.java similarity index 82% rename from akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IFoo.java rename to akka-spring/src/test/java/akka/spring/foo/IFoo.java index b7e6b622d5..e47809f3af 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IFoo.java +++ b/akka-spring/src/test/java/akka/spring/foo/IFoo.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.spring.foo; +package akka.spring.foo; /** * Created by IntelliJ IDEA. diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IMyPojo.java b/akka-spring/src/test/java/akka/spring/foo/IMyPojo.java similarity index 86% rename from akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IMyPojo.java rename to akka-spring/src/test/java/akka/spring/foo/IMyPojo.java index 5a2a272e6c..825d797cf2 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IMyPojo.java +++ b/akka-spring/src/test/java/akka/spring/foo/IMyPojo.java @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.spring.foo; +package akka.spring.foo; /** * Created by IntelliJ IDEA. diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/MyPojo.java b/akka-spring/src/test/java/akka/spring/foo/MyPojo.java similarity index 85% rename from akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/MyPojo.java rename to akka-spring/src/test/java/akka/spring/foo/MyPojo.java index 8f610eef63..54019f53d2 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/MyPojo.java +++ b/akka-spring/src/test/java/akka/spring/foo/MyPojo.java @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.spring.foo; +package akka.spring.foo; -import se.scalablesolutions.akka.actor.TypedActor; +import akka.actor.TypedActor; import java.util.concurrent.CountDownLatch; diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/PingActor.java b/akka-spring/src/test/java/akka/spring/foo/PingActor.java similarity index 91% rename from akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/PingActor.java rename to akka-spring/src/test/java/akka/spring/foo/PingActor.java index 3063a1b529..b60441699a 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/PingActor.java +++ b/akka-spring/src/test/java/akka/spring/foo/PingActor.java @@ -1,7 +1,7 @@ -package se.scalablesolutions.akka.spring.foo; +package akka.spring.foo; -import se.scalablesolutions.akka.actor.UntypedActor; -import se.scalablesolutions.akka.actor.ActorRef; +import akka.actor.UntypedActor; +import akka.actor.ActorRef; import org.springframework.context.ApplicationContext; import org.springframework.context.ApplicationContextAware; diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/PongActor.java b/akka-spring/src/test/java/akka/spring/foo/PongActor.java similarity index 80% rename from akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/PongActor.java rename to akka-spring/src/test/java/akka/spring/foo/PongActor.java index b67c0809fb..d4f19078a6 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/PongActor.java +++ b/akka-spring/src/test/java/akka/spring/foo/PongActor.java @@ -1,6 +1,6 @@ -package se.scalablesolutions.akka.spring.foo; +package akka.spring.foo; -import se.scalablesolutions.akka.actor.UntypedActor; +import akka.actor.UntypedActor; /** * test class diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/StatefulPojo.java b/akka-spring/src/test/java/akka/spring/foo/StatefulPojo.java similarity index 80% rename from akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/StatefulPojo.java rename to akka-spring/src/test/java/akka/spring/foo/StatefulPojo.java index ce85267edc..62f876a169 100644 --- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/StatefulPojo.java +++ b/akka-spring/src/test/java/akka/spring/foo/StatefulPojo.java @@ -1,11 +1,11 @@ -package se.scalablesolutions.akka.spring.foo; +package akka.spring.foo; -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; +import akka.stm.TransactionalMap; +import akka.stm.TransactionalVector; +import akka.stm.Ref; +import akka.actor.*; +import akka.stm.local.Atomic; public class StatefulPojo extends TypedActor { private TransactionalMap mapState; diff --git a/akka-spring/src/test/resources/appContext.xml b/akka-spring/src/test/resources/appContext.xml index 1b3b02d9fd..d000bd67f3 100644 --- a/akka-spring/src/test/resources/appContext.xml +++ b/akka-spring/src/test/resources/appContext.xml @@ -8,22 +8,22 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> @@ -31,7 +31,7 @@ diff --git a/akka-spring/src/test/resources/appContextCamelServiceCustom.xml b/akka-spring/src/test/resources/appContextCamelServiceCustom.xml index 32be55249d..c567d7ca32 100644 --- a/akka-spring/src/test/resources/appContextCamelServiceCustom.xml +++ b/akka-spring/src/test/resources/appContextCamelServiceCustom.xml @@ -10,7 +10,7 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd"> - + @@ -21,7 +21,7 @@ http://camel.apache.org/schema/spring/camel-spring.xsd"> diff --git a/akka-spring/src/test/resources/dispatcher-config.xml b/akka-spring/src/test/resources/dispatcher-config.xml index 728917c6c8..ffbf9dffc1 100644 --- a/akka-spring/src/test/resources/dispatcher-config.xml +++ b/akka-spring/src/test/resources/dispatcher-config.xml @@ -11,15 +11,15 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> @@ -74,15 +74,15 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> @@ -96,16 +96,16 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> - - - diff --git a/akka-spring/src/test/resources/property-config.xml b/akka-spring/src/test/resources/property-config.xml index c61255aa9b..b3f8adaa29 100644 --- a/akka-spring/src/test/resources/property-config.xml +++ b/akka-spring/src/test/resources/property-config.xml @@ -13,7 +13,7 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> diff --git a/akka-spring/src/test/resources/server-managed-config.xml b/akka-spring/src/test/resources/server-managed-config.xml index 128b16c8b6..652ff7bbd7 100644 --- a/akka-spring/src/test/resources/server-managed-config.xml +++ b/akka-spring/src/test/resources/server-managed-config.xml @@ -11,45 +11,45 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> + implementation="akka.spring.foo.PingActor"> + implementation="akka.spring.foo.PingActor"> + implementation="akka.spring.foo.PingActor"> diff --git a/akka-spring/src/test/resources/supervisor-config.xml b/akka-spring/src/test/resources/supervisor-config.xml index 0a395d2ce4..23917b4e50 100644 --- a/akka-spring/src/test/resources/supervisor-config.xml +++ b/akka-spring/src/test/resources/supervisor-config.xml @@ -17,16 +17,16 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> - - - @@ -41,10 +41,10 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> - - @@ -58,7 +58,7 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> - @@ -79,14 +79,14 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> - - @@ -101,8 +101,8 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> - - + + @@ -111,7 +111,7 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> - + diff --git a/akka-spring/src/test/resources/typed-actor-config.xml b/akka-spring/src/test/resources/typed-actor-config.xml index 989884e4fa..fedfcada27 100644 --- a/akka-spring/src/test/resources/typed-actor-config.xml +++ b/akka-spring/src/test/resources/typed-actor-config.xml @@ -10,39 +10,39 @@ http://www.akkasource.org/schema/akka http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> @@ -64,16 +64,16 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> - - - diff --git a/akka-spring/src/test/resources/untyped-actor-config.xml b/akka-spring/src/test/resources/untyped-actor-config.xml index bf6bb14baf..5bfb7d6de8 100644 --- a/akka-spring/src/test/resources/untyped-actor-config.xml +++ b/akka-spring/src/test/resources/untyped-actor-config.xml @@ -10,25 +10,25 @@ http://www.akkasource.org/schema/akka http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd"> + implementation="akka.spring.foo.PingActor"/> diff --git a/akka-spring/src/test/scala/ActorFactoryBeanTest.scala b/akka-spring/src/test/scala/ActorFactoryBeanTest.scala index 29cd48017b..e6e07ed5cd 100644 --- a/akka-spring/src/test/scala/ActorFactoryBeanTest.scala +++ b/akka-spring/src/test/scala/ActorFactoryBeanTest.scala @@ -1,10 +1,10 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring -import se.scalablesolutions.akka.actor.{ActorRegistry, ActorRef} -import se.scalablesolutions.akka.spring.foo.PingActor +import akka.actor.{ActorRegistry, ActorRef} +import akka.spring.foo.PingActor import org.junit.runner.RunWith import org.springframework.context.support.ClassPathXmlApplicationContext @@ -48,8 +48,8 @@ class ActorFactoryBeanTest extends Spec with ShouldMatchers with BeforeAndAfterA it("should create a proxy of type PojoInf") { val bean = new ActorFactoryBean() - bean.setInterface("se.scalablesolutions.akka.spring.PojoInf") - bean.setImplementation("se.scalablesolutions.akka.spring.Pojo") + bean.setInterface("akka.spring.PojoInf") + bean.setImplementation("akka.spring.Pojo") bean.timeoutStr = "1000" bean.typed = AkkaSpringConfigurationTags.TYPED_ACTOR_TAG val entries = new PropertyEntries() diff --git a/akka-spring/src/test/scala/CamelServiceSpringFeatureTest.scala b/akka-spring/src/test/scala/CamelServiceSpringFeatureTest.scala index 246ad88f37..d10cb60265 100644 --- a/akka-spring/src/test/scala/CamelServiceSpringFeatureTest.scala +++ b/akka-spring/src/test/scala/CamelServiceSpringFeatureTest.scala @@ -1,12 +1,12 @@ -package se.scalablesolutions.akka.spring +package akka.spring import org.apache.camel.impl.{SimpleRegistry, DefaultCamelContext} import org.apache.camel.spring.SpringCamelContext import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FeatureSpec} import org.springframework.context.support.ClassPathXmlApplicationContext -import se.scalablesolutions.akka.camel.CamelContextManager -import se.scalablesolutions.akka.actor.{TypedActor, ActorRegistry} +import akka.camel.CamelContextManager +import akka.actor.{TypedActor, ActorRegistry} class CamelServiceSpringFeatureTest extends FeatureSpec with BeforeAndAfterEach with BeforeAndAfterAll { override protected def beforeAll = { diff --git a/akka-spring/src/test/scala/ConfiggyPropertyPlaceholderConfigurerSpec.scala b/akka-spring/src/test/scala/ConfiggyPropertyPlaceholderConfigurerSpec.scala index 4f28b2fc8f..852ca88ca5 100644 --- a/akka-spring/src/test/scala/ConfiggyPropertyPlaceholderConfigurerSpec.scala +++ b/akka-spring/src/test/scala/ConfiggyPropertyPlaceholderConfigurerSpec.scala @@ -1,11 +1,11 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import foo.{IMyPojo, MyPojo, PingActor} -import se.scalablesolutions.akka.dispatch._ +import akka.dispatch._ import org.scalatest.FeatureSpec import org.scalatest.matchers.ShouldMatchers import org.scalatest.junit.JUnitRunner @@ -16,7 +16,7 @@ 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} +import akka.actor.{UntypedActor, Actor, ActorRef} diff --git a/akka-spring/src/test/scala/DispatcherBeanDefinitionParserTest.scala b/akka-spring/src/test/scala/DispatcherBeanDefinitionParserTest.scala index 85b233e034..ef6c0c23cc 100644 --- a/akka-spring/src/test/scala/DispatcherBeanDefinitionParserTest.scala +++ b/akka-spring/src/test/scala/DispatcherBeanDefinitionParserTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers diff --git a/akka-spring/src/test/scala/DispatcherFactoryBeanTest.scala b/akka-spring/src/test/scala/DispatcherFactoryBeanTest.scala index f4e9f640a4..486ec8820c 100644 --- a/akka-spring/src/test/scala/DispatcherFactoryBeanTest.scala +++ b/akka-spring/src/test/scala/DispatcherFactoryBeanTest.scala @@ -1,14 +1,14 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.config.Supervision._ -import se.scalablesolutions.akka.dispatch.MessageDispatcher +import akka.config.Supervision._ +import akka.dispatch.MessageDispatcher @RunWith(classOf[JUnitRunner]) class DispatcherFactoryBeanTest extends Spec with ShouldMatchers { diff --git a/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala b/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala index fac5358fc3..ac90495e9b 100644 --- a/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala +++ b/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala @@ -1,11 +1,11 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import foo.{IMyPojo, MyPojo, PingActor} -import se.scalablesolutions.akka.dispatch._ +import akka.dispatch._ import org.scalatest.FeatureSpec import org.scalatest.matchers.ShouldMatchers import org.scalatest.junit.JUnitRunner @@ -16,7 +16,7 @@ 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} +import akka.actor.{UntypedActor, Actor, ActorRef} /** * Tests for spring configuration of typed actors. @@ -119,7 +119,7 @@ class DispatcherSpringFeatureTest extends FeatureSpec with ShouldMatchers { scenario("get a thread-based-dispatcher for untyped from context") { val context = new ClassPathXmlApplicationContext("/dispatcher-config.xml") val actorRef = context.getBean("untyped-actor-with-thread-based-dispatcher").asInstanceOf[ActorRef] - assert(actorRef.getActorClassName() === "se.scalablesolutions.akka.spring.foo.PingActor") + assert(actorRef.getActorClassName() === "akka.spring.foo.PingActor") actorRef.start() actorRef.sendOneWay("Hello") assert(actorRef.getDispatcher.isInstanceOf[ThreadBasedDispatcher]) diff --git a/akka-spring/src/test/scala/ScalaDom.scala b/akka-spring/src/test/scala/ScalaDom.scala index 0f347e89df..9319b0c328 100644 --- a/akka-spring/src/test/scala/ScalaDom.scala +++ b/akka-spring/src/test/scala/ScalaDom.scala @@ -1,4 +1,4 @@ -package se.scalablesolutions.akka.spring +package akka.spring /** * from http://stackoverflow.com/questions/2002685/any-conversion-from-scalas-xml-to-w3c-dom */ diff --git a/akka-spring/src/test/scala/SupervisionBeanDefinitionParserTest.scala b/akka-spring/src/test/scala/SupervisionBeanDefinitionParserTest.scala index 85c28b0c0e..f033930ce1 100644 --- a/akka-spring/src/test/scala/SupervisionBeanDefinitionParserTest.scala +++ b/akka-spring/src/test/scala/SupervisionBeanDefinitionParserTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -11,7 +11,7 @@ import ScalaDom._ import org.w3c.dom.Element import org.springframework.beans.factory.support.BeanDefinitionBuilder -import se.scalablesolutions.akka.config.Supervision. {FaultHandlingStrategy, AllForOneStrategy} +import akka.config.Supervision. {FaultHandlingStrategy, AllForOneStrategy} /** * Test for SupervisionBeanDefinitionParser diff --git a/akka-spring/src/test/scala/SupervisionFactoryBeanTest.scala b/akka-spring/src/test/scala/SupervisionFactoryBeanTest.scala index bb2fbb0b27..542b8a1377 100644 --- a/akka-spring/src/test/scala/SupervisionFactoryBeanTest.scala +++ b/akka-spring/src/test/scala/SupervisionFactoryBeanTest.scala @@ -1,14 +1,14 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.config.Supervision._ -import se.scalablesolutions.akka.config.TypedActorConfigurator +import akka.config.Supervision._ +import akka.config.TypedActorConfigurator private[akka] class Foo @@ -16,7 +16,7 @@ private[akka] class Foo class SupervisionFactoryBeanTest extends Spec with ShouldMatchers { val faultHandlingStrategy = new AllForOneStrategy(List(classOf[Exception]), 3, 1000) - val typedActors = List(createTypedActorProperties("se.scalablesolutions.akka.spring.Foo", "1000")) + val typedActors = List(createTypedActorProperties("akka.spring.Foo", "1000")) private def createTypedActorProperties(target: String, timeout: String) : ActorProperties = { val properties = new ActorProperties() diff --git a/akka-spring/src/test/scala/SupervisorSpringFeatureTest.scala b/akka-spring/src/test/scala/SupervisorSpringFeatureTest.scala index 89a779039c..2ce629ed38 100644 --- a/akka-spring/src/test/scala/SupervisorSpringFeatureTest.scala +++ b/akka-spring/src/test/scala/SupervisorSpringFeatureTest.scala @@ -1,13 +1,13 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring -import se.scalablesolutions.akka.spring.foo.{IMyPojo, MyPojo, IFoo, IBar} -import se.scalablesolutions.akka.dispatch._ -import se.scalablesolutions.akka.config.TypedActorConfigurator -import se.scalablesolutions.akka.actor.Supervisor +import akka.spring.foo.{IMyPojo, MyPojo, IFoo, IBar} +import akka.dispatch._ +import akka.config.TypedActorConfigurator +import akka.actor.Supervisor import org.scalatest.FeatureSpec import org.scalatest.matchers.ShouldMatchers diff --git a/akka-spring/src/test/scala/TypedActorBeanDefinitionParserTest.scala b/akka-spring/src/test/scala/TypedActorBeanDefinitionParserTest.scala index 15ed97bd27..97766341a4 100644 --- a/akka-spring/src/test/scala/TypedActorBeanDefinitionParserTest.scala +++ b/akka-spring/src/test/scala/TypedActorBeanDefinitionParserTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers @@ -48,7 +48,7 @@ class TypedActorBeanDefinitionParserTest extends Spec with ShouldMatchers { } it("should parse TypedActors configuration with dispatcher") { - val xml = @@ -58,7 +58,7 @@ class TypedActorBeanDefinitionParserTest extends Spec with ShouldMatchers { } it("should parse remote TypedActors configuration") { - val xml = @@ -70,7 +70,7 @@ class TypedActorBeanDefinitionParserTest extends Spec with ShouldMatchers { } it("should parse remote server managed TypedActors configuration") { - val xml = diff --git a/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala b/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala index 3cdcd17cb0..0f8a316653 100644 --- a/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala +++ b/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala @@ -1,11 +1,11 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import foo.{PingActor, IMyPojo, MyPojo} -import se.scalablesolutions.akka.dispatch.FutureTimeoutException +import akka.dispatch.FutureTimeoutException import org.scalatest.matchers.ShouldMatchers import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith @@ -15,10 +15,10 @@ 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 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 +import akka.actor.{TypedActor, RemoteTypedActorOne, Actor} +import akka.actor.remote.RemoteTypedActorOneImpl /** * Tests for spring configuration of typed actors. diff --git a/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala b/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala index 0397d30bf0..c2bbe7364d 100644 --- a/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala +++ b/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala @@ -1,20 +1,20 @@ /** * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.spring +package akka.spring import foo.PingActor -import se.scalablesolutions.akka.dispatch.ExecutorBasedEventDrivenWorkStealingDispatcher +import akka.dispatch.ExecutorBasedEventDrivenWorkStealingDispatcher import org.scalatest.matchers.ShouldMatchers import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith import org.springframework.context.support.ClassPathXmlApplicationContext -import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer} +import akka.remote.{RemoteClient, RemoteServer} import org.scalatest.{BeforeAndAfterAll, FeatureSpec} import java.util.concurrent.CountDownLatch -import se.scalablesolutions.akka.actor.{RemoteActorRef, ActorRegistry, Actor, ActorRef} +import akka.actor.{RemoteActorRef, ActorRegistry, Actor, ActorRef} /** * Tests for spring configuration of typed actors. @@ -52,7 +52,7 @@ class UntypedActorSpringFeatureTest extends FeatureSpec with ShouldMatchers with 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") + assert(pingActor.getActorClassName() === "akka.spring.foo.PingActor") pingActor.start() } @@ -114,7 +114,7 @@ class UntypedActorSpringFeatureTest extends FeatureSpec with ShouldMatchers with 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) + val actorRef = RemoteClient.actorFor("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") @@ -135,7 +135,7 @@ class UntypedActorSpringFeatureTest extends FeatureSpec with ShouldMatchers with 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") + assert(pingActor.getActorClassName() === "akka.spring.foo.PingActor") pingActor.start() val nrOfActors = ActorRegistry.actors.length // get client actor ref from spring context diff --git a/akka-typed-actor/src/main/java/se/scalablesolutions/akka/config/DependencyBinding.java b/akka-typed-actor/src/main/java/akka/config/DependencyBinding.java similarity index 91% rename from akka-typed-actor/src/main/java/se/scalablesolutions/akka/config/DependencyBinding.java rename to akka-typed-actor/src/main/java/akka/config/DependencyBinding.java index 2956e6860f..207e11cda7 100644 --- a/akka-typed-actor/src/main/java/se/scalablesolutions/akka/config/DependencyBinding.java +++ b/akka-typed-actor/src/main/java/akka/config/DependencyBinding.java @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.config; +package akka.config; /** * @author Jonas Bonér diff --git a/akka-typed-actor/src/main/java/se/scalablesolutions/akka/config/TypedActorGuiceModule.java b/akka-typed-actor/src/main/java/akka/config/TypedActorGuiceModule.java similarity index 95% rename from akka-typed-actor/src/main/java/se/scalablesolutions/akka/config/TypedActorGuiceModule.java rename to akka-typed-actor/src/main/java/akka/config/TypedActorGuiceModule.java index 0c2ed11402..fbeafce9f6 100644 --- a/akka-typed-actor/src/main/java/se/scalablesolutions/akka/config/TypedActorGuiceModule.java +++ b/akka-typed-actor/src/main/java/akka/config/TypedActorGuiceModule.java @@ -2,7 +2,7 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.config; +package akka.config; import java.util.List; diff --git a/akka-typed-actor/src/main/resources/META-INF/aop.xml b/akka-typed-actor/src/main/resources/META-INF/aop.xml index be133a51b8..85882c299d 100644 --- a/akka-typed-actor/src/main/resources/META-INF/aop.xml +++ b/akka-typed-actor/src/main/resources/META-INF/aop.xml @@ -1,6 +1,6 @@ - + diff --git a/akka-typed-actor/src/main/scala/actor/TypedActor.scala b/akka-typed-actor/src/main/scala/actor/TypedActor.scala index 762557d5bd..ba9115c22c 100644 --- a/akka-typed-actor/src/main/scala/actor/TypedActor.scala +++ b/akka-typed-actor/src/main/scala/actor/TypedActor.scala @@ -2,12 +2,12 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.actor +package akka.actor import Actor._ -import se.scalablesolutions.akka.dispatch.{MessageDispatcher, Future, CompletableFuture, Dispatchers} -import se.scalablesolutions.akka.config.Supervision._ -import se.scalablesolutions.akka.util._ +import akka.dispatch.{MessageDispatcher, Future, CompletableFuture, Dispatchers} +import akka.config.Supervision._ +import akka.util._ import ReflectiveAccess._ import org.codehaus.aspectwerkz.joinpoint.{MethodRtti, JoinPoint} @@ -24,7 +24,7 @@ import java.lang.reflect.{Method, Field, InvocationHandler, Proxy => JProxy} * Non-void methods are turned into request-reply messages with the exception of methods returning * a 'Future' which will be sent using request-reply-with-future semantics and need to return the * result using the 'future(..)' method: 'return future(... future result ...);'. - * Methods returning se.scalablesolutions.akka.japi.Option will block until a timeout expires, + * Methods returning akka.japi.Option will block until a timeout expires, * if the implementation of the method returns "none", some(null) will be returned, "none" will only be * returned when the method didn't respond within the timeout. * @@ -149,7 +149,7 @@ abstract class TypedActor extends Actor with Proxyable { /** * This method is used to resolve the Future for TypedActor methods that are defined to return a - * {@link se.scalablesolutions.akka.actor.dispatch.Future }. + * {@link akka.actor.dispatch.Future }. *

* Here is an example: *

@@ -735,7 +735,7 @@ object TypedActor extends Logging {
     classOf[Future[_]].isAssignableFrom(methodRtti.getMethod.getReturnType)
 
   private[akka] def returnsOption_?(methodRtti: MethodRtti): Boolean =
-    classOf[se.scalablesolutions.akka.japi.Option[_]].isAssignableFrom(methodRtti.getMethod.getReturnType)
+    classOf[akka.japi.Option[_]].isAssignableFrom(methodRtti.getMethod.getReturnType)
 
   private[akka] def supervise(faultHandlingStrategy: FaultHandlingStrategy, components: List[Supervise]): Supervisor =
     Supervisor(SupervisorConfig(faultHandlingStrategy, components))
@@ -759,7 +759,7 @@ object TypedActor extends Logging {
 @Aspect("perInstance")
 private[akka] sealed class ServerManagedTypedActorAspect extends ActorAspect {
   
-  @Around("execution(* *.*(..)) && this(se.scalablesolutions.akka.actor.ServerManagedTypedActor)")
+  @Around("execution(* *.*(..)) && this(akka.actor.ServerManagedTypedActor)")
   def invoke(joinPoint: JoinPoint): AnyRef = {
     if (!isInitialized) initialize(joinPoint)
     remoteDispatch(joinPoint)
@@ -782,7 +782,7 @@ private[akka] sealed class ServerManagedTypedActorAspect extends ActorAspect {
 @Aspect("perInstance")
 private[akka] sealed class TypedActorAspect extends ActorAspect {
 
-  @Around("execution(* *.*(..)) && !this(se.scalablesolutions.akka.actor.ServerManagedTypedActor)")
+  @Around("execution(* *.*(..)) && !this(akka.actor.ServerManagedTypedActor)")
   def invoke(joinPoint: JoinPoint): AnyRef = {
     if (!isInitialized) initialize(joinPoint)
     dispatch(joinPoint)
@@ -825,7 +825,7 @@ private[akka] abstract class ActorAspect {
     } else if (TypedActor.returnsFuture_?(methodRtti)) {
       actorRef.!!!(joinPoint, timeout)(senderActorRef)
     } else if (TypedActor.returnsOption_?(methodRtti)) {
-        import se.scalablesolutions.akka.japi.{Option => JOption}
+        import akka.japi.{Option => JOption}
       (actorRef.!!(joinPoint, timeout)(senderActorRef)).as[JOption[AnyRef]] match {
         case None => JOption.none[AnyRef]
         case Some(x) if ((x eq null) || x.isEmpty) => JOption.some[AnyRef](null)
diff --git a/akka-typed-actor/src/main/scala/config/TypedActorConfigurator.scala b/akka-typed-actor/src/main/scala/config/TypedActorConfigurator.scala
index f23cb6b8ec..e44db8d2c4 100644
--- a/akka-typed-actor/src/main/scala/config/TypedActorConfigurator.scala
+++ b/akka-typed-actor/src/main/scala/config/TypedActorConfigurator.scala
@@ -2,7 +2,7 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.config
+package akka.config
 
 import Supervision._
 
diff --git a/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala b/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala
index 38378f5e8b..0f7532ea0f 100644
--- a/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala
+++ b/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala
@@ -2,11 +2,11 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.config
+package akka.config
 
-import se.scalablesolutions.akka.actor._
-import se.scalablesolutions.akka.config.Supervision._
-import se.scalablesolutions.akka.util._
+import akka.actor._
+import akka.config.Supervision._
+import akka.util._
 import ReflectiveAccess._
 
 import org.codehaus.aspectwerkz.proxy.Proxy
@@ -19,7 +19,7 @@ import java.lang.reflect.Method
 import com.google.inject._
 
 /**
- * This is an class for internal usage. Instead use the se.scalablesolutions.akka.config.TypedActorConfigurator
+ * This is an class for internal usage. Instead use the akka.config.TypedActorConfigurator
  * class for creating TypedActors.
  *
  * @author Jonas Bonér
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Bar.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Bar.java
index 906476b789..3ac5c51961 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Bar.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Bar.java
@@ -1,4 +1,4 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
 public interface Bar {
   void bar(String msg);
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/BarImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/BarImpl.java
index 9cb41a85cf..e52a34a09d 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/BarImpl.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/BarImpl.java
@@ -1,7 +1,7 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
 import com.google.inject.Inject;
-import se.scalablesolutions.akka.actor.*;
+import akka.actor.*;
 
 public class BarImpl extends TypedActor implements Bar {
   @Inject
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Ext.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Ext.java
index c37219cf00..cb0d5ed279 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Ext.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Ext.java
@@ -1,4 +1,4 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
 public interface Ext {
   void ext();
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/ExtImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/ExtImpl.java
index dd8ca55089..2180e9f440 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/ExtImpl.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/ExtImpl.java
@@ -1,4 +1,4 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
 public class ExtImpl implements Ext {
   public void ext() {
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Foo.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Foo.java
index a64f975bce..12b2ed2fbf 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Foo.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Foo.java
@@ -1,4 +1,4 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
 public interface Foo {
   public Foo body();
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/FooImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/FooImpl.java
index ded09f4e07..d717c8cb01 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/FooImpl.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/FooImpl.java
@@ -1,7 +1,7 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
 import com.google.inject.Inject;
-import se.scalablesolutions.akka.actor.*;
+import akka.actor.*;
 
 public class FooImpl extends TypedActor implements Foo {
   @Inject
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActor.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActor.java
index fbd241763f..a257204daf 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActor.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActor.java
@@ -1,4 +1,4 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
 public interface NestedTransactionalTypedActor {
   public String getMapState(String key);
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 cb002b0a9e..bbc7ae9306 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
@@ -1,7 +1,7 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
-import se.scalablesolutions.akka.actor.*;
-import se.scalablesolutions.akka.stm.*;
+import akka.actor.*;
+import akka.stm.*;
 
 public class NestedTransactionalTypedActorImpl extends TypedTransactor implements NestedTransactionalTypedActor {
   private TransactionalMap mapState;
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojo.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojo.java
index 5d06afdc9c..743a189bf6 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojo.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojo.java
@@ -1,4 +1,4 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
 import java.util.concurrent.CountDownLatch;
 
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 1e567014d9..093904e5e1 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
@@ -1,6 +1,6 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
-import se.scalablesolutions.akka.actor.*;
+import akka.actor.*;
 
 import java.util.concurrent.CountDownLatch;
 
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojo.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojo.java
index f4aafa6e1d..699ccdb355 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojo.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojo.java
@@ -1,9 +1,9 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
-import se.scalablesolutions.akka.dispatch.Future;
-import se.scalablesolutions.akka.dispatch.CompletableFuture;
-import se.scalablesolutions.akka.dispatch.Future;
-import se.scalablesolutions.akka.japi.Option;
+import akka.dispatch.Future;
+import akka.dispatch.CompletableFuture;
+import akka.dispatch.Future;
+import akka.japi.Option;
 
 public interface SimpleJavaPojo {
   public Object getSender();
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCaller.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCaller.java
index e35702846f..f390c16364 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCaller.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCaller.java
@@ -1,6 +1,6 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
-import se.scalablesolutions.akka.dispatch.CompletableFuture;
+import akka.dispatch.CompletableFuture;
 
 public interface SimpleJavaPojoCaller {
   public void setPojo(SimpleJavaPojo pojo);
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCallerImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCallerImpl.java
index 760b69f8b9..9d7b4034ae 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCallerImpl.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCallerImpl.java
@@ -1,7 +1,7 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
-import se.scalablesolutions.akka.actor.*;
-import se.scalablesolutions.akka.dispatch.Future;
+import akka.actor.*;
+import akka.dispatch.Future;
 
 public class SimpleJavaPojoCallerImpl extends TypedActor implements SimpleJavaPojoCaller {
 
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoImpl.java
index 103d84de2d..7da66009ca 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoImpl.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoImpl.java
@@ -1,9 +1,9 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
-import se.scalablesolutions.akka.actor.*;
-import se.scalablesolutions.akka.dispatch.Future;
-import se.scalablesolutions.akka.dispatch.CompletableFuture;
-import se.scalablesolutions.akka.japi.Option;
+import akka.actor.*;
+import akka.dispatch.Future;
+import akka.dispatch.CompletableFuture;
+import akka.japi.Option;
 
 public class SimpleJavaPojoImpl extends TypedActor implements SimpleJavaPojo {
 
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActor.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActor.java
index 6e7c43745b..4c7b262772 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActor.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActor.java
@@ -1,4 +1,4 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
 public interface TransactionalTypedActor {
   public String getMapState(String key);
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 45bda4a675..599bd272a6 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
@@ -1,9 +1,9 @@
-package se.scalablesolutions.akka.actor;
+package 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;
+import akka.actor.*;
+import akka.stm.*;
+import akka.stm.local.*;
+import akka.stm.local.Atomic;
 
 public class TransactionalTypedActorImpl extends TypedTransactor implements TransactionalTypedActor {
   private TransactionalMap mapState;
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailer.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailer.java
index e0b1e72c33..7722a6214c 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailer.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailer.java
@@ -1,4 +1,4 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
 public interface TypedActorFailer extends java.io.Serializable {
   public int fail();
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailerImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailerImpl.java
index 89a97330df..5a1a4d1e01 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailerImpl.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailerImpl.java
@@ -1,6 +1,6 @@
-package se.scalablesolutions.akka.actor;
+package akka.actor;
 
-import se.scalablesolutions.akka.actor.*;
+import akka.actor.*;
 
 public class TypedActorFailerImpl extends TypedActor implements TypedActorFailer {
   public int fail() {
diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/NestedTransactionalTypedActorSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/NestedTransactionalTypedActorSpec.scala
index 7338e8df41..7111e60980 100644
--- a/akka-typed-actor/src/test/scala/actor/typed-actor/NestedTransactionalTypedActorSpec.scala
+++ b/akka-typed-actor/src/test/scala/actor/typed-actor/NestedTransactionalTypedActorSpec.scala
@@ -2,7 +2,7 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.actor
+package akka.actor
 
 import org.scalatest.Spec
 import org.scalatest.Assertions
@@ -11,7 +11,7 @@ import org.scalatest.BeforeAndAfterAll
 import org.scalatest.junit.JUnitRunner
 import org.junit.runner.RunWith
 
-import se.scalablesolutions.akka.actor._
+import akka.actor._
 
 @RunWith(classOf[JUnitRunner])
 class NestedTransactionalTypedActorSpec extends
diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/RestartNestedTransactionalTypedActorSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/RestartNestedTransactionalTypedActorSpec.scala
index 63c8856075..d35ec3c280 100644
--- a/akka-typed-actor/src/test/scala/actor/typed-actor/RestartNestedTransactionalTypedActorSpec.scala
+++ b/akka-typed-actor/src/test/scala/actor/typed-actor/RestartNestedTransactionalTypedActorSpec.scala
@@ -2,7 +2,7 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.actor
+package akka.actor
 
 import org.scalatest.Spec
 import org.scalatest.Assertions
@@ -11,11 +11,11 @@ import org.scalatest.BeforeAndAfterAll
 import org.scalatest.junit.JUnitRunner
 import org.junit.runner.RunWith
 
-import se.scalablesolutions.akka.config.Config
-import se.scalablesolutions.akka.config._
-import se.scalablesolutions.akka.config.TypedActorConfigurator
-import se.scalablesolutions.akka.config.Supervision._
-import se.scalablesolutions.akka.actor._
+import akka.config.Config
+import akka.config._
+import akka.config.TypedActorConfigurator
+import akka.config.Supervision._
+import akka.actor._
 
 @RunWith(classOf[JUnitRunner])
 class RestartNestedTransactionalTypedActorSpec extends
diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/RestartTransactionalTypedActorSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/RestartTransactionalTypedActorSpec.scala
index 968379a003..a06975c291 100644
--- a/akka-typed-actor/src/test/scala/actor/typed-actor/RestartTransactionalTypedActorSpec.scala
+++ b/akka-typed-actor/src/test/scala/actor/typed-actor/RestartTransactionalTypedActorSpec.scala
@@ -2,7 +2,7 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.actor
+package akka.actor
 
 import org.scalatest.Spec
 import org.scalatest.Assertions
@@ -11,9 +11,9 @@ import org.scalatest.BeforeAndAfterAll
 import org.scalatest.junit.JUnitRunner
 import org.junit.runner.RunWith
 
-import se.scalablesolutions.akka.config.Supervision._
-import se.scalablesolutions.akka.actor._
-import se.scalablesolutions.akka.config.{Config, TypedActorConfigurator}
+import akka.config.Supervision._
+import akka.actor._
+import akka.config.{Config, TypedActorConfigurator}
 
 @RunWith(classOf[JUnitRunner])
 class RestartTransactionalTypedActorSpec extends
diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/TransactionalTypedActorSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TransactionalTypedActorSpec.scala
index b55f52c875..7f1c23ed3c 100644
--- a/akka-typed-actor/src/test/scala/actor/typed-actor/TransactionalTypedActorSpec.scala
+++ b/akka-typed-actor/src/test/scala/actor/typed-actor/TransactionalTypedActorSpec.scala
@@ -2,7 +2,7 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.actor
+package akka.actor
 
 import org.scalatest.Spec
 import org.scalatest.Assertions
@@ -11,7 +11,7 @@ import org.scalatest.BeforeAndAfterAll
 import org.scalatest.junit.JUnitRunner
 import org.junit.runner.RunWith
 
-import se.scalablesolutions.akka.actor._
+import akka.actor._
 
 @RunWith(classOf[JUnitRunner])
 class TransactionalTypedActorSpec extends
diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorContextSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorContextSpec.scala
index adc0879c84..7a34454a24 100644
--- a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorContextSpec.scala
+++ b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorContextSpec.scala
@@ -2,7 +2,7 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.actor
+package akka.actor
 
 import org.scalatest.Spec
 import org.scalatest.Assertions
@@ -11,7 +11,7 @@ import org.scalatest.BeforeAndAfterAll
 import org.scalatest.junit.JUnitRunner
 import org.junit.runner.RunWith
 
-import se.scalablesolutions.akka.dispatch.DefaultCompletableFuture;
+import akka.dispatch.DefaultCompletableFuture;
 
 @RunWith(classOf[JUnitRunner])
 class TypedActorContextSpec extends
diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorGuiceConfiguratorSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorGuiceConfiguratorSpec.scala
index 5d2e9fd315..aca3d0b998 100644
--- a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorGuiceConfiguratorSpec.scala
+++ b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorGuiceConfiguratorSpec.scala
@@ -2,7 +2,7 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.actor
+package akka.actor
 
 import com.google.inject.AbstractModule
 import com.google.inject.Scopes
@@ -14,10 +14,10 @@ import org.scalatest.BeforeAndAfterAll
 import org.scalatest.junit.JUnitRunner
 import org.junit.runner.RunWith
 
-import se.scalablesolutions.akka.config.Supervision._
-import se.scalablesolutions.akka.dispatch._
-import se.scalablesolutions.akka.dispatch.FutureTimeoutException
-import se.scalablesolutions.akka.config.{Config, TypedActorConfigurator}
+import akka.config.Supervision._
+import akka.dispatch._
+import akka.dispatch.FutureTimeoutException
+import akka.config.{Config, TypedActorConfigurator}
 
 @RunWith(classOf[JUnitRunner])
 class TypedActorGuiceConfiguratorSpec extends
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 7bc521fe73..a8dd7a75ad 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
@@ -1,16 +1,16 @@
-package se.scalablesolutions.akka.actor
+package akka.actor
 
 import org.junit.runner.RunWith
 import org.scalatest.{BeforeAndAfterAll, Spec}
 import org.scalatest.junit.JUnitRunner
 import org.scalatest.matchers.ShouldMatchers
 
-import se.scalablesolutions.akka.actor.TypedActor._
+import akka.actor.TypedActor._
 
-import se.scalablesolutions.akka.config.Supervision._
+import akka.config.Supervision._
 
 import java.util.concurrent.CountDownLatch
-import se.scalablesolutions.akka.config.TypedActorConfigurator
+import akka.config.TypedActorConfigurator
 
 /**
  * @author Martin Krasser
diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorSpec.scala
index 219f96f3e2..9a9eb795f6 100644
--- a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorSpec.scala
+++ b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorSpec.scala
@@ -2,7 +2,7 @@
  * Copyright (C) 2009-2010 Scalable Solutions AB 
  */
 
-package se.scalablesolutions.akka.actor
+package akka.actor
 
 import org.scalatest.Spec
 import org.scalatest.Assertions
@@ -10,9 +10,9 @@ import org.scalatest.matchers.ShouldMatchers
 import org.scalatest.BeforeAndAfterEach
 import org.scalatest.junit.JUnitRunner
 import org.junit.runner.RunWith
-import se.scalablesolutions.akka.japi.Option;
+import akka.japi.Option;
 
-import se.scalablesolutions.akka.dispatch.DefaultCompletableFuture
+import akka.dispatch.DefaultCompletableFuture
 import TypedActorSpec._
 
 
diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorUtilFunctionsSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorUtilFunctionsSpec.scala
index 48424f3c17..d2243e92e6 100644
--- a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorUtilFunctionsSpec.scala
+++ b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorUtilFunctionsSpec.scala
@@ -1,4 +1,4 @@
-package se.scalablesolutions.akka.actor
+package akka.actor
 
 import org.scalatest.Suite
 import org.junit.runner.RunWith
diff --git a/config/akka-reference.conf b/config/akka-reference.conf
index 4305b27e3b..b218045c70 100644
--- a/config/akka-reference.conf
+++ b/config/akka-reference.conf
@@ -88,7 +88,7 @@ akka {
     hostname = "localhost"
     port = 9998
     #cometSupport = "org.atmosphere.container.Jetty7CometSupport" # Disregard autodetection, for valid values: http://doc.akkasource.org/comet
-    filters = ["se.scalablesolutions.akka.security.AkkaSecurityFilterFactory"] # List with all jersey filters to use
+    filters = ["akka.security.AkkaSecurityFilterFactory"] # List with all jersey filters to use
     resource_packages = ["sample.rest.scala",
                          "sample.rest.java",
                          "sample.security"]  # List with all resource packages for your Jersey services
@@ -149,7 +149,7 @@ akka {
     cluster {
       service = on
       name = "default"                                                        # The name of the cluster
-      serializer = "se.scalablesolutions.akka.serialization.Serializer$Java$" # FQN of the serializer class
+      serializer = "akka.serialization.Serializer$Java$" # FQN of the serializer class
     }
   }
 
diff --git a/config/cassandra-akka-storage-conf.xml b/config/cassandra-akka-storage-conf.xml
index 2647fdcd0e..f87d2eedf5 100644
--- a/config/cassandra-akka-storage-conf.xml
+++ b/config/cassandra-akka-storage-conf.xml
@@ -101,15 +101,18 @@
        ~ your data set!
       -->
       
-      
       
-      
+        
+