From 3ecb38b5b7201f0657a8b129157f2a90ee71a87d Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 22 Oct 2010 17:50:48 +0200 Subject: [PATCH 01/39] WIP --- .../src/main/scala/dispatch/Dispatchers.scala | 48 +-- .../ExecutorBasedEventDrivenDispatcher.scala | 54 ++- ...sedEventDrivenWorkStealingDispatcher.scala | 13 +- .../main/scala/dispatch/MessageHandling.scala | 1 + .../dispatch/ThreadBasedDispatcher.scala | 6 +- .../scala/dispatch/ThreadPoolBuilder.scala | 314 ++++++++++++------ akka-actor/src/main/scala/util/LockUtil.scala | 8 +- .../main/scala/DispatcherFactoryBean.scala | 90 +++-- 8 files changed, 322 insertions(+), 212 deletions(-) diff --git a/akka-actor/src/main/scala/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/dispatch/Dispatchers.scala index 55e819a2c8..4a9c3c0917 100644 --- a/akka-actor/src/main/scala/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/dispatch/Dispatchers.scala @@ -52,7 +52,7 @@ object Dispatchers extends Logging { val MAILBOX_PUSH_TIME_OUT = Duration(config.getInt("akka.actor.default-dispatcher.mailbox-push-timeout-time", 10), TIME_UNIT) val THROUGHPUT_DEADLINE_TIME = Duration(config.getInt("akka.actor.throughput-deadline-time",-1), TIME_UNIT) val THROUGHPUT_DEADLINE_TIME_MILLIS = THROUGHPUT_DEADLINE_TIME.toMillis.toInt - val MAILBOX_TYPE = if (MAILBOX_CAPACITY < 0) UnboundedMailbox() else BoundedMailbox() + val MAILBOX_TYPE: MailboxType = if (MAILBOX_CAPACITY < 0) UnboundedMailbox() else BoundedMailbox() lazy val defaultGlobalDispatcher = { config.getConfigMap("akka.actor.default-dispatcher").flatMap(from).getOrElse(globalExecutorBasedEventDrivenDispatcher) @@ -60,13 +60,7 @@ object Dispatchers extends Logging { object globalHawtDispatcher extends HawtDispatcher - object globalExecutorBasedEventDrivenDispatcher extends ExecutorBasedEventDrivenDispatcher( - "global", THROUGHPUT, THROUGHPUT_DEADLINE_TIME_MILLIS, MAILBOX_TYPE) { - override def register(actor: ActorRef) = { - if (isShutdown) init - super.register(actor) - } - } + object globalExecutorBasedEventDrivenDispatcher extends ExecutorBasedEventDrivenDispatcher("global", THROUGHPUT, THROUGHPUT_DEADLINE_TIME_MILLIS, MAILBOX_TYPE) /** * Creates an event-driven dispatcher based on the excellent HawtDispatch library. @@ -172,23 +166,27 @@ object Dispatchers extends Logging { def from(cfg: ConfigMap): Option[MessageDispatcher] = { lazy val name = cfg.getString("name", newUuid.toString) - def threadPoolConfig(b: ThreadPoolBuilder) { - b.configureIfPossible( builder => { - cfg.getInt("keep-alive-time").foreach(time => builder.setKeepAliveTimeInMillis(Duration(time, TIME_UNIT).toMillis.toInt)) - cfg.getDouble("core-pool-size-factor").foreach(builder.setCorePoolSizeFromFactor(_)) - cfg.getDouble("max-pool-size-factor").foreach(builder.setMaxPoolSizeFromFactor(_)) - cfg.getInt("executor-bounds").foreach(builder.setExecutorBounds(_)) - cfg.getBool("allow-core-timeout").foreach(builder.setAllowCoreThreadTimeout(_)) - cfg.getInt("mailbox-capacity").foreach(builder.setMailboxCapacity(_)) + def configureThreadPool(createDispatcher: => (ThreadPoolConfig) => MessageDispatcher): ThreadPoolConfigDispatcherBuilder = { + val builder = ThreadPoolConfigDispatcherBuilder(createDispatcher,ThreadPoolConfig()) //Create a new builder + //Creates a transformation from builder to builder, if the option isDefined + def conf_?[T](opt: Option[T])(fun: (T) => ThreadPoolConfigDispatcherBuilder => ThreadPoolConfigDispatcherBuilder): + Option[(ThreadPoolConfigDispatcherBuilder) => ThreadPoolConfigDispatcherBuilder] = opt map fun - cfg.getString("rejection-policy").map({ + //Apply the following options to the config if they are present in the cfg + List( + conf_?(cfg getInt "keep-alive-time" )(time => _.setKeepAliveTime(Duration(time, TIME_UNIT))), + conf_?(cfg getDouble "core-pool-size-factor")(factor => _.setCorePoolSizeFromFactor(factor)), + conf_?(cfg getDouble "max-pool-size-factor" )(factor => _.setMaxPoolSizeFromFactor(factor)), + conf_?(cfg getInt "executor-bounds" )(bounds => _.setExecutorBounds(bounds)), + conf_?(cfg getBool "allow-core-timeout" )(allow => _.setAllowCoreThreadTimeout(allow)), + conf_?(cfg getString "rejection-policy" map { case "abort" => new AbortPolicy() case "caller-runs" => new CallerRunsPolicy() case "discard-oldest" => new DiscardOldestPolicy() case "discard" => new DiscardPolicy() - case x => throw new IllegalArgumentException("[%s] is not a valid rejectionPolicy!" format x) - }).foreach(builder.setRejectionPolicy(_)) - }) + case x => throw new IllegalArgumentException("[%s] is not a valid rejectionPolicy!" format x) + })(policy => _.setRejectionPolicy(policy)) + ).foldLeft(builder)( (c,f) => f.map( _(c) ).getOrElse(c)) //Returns the builder with all the specified options set } lazy val mailboxType: MailboxType = { @@ -200,15 +198,17 @@ object Dispatchers extends Logging { cfg.getString("type") map { case "ExecutorBasedEventDriven" => - new ExecutorBasedEventDrivenDispatcher( + configureThreadPool(threadPoolConfig => new ExecutorBasedEventDrivenDispatcher( name, cfg.getInt("throughput", THROUGHPUT), cfg.getInt("throughput-deadline", THROUGHPUT_DEADLINE_TIME_MILLIS), mailboxType, - threadPoolConfig) + threadPoolConfig)).build - case "ExecutorBasedEventDrivenWorkStealing" => new ExecutorBasedEventDrivenWorkStealingDispatcher(name, mailboxType, threadPoolConfig) - case "Hawt" => new HawtDispatcher(cfg.getBool("aggregate").getOrElse(true)) + case "ExecutorBasedEventDrivenWorkStealing" => + configureThreadPool(poolCfg => new ExecutorBasedEventDrivenWorkStealingDispatcher(name, mailboxType,poolCfg)).build + + case "Hawt" => new HawtDispatcher(cfg.getBool("aggregate",true)) case "GlobalExecutorBasedEventDriven" => globalExecutorBasedEventDrivenDispatcher case "GlobalHawt" => globalHawtDispatcher case unknown => throw new IllegalArgumentException("Unknown dispatcher type [%s]" format unknown) diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala index 213e68b863..4e4a1f9c59 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala @@ -8,8 +8,9 @@ import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException} import se.scalablesolutions.akka.util.ReflectiveAccess.EnterpriseModule import java.util.Queue -import java.util.concurrent.{RejectedExecutionException, ConcurrentLinkedQueue, LinkedBlockingQueue} import se.scalablesolutions.akka.util.Switch +import java.util.concurrent.atomic.AtomicReference +import java.util.concurrent. {ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue, LinkedBlockingQueue} /** * Default settings are: @@ -69,11 +70,11 @@ class ExecutorBasedEventDrivenDispatcher( val throughput: Int = Dispatchers.THROUGHPUT, val throughputDeadlineTime: Int = Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS, _mailboxType: MailboxType = Dispatchers.MAILBOX_TYPE, - config: (ThreadPoolBuilder) => Unit = _ => ()) - extends MessageDispatcher with ThreadPoolBuilder { + val config: ThreadPoolConfig = ThreadPoolConfig()) + extends MessageDispatcher { def this(_name: String, throughput: Int, throughputDeadlineTime: Int, mailboxType: MailboxType) = - this(_name, throughput, throughputDeadlineTime, mailboxType, _ => ()) // Needed for Java API usage + this(_name, throughput, throughputDeadlineTime, mailboxType,ThreadPoolConfig()) // Needed for Java API usage def this(_name: String, throughput: Int, mailboxType: MailboxType) = this(_name, throughput, Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS, mailboxType) // Needed for Java API usage @@ -84,14 +85,12 @@ class ExecutorBasedEventDrivenDispatcher( def this(_name: String) = this(_name, Dispatchers.THROUGHPUT, Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS, Dispatchers.MAILBOX_TYPE) // Needed for Java API usage + val name = "akka:event-driven:dispatcher:" + _name val mailboxType = Some(_mailboxType) private[akka] val active = new Switch(false) - - val name = "akka:event-driven:dispatcher:" + _name - - //Initialize - init + private[akka] val threadFactory = new MonitorableThreadFactory(name) + private[akka] val executorService = new AtomicReference[ExecutorService](null) def dispatch(invocation: MessageInvocation) = { val mbox = getMailbox(invocation.receiver) @@ -99,6 +98,8 @@ class ExecutorBasedEventDrivenDispatcher( registerForExecution(mbox) } + def isShutdown = active.isOff + /** * @return the mailbox associated with the actor */ @@ -112,8 +113,7 @@ class ExecutorBasedEventDrivenDispatcher( } case BoundedMailbox(blocking, capacity, pushTimeOut) => - val cap = if (mailboxCapacity == -1) capacity else mailboxCapacity - new DefaultBoundedMessageQueue(cap, pushTimeOut, blocking) with ExecutableMailbox { + new DefaultBoundedMessageQueue(capacity, pushTimeOut, blocking) with ExecutableMailbox { def dispatcher = ExecutorBasedEventDrivenDispatcher.this } } @@ -131,25 +131,28 @@ class ExecutorBasedEventDrivenDispatcher( case JMSBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("JMSBasedDurableMailbox is not yet supported") } - def start = active switchOn { + def start: Unit = if (active.isOff) active switchOn { log.debug("Starting up %s\n\twith throughput [%d]", toString, throughput) + if (executorService.get() eq null) { + val newExecutor = config.createExecutorService(threadFactory) + if (!executorService.compareAndSet(null,newExecutor)) + log.error("Thought the ExecutorService was missing but appeared out of nowhere!") + } } - def shutdown = active switchOff { - log.debug("Shutting down %s", toString) - executor.shutdownNow - uuids.clear - } - - def ensureNotActive(): Unit = if (active.isOn) { - throw new IllegalActorStateException( - "Can't build a new thread pool for a dispatcher that is already up and running") + def shutdown: Unit = if (active.isOn) active switchOff { + val old = executorService.getAndSet(null) + if (old ne null) { + log.debug("Shutting down %s", toString) + old.shutdownNow() + uuids.clear + } } private[akka] def registerForExecution(mbox: MessageQueue with ExecutableMailbox): Unit = if (active.isOn) { if (mbox.suspended.isOff && mbox.dispatcherLock.tryLock()) { try { - executor execute mbox + executorService.get() execute mbox } catch { case e: RejectedExecutionException => mbox.dispatcherLock.unlock() @@ -171,13 +174,6 @@ class ExecutorBasedEventDrivenDispatcher( mbox.suspended.switchOff registerForExecution(mbox) } - - // FIXME: should we have an unbounded queue and not bounded as default ???? - private[akka] def init { - withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity - config(this) - buildThreadPool - } } /** diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala index 2bcc7c489f..f44bd60c50 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala @@ -33,11 +33,11 @@ import se.scalablesolutions.akka.util.Switch class ExecutorBasedEventDrivenWorkStealingDispatcher( _name: String, _mailboxType: MailboxType = Dispatchers.MAILBOX_TYPE, - config: (ThreadPoolBuilder) => Unit = _ => ()) extends MessageDispatcher with ThreadPoolBuilder { + config: ThreadPoolConfig = ThreadPoolConfig()) extends MessageDispatcher with ThreadPoolBuilder { - def this(_name: String, mailboxType: MailboxType) = this(_name, mailboxType, _ => ()) + def this(_name: String, mailboxType: MailboxType) = this(_name, mailboxType,ThreadPoolConfig()) - def this(_name: String) = this(_name, Dispatchers.MAILBOX_TYPE, _ => ()) + def this(_name: String) = this(_name, Dispatchers.MAILBOX_TYPE,ThreadPoolConfig()) val mailboxType = Some(_mailboxType) @@ -54,7 +54,6 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( @volatile private var lastThiefIndex = 0 val name = "akka:event-driven-work-stealing:dispatcher:" + _name - init /** * @return the mailbox associated with the actor @@ -195,12 +194,6 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( override val toString = "ExecutorBasedEventDrivenWorkStealingDispatcher[" + name + "]" - private[akka] def init = { - withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity - config(this) - buildThreadPool - } - def createTransientMailbox(actorRef: ActorRef, mailboxType: TransientMailboxType): AnyRef = mailboxType match { case UnboundedMailbox(blocking) => // FIXME make use of 'blocking' in work stealer ConcurrentLinkedDeque new ConcurrentLinkedDeque[MessageInvocation] with MessageQueue with Runnable { diff --git a/akka-actor/src/main/scala/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/dispatch/MessageHandling.scala index 86e4552151..88d978b311 100644 --- a/akka-actor/src/main/scala/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/dispatch/MessageHandling.scala @@ -69,6 +69,7 @@ trait MessageDispatcher extends MailboxFactory with Logging { def shutdown: Unit def register(actorRef: ActorRef) { + start if (actorRef.mailbox eq null) actorRef.mailbox = createMailbox(actorRef) uuids add actorRef.uuid } diff --git a/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala b/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala index 8c2229c6b3..247399818f 100644 --- a/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala @@ -37,10 +37,6 @@ class ThreadBasedDispatcher(private val actor: ActorRef, _mailboxType: MailboxTy } object ThreadBasedDispatcher { - val oneThread: (ThreadPoolBuilder) => Unit = b => { - b setCorePoolSize 1 - b setMaxPoolSize 1 - b setAllowCoreThreadTimeout true - } + val oneThread: ThreadPoolConfig = ThreadPoolConfig(allowCorePoolTimeout = true, corePoolSize = 1, maxPoolSize = 1) } diff --git a/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala index 1e5e257349..e65d47796e 100644 --- a/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala @@ -10,7 +10,131 @@ import atomic.{AtomicLong, AtomicInteger} import ThreadPoolExecutor.CallerRunsPolicy import se.scalablesolutions.akka.actor.IllegalActorStateException -import se.scalablesolutions.akka.util.{Logger, Logging} +import se.scalablesolutions.akka.util. {Duration, Logger, Logging} + +object ThreadPoolConfig { + type Bounds = Int + type FlowHandler = Either[RejectedExecutionHandler,Bounds] + type QueueFactory = () => BlockingQueue[Runnable] + + val defaultAllowCoreThreadTimeout: Boolean = false + val defaultCorePoolSize: Int = 16 + val defaultMaxPoolSize: Int = 128 + val defaultTimeout: Duration = Duration(60000L,TimeUnit.MILLISECONDS) + def defaultFlowHandler: FlowHandler = flowHandler(new CallerRunsPolicy) + + def flowHandler(rejectionHandler: RejectedExecutionHandler): FlowHandler = Left(rejectionHandler) + def flowHandler(bounds: Int): FlowHandler = Right(bounds) + + def fixedPoolSize(size: Int): Int = size + def scaledPoolSize(multiplier: Double): Int = + (Runtime.getRuntime.availableProcessors * multiplier).ceil.toInt + + def arrayBlockingQueue(capacity: Int, fair: Boolean): QueueFactory = + () => new ArrayBlockingQueue[Runnable](capacity,fair) + + def synchronousQueue(fair: Boolean): QueueFactory = + () => new SynchronousQueue[Runnable](fair) + + def linkedBlockingQueue(): QueueFactory = + () => new LinkedBlockingQueue[Runnable]() + + def linkedBlockingQueue(capacity: Int): QueueFactory = + () => new LinkedBlockingQueue[Runnable](capacity) + + def reusableQueue(queue: BlockingQueue[Runnable]): QueueFactory = + () => queue + + def reusableQueue(queueFactory: QueueFactory): QueueFactory = { + val queue = queueFactory() + () => queue + } +} + +case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout, + corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize, + maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize, + threadTimeout: Duration = ThreadPoolConfig.defaultTimeout, + flowHandler: ThreadPoolConfig.FlowHandler = ThreadPoolConfig.defaultFlowHandler, + queueFactory: ThreadPoolConfig.QueueFactory = ThreadPoolConfig.linkedBlockingQueue()) { + + final def createExecutorService(threadFactory: ThreadFactory): ExecutorService = { + flowHandler match { + case Left(rejectHandler) => + val service = new ThreadPoolExecutor(corePoolSize, maxPoolSize, threadTimeout.length, threadTimeout.unit, queueFactory(), threadFactory, rejectHandler) + service.allowCoreThreadTimeOut(allowCorePoolTimeout) + service + case Right(bounds) => + val service = new ThreadPoolExecutor(corePoolSize, maxPoolSize, threadTimeout.length, threadTimeout.unit, queueFactory(), threadFactory) + service.allowCoreThreadTimeOut(allowCorePoolTimeout) + new BoundedExecutorDecorator(service,bounds) + } + } +} + +trait DispatcherBuilder { + def build: MessageDispatcher +} + +case class ThreadPoolConfigDispatcherBuilder(dispatcherFactory: (ThreadPoolConfig) => MessageDispatcher, config: ThreadPoolConfig) extends DispatcherBuilder { + import ThreadPoolConfig._ + def build = dispatcherFactory(config) + + //TODO remove this, for backwards compat only + def buildThreadPool = build + + def withNewBoundedThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity(bounds: Int): ThreadPoolConfigDispatcherBuilder = + this.copy(config = config.copy(flowHandler = flowHandler(bounds), queueFactory = linkedBlockingQueue())) + + def withNewThreadPoolWithCustomBlockingQueue(newQueueFactory: QueueFactory): ThreadPoolConfigDispatcherBuilder = + this.copy(config = config.copy(flowHandler = defaultFlowHandler, queueFactory = newQueueFactory)) + + def withNewThreadPoolWithCustomBlockingQueue(queue: BlockingQueue[Runnable]): ThreadPoolConfigDispatcherBuilder = + withNewThreadPoolWithCustomBlockingQueue(reusableQueue(queue)) + + def withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity: ThreadPoolConfigDispatcherBuilder = + this.copy(config = config.copy(queueFactory = linkedBlockingQueue(), flowHandler = defaultFlowHandler)) + + def withNewThreadPoolWithLinkedBlockingQueueWithCapacity(capacity: Int): ThreadPoolConfigDispatcherBuilder = + this.copy(config = config.copy(queueFactory = linkedBlockingQueue(capacity), flowHandler = defaultFlowHandler)) + + def withNewThreadPoolWithSynchronousQueueWithFairness(fair: Boolean): ThreadPoolConfigDispatcherBuilder = + this.copy(config = config.copy(queueFactory = synchronousQueue(fair), flowHandler = defaultFlowHandler)) + + def withNewThreadPoolWithArrayBlockingQueueWithCapacityAndFairness(capacity: Int, fair: Boolean): ThreadPoolConfigDispatcherBuilder = + this.copy(config = config.copy(queueFactory = arrayBlockingQueue(capacity,fair), flowHandler = defaultFlowHandler)) + + def setCorePoolSize(size: Int): ThreadPoolConfigDispatcherBuilder = + this.copy(config = config.copy(corePoolSize = size)) + + def setMaxPoolSize(size: Int): ThreadPoolConfigDispatcherBuilder = + this.copy(config = config.copy(maxPoolSize = size)) + + def setCorePoolSizeFromFactor(multiplier: Double): ThreadPoolConfigDispatcherBuilder = + setCorePoolSize(scaledPoolSize(multiplier)) + + def setMaxPoolSizeFromFactor(multiplier: Double): ThreadPoolConfigDispatcherBuilder = + setMaxPoolSize(scaledPoolSize(multiplier)) + + def setExecutorBounds(bounds: Int): ThreadPoolConfigDispatcherBuilder = + this.copy(config = config.copy(flowHandler = flowHandler(bounds))) + + def setKeepAliveTimeInMillis(time: Long): ThreadPoolConfigDispatcherBuilder = + setKeepAliveTime(Duration(time,TimeUnit.MILLISECONDS)) + + def setKeepAliveTime(time: Duration): ThreadPoolConfigDispatcherBuilder = + this.copy(config = config.copy(threadTimeout = time)) + + def setRejectionPolicy(policy: RejectedExecutionHandler): ThreadPoolConfigDispatcherBuilder = + setFlowHandler(flowHandler(policy)) + + def setFlowHandler(newFlowHandler: FlowHandler): ThreadPoolConfigDispatcherBuilder = + this.copy(config = config.copy(flowHandler = newFlowHandler)) + + def setAllowCoreThreadTimeout(allow: Boolean): ThreadPoolConfigDispatcherBuilder = + this.copy(config = config.copy(allowCorePoolTimeout = allow)) +} + trait ThreadPoolBuilder extends Logging { val name: String @@ -26,13 +150,13 @@ trait ThreadPoolBuilder extends Logging { @volatile private var inProcessOfBuilding = false private var blockingQueue: BlockingQueue[Runnable] = _ - private lazy val threadFactory = new MonitorableThreadFactory(name) + protected lazy val threadFactory = new MonitorableThreadFactory(name) - protected var executor: ExecutorService = _ + @volatile var executor: ExecutorService = _ def isShutdown = executor.isShutdown - def buildThreadPool(): Unit = synchronized { + def buildThreadPool(): ExecutorService = synchronized { ensureNotActive inProcessOfBuilding = false @@ -51,6 +175,7 @@ trait ThreadPoolBuilder extends Logging { } else { executor = threadPoolBuilder } + executor } def withNewThreadPoolWithCustomBlockingQueue(queue: BlockingQueue[Runnable]): ThreadPoolBuilder = synchronized { @@ -204,104 +329,105 @@ trait ThreadPoolBuilder extends Logging { } def ensureNotActive(): Unit +} - /** - * @author Jonas Bonér - */ - class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extends ExecutorService with Logging { - protected val semaphore = new Semaphore(bound) - def execute(command: Runnable) = { - semaphore.acquire - try { - executor.execute(new Runnable() { - def run = { - try { - command.run - } finally { - semaphore.release - } - } - }) - } catch { - case e: RejectedExecutionException => - semaphore.release - case e => - log.error(e,"Unexpected exception") - throw e - } - } +/** + * @author Jonas Bonér + */ +class MonitorableThreadFactory(val name: String) extends ThreadFactory { + protected val counter = new AtomicLong - // Delegating methods for the ExecutorService interface - def shutdown = executor.shutdown + def newThread(runnable: Runnable) = + new MonitorableThread(runnable, name) +// new Thread(runnable, name + "-" + counter.getAndIncrement) +} - def shutdownNow = executor.shutdownNow +/** + * @author Jonas Bonér + */ +object MonitorableThread { + val DEFAULT_NAME = "MonitorableThread" + val created = new AtomicInteger + val alive = new AtomicInteger + @volatile var debugLifecycle = false +} - def isShutdown = executor.isShutdown +// FIXME fix the issues with using the monitoring in MonitorableThread - def isTerminated = executor.isTerminated +/** + * @author Jonas Bonér + */ +class MonitorableThread(runnable: Runnable, name: String) + extends Thread(runnable, name + "-" + MonitorableThread.created.incrementAndGet) with Logging { - def awaitTermination(l: Long, timeUnit: TimeUnit) = executor.awaitTermination(l, timeUnit) + setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { + def uncaughtException(thread: Thread, cause: Throwable) = + log.error(cause, "UNCAUGHT in thread [%s]", thread.getName) + }) - def submit[T](callable: Callable[T]) = executor.submit(callable) - - def submit[T](runnable: Runnable, t: T) = executor.submit(runnable, t) - - def submit(runnable: Runnable) = executor.submit(runnable) - - def invokeAll[T](callables: Collection[_ <: Callable[T]]) = executor.invokeAll(callables) - - def invokeAll[T](callables: Collection[_ <: Callable[T]], l: Long, timeUnit: TimeUnit) = executor.invokeAll(callables, l, timeUnit) - - def invokeAny[T](callables: Collection[_ <: Callable[T]]) = executor.invokeAny(callables) - - def invokeAny[T](callables: Collection[_ <: Callable[T]], l: Long, timeUnit: TimeUnit) = executor.invokeAny(callables, l, timeUnit) - } - - /** - * @author Jonas Bonér - */ - class MonitorableThreadFactory(val name: String) extends ThreadFactory { - protected val counter = new AtomicLong - - def newThread(runnable: Runnable) = - new MonitorableThread(runnable, name) - // new Thread(runnable, name + "-" + counter.getAndIncrement) - } - - /** - * @author Jonas Bonér - */ - object MonitorableThread { - val DEFAULT_NAME = "MonitorableThread" - val created = new AtomicInteger - val alive = new AtomicInteger - @volatile var debugLifecycle = false - } - - // FIXME fix the issues with using the monitoring in MonitorableThread - - /** - * @author Jonas Bonér - */ - class MonitorableThread(runnable: Runnable, name: String) - extends Thread(runnable, name + "-" + MonitorableThread.created.incrementAndGet) with Logging { - - setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { - def uncaughtException(thread: Thread, cause: Throwable) = - log.error(cause, "UNCAUGHT in thread [%s]", thread.getName) - }) - - override def run = { - val debug = MonitorableThread.debugLifecycle - log.debug("Created thread %s", getName) - try { - MonitorableThread.alive.incrementAndGet - super.run - } finally { - MonitorableThread.alive.decrementAndGet - log.debug("Exiting thread %s", getName) - } + override def run = { + val debug = MonitorableThread.debugLifecycle + log.debug("Created thread %s", getName) + try { + MonitorableThread.alive.incrementAndGet + super.run + } finally { + MonitorableThread.alive.decrementAndGet + log.debug("Exiting thread %s", getName) } } } + +/** + * @author Jonas Bonér + */ +class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extends ExecutorService with Logging { + protected val semaphore = new Semaphore(bound) + + def execute(command: Runnable) = { + semaphore.acquire + try { + executor.execute(new Runnable() { + def run = { + try { + command.run + } finally { + semaphore.release + } + } + }) + } catch { + case e: RejectedExecutionException => + semaphore.release + case e => + log.error(e,"Unexpected exception") + throw e + } + } + + // Delegating methods for the ExecutorService interface + def shutdown = executor.shutdown + + def shutdownNow = executor.shutdownNow + + def isShutdown = executor.isShutdown + + def isTerminated = executor.isTerminated + + def awaitTermination(l: Long, timeUnit: TimeUnit) = executor.awaitTermination(l, timeUnit) + + def submit[T](callable: Callable[T]) = executor.submit(callable) + + def submit[T](runnable: Runnable, t: T) = executor.submit(runnable, t) + + def submit(runnable: Runnable) = executor.submit(runnable) + + def invokeAll[T](callables: Collection[_ <: Callable[T]]) = executor.invokeAll(callables) + + def invokeAll[T](callables: Collection[_ <: Callable[T]], l: Long, timeUnit: TimeUnit) = executor.invokeAll(callables, l, timeUnit) + + def invokeAny[T](callables: Collection[_ <: Callable[T]]) = executor.invokeAny(callables) + + def invokeAny[T](callables: Collection[_ <: Callable[T]], l: Long, timeUnit: TimeUnit) = executor.invokeAny(callables, l, timeUnit) +} diff --git a/akka-actor/src/main/scala/util/LockUtil.scala b/akka-actor/src/main/scala/util/LockUtil.scala index 7c37267157..909713194b 100644 --- a/akka-actor/src/main/scala/util/LockUtil.scala +++ b/akka-actor/src/main/scala/util/LockUtil.scala @@ -5,7 +5,7 @@ package se.scalablesolutions.akka.util import java.util.concurrent.locks.{ReentrantReadWriteLock, ReentrantLock} -import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.atomic. {AtomicBoolean} /** * @author Jonas Bonér @@ -119,7 +119,7 @@ class SimpleLock { class Switch(startAsOn: Boolean = false) { private val switch = new AtomicBoolean(startAsOn) - protected def transcend(from: Boolean,action: => Unit): Boolean = { + protected def transcend(from: Boolean,action: => Unit): Boolean = synchronized { if (switch.compareAndSet(from,!from)) { try { action @@ -135,8 +135,8 @@ class Switch(startAsOn: Boolean = false) { def switchOff(action: => Unit): Boolean = transcend(from = true, action) def switchOn(action: => Unit): Boolean = transcend(from = false,action) - def switchOff: Boolean = switch.compareAndSet(true,false) - def switchOn: Boolean = switch.compareAndSet(false,true) + def switchOff: Boolean = synchronized { switch.compareAndSet(true,false) } + def switchOn: Boolean = synchronized { switch.compareAndSet(false,true) } def ifOnYield[T](action: => T): Option[T] = { if (switch.get) diff --git a/akka-spring/src/main/scala/DispatcherFactoryBean.scala b/akka-spring/src/main/scala/DispatcherFactoryBean.scala index 893b44e24d..d273e75011 100644 --- a/akka-spring/src/main/scala/DispatcherFactoryBean.scala +++ b/akka-spring/src/main/scala/DispatcherFactoryBean.scala @@ -8,9 +8,9 @@ import se.scalablesolutions.akka.config.Supervision._ import AkkaSpringConfigurationTags._ import reflect.BeanProperty import se.scalablesolutions.akka.actor.ActorRef -import se.scalablesolutions.akka.dispatch.{ThreadPoolBuilder, Dispatchers, MessageDispatcher} import java.util.concurrent.RejectedExecutionHandler import java.util.concurrent.ThreadPoolExecutor.{DiscardPolicy, DiscardOldestPolicy, CallerRunsPolicy, AbortPolicy} +import se.scalablesolutions.akka.dispatch._ /** * Reusable factory method for dispatchers. @@ -23,56 +23,54 @@ object DispatcherFactoryBean { * @param actorRef actorRef needed for thread based dispatcher */ def createNewInstance(properties: DispatcherProperties, actorRef: Option[ActorRef] = None): MessageDispatcher = { + + def configThreadPool(): ThreadPoolConfig = { + val poolCfg = ThreadPoolConfig() + if ((properties.threadPool ne null) && (properties.threadPool.queue ne null)) { + properties.threadPool.queue match { + case VAL_BOUNDED_ARRAY_BLOCKING_QUEUE => threadPoolBuilder.withNewThreadPoolWithArrayBlockingQueueWithCapacityAndFairness(properties.threadPool.capacity, properties.threadPool.fairness) + case VAL_UNBOUNDED_LINKED_BLOCKING_QUEUE if properties.threadPool.capacity > -1 => threadPoolBuilder.withNewThreadPoolWithLinkedBlockingQueueWithCapacity(properties.threadPool.capacity) + case VAL_UNBOUNDED_LINKED_BLOCKING_QUEUE if properties.threadPool.capacity <= 0 => threadPoolBuilder.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity + case VAL_BOUNDED_LINKED_BLOCKING_QUEUE => threadPoolBuilder.withNewBoundedThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity(properties.threadPool.bound) + case VAL_SYNCHRONOUS_QUEUE => threadPoolBuilder.withNewThreadPoolWithSynchronousQueueWithFairness(properties.threadPool.fairness) + case _ => throw new IllegalArgumentException("unknown queue type") + } + + if (properties.threadPool.corePoolSize > -1) + threadPoolBuilder.setCorePoolSize(properties.threadPool.corePoolSize) + + if (properties.threadPool.maxPoolSize > -1) + threadPoolBuilder.setMaxPoolSize(properties.threadPool.maxPoolSize) + + if (properties.threadPool.keepAlive > -1) + threadPoolBuilder.setKeepAliveTimeInMillis(properties.threadPool.keepAlive) + + if (properties.threadPool.mailboxCapacity > -1) + threadPoolBuilder.setMailboxCapacity(properties.threadPool.mailboxCapacity) + + if ((properties.threadPool.rejectionPolicy ne null) && (!properties.threadPool.rejectionPolicy.isEmpty)) { + val policy: RejectedExecutionHandler = properties.threadPool.rejectionPolicy match { + case "abort-policy" => new AbortPolicy() + case "caller-runs-policy" => new CallerRunsPolicy() + case "discard-oldest-policy" => new DiscardOldestPolicy() + case "discard-policy" => new DiscardPolicy() + case _ => throw new IllegalArgumentException("Unknown rejection-policy '" + properties.threadPool.rejectionPolicy + "'") + } + threadPoolBuilder.setRejectionPolicy(policy) + } + } else poolCfg + } + var dispatcher = properties.dispatcherType match { - case EXECUTOR_BASED_EVENT_DRIVEN => Dispatchers.newExecutorBasedEventDrivenDispatcher(properties.name) + case EXECUTOR_BASED_EVENT_DRIVEN => new ExecutorBasedEventDrivenDispatcher(properties.name, config = configThreadPool) case EXECUTOR_BASED_EVENT_DRIVEN_WORK_STEALING => Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher(properties.name) - case THREAD_BASED => if (!actorRef.isDefined) { - throw new IllegalArgumentException("Need an ActorRef to create a thread based dispatcher.") - } else { - Dispatchers.newThreadBasedDispatcher(actorRef.get) - } + case THREAD_BASED if actorRef.isEmpty => throw new IllegalArgumentException("Need an ActorRef to create a thread based dispatcher.") + case THREAD_BASED if actorRef.isDefined => Dispatchers.newThreadBasedDispatcher(actorRef.get) case HAWT => Dispatchers.newHawtDispatcher(properties.aggregate) case _ => throw new IllegalArgumentException("unknown dispatcher type") } - // build threadpool - if ((properties.threadPool ne null) && (properties.threadPool.queue ne null)) { - var threadPoolBuilder = dispatcher.asInstanceOf[ThreadPoolBuilder] - threadPoolBuilder = properties.threadPool.queue match { - case VAL_BOUNDED_ARRAY_BLOCKING_QUEUE => threadPoolBuilder.withNewThreadPoolWithArrayBlockingQueueWithCapacityAndFairness(properties.threadPool.capacity, properties.threadPool.fairness) - case VAL_UNBOUNDED_LINKED_BLOCKING_QUEUE => if (properties.threadPool.capacity > -1) - threadPoolBuilder.withNewThreadPoolWithLinkedBlockingQueueWithCapacity(properties.threadPool.capacity) - else - threadPoolBuilder.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity - case VAL_BOUNDED_LINKED_BLOCKING_QUEUE => threadPoolBuilder.withNewBoundedThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity(properties.threadPool.bound) - case VAL_SYNCHRONOUS_QUEUE => threadPoolBuilder.withNewThreadPoolWithSynchronousQueueWithFairness(properties.threadPool.fairness) - case _ => throw new IllegalArgumentException("unknown queue type") - } - if (properties.threadPool.corePoolSize > -1) { - threadPoolBuilder.setCorePoolSize(properties.threadPool.corePoolSize) - } - if (properties.threadPool.maxPoolSize > -1) { - threadPoolBuilder.setMaxPoolSize(properties.threadPool.maxPoolSize) - } - if (properties.threadPool.keepAlive > -1) { - threadPoolBuilder.setKeepAliveTimeInMillis(properties.threadPool.keepAlive) - } - if (properties.threadPool.mailboxCapacity > -1) { - threadPoolBuilder.setMailboxCapacity(properties.threadPool.mailboxCapacity) - } - if ((properties.threadPool.rejectionPolicy ne null) && (!properties.threadPool.rejectionPolicy.isEmpty)) { - val policy: RejectedExecutionHandler = properties.threadPool.rejectionPolicy match { - case "abort-policy" => new AbortPolicy() - case "caller-runs-policy" => new CallerRunsPolicy() - case "discard-oldest-policy" => new DiscardOldestPolicy() - case "discard-policy" => new DiscardPolicy() - case _ => throw new IllegalArgumentException("Unknown rejection-policy '" + properties.threadPool.rejectionPolicy + "'") - } - threadPoolBuilder.setRejectionPolicy(policy) - } - threadPoolBuilder.asInstanceOf[MessageDispatcher] - } else { - dispatcher - } + + dispatcher } } From c241703a019d994e7b1e1a0d76d0c965b47374f6 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sun, 24 Oct 2010 00:36:56 +0200 Subject: [PATCH 02/39] Tests green, config basically in place, need to work on start/stop semantics and countdowns --- .../src/main/scala/dispatch/Dispatchers.scala | 30 +-- .../ExecutorBasedEventDrivenDispatcher.scala | 12 +- ...sedEventDrivenWorkStealingDispatcher.scala | 46 ++-- .../scala/dispatch/ThreadPoolBuilder.scala | 235 +++--------------- .../actor/supervisor/SupervisorMiscSpec.scala | 2 +- ...rBasedEventDrivenDispatcherActorSpec.scala | 19 +- ...ventDrivenWorkStealingDispatcherSpec.scala | 6 +- .../main/scala/DispatcherFactoryBean.scala | 98 ++++---- .../scala/DispatcherSpringFeatureTest.scala | 19 +- .../TypedActorGuiceConfiguratorSpec.scala | 2 +- 10 files changed, 161 insertions(+), 308 deletions(-) diff --git a/akka-actor/src/main/scala/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/dispatch/Dispatchers.scala index 4a9c3c0917..d7592d49b7 100644 --- a/akka-actor/src/main/scala/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/dispatch/Dispatchers.scala @@ -100,7 +100,8 @@ object Dispatchers extends Logging { *

* Has a fluent builder interface for configuring its semantics. */ - def newExecutorBasedEventDrivenDispatcher(name: String) = new ExecutorBasedEventDrivenDispatcher(name) + def newExecutorBasedEventDrivenDispatcher(name: String) = + ThreadPoolConfigDispatcherBuilder(config => new ExecutorBasedEventDrivenDispatcher(name,config),ThreadPoolConfig()) /** * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool. @@ -108,7 +109,8 @@ object Dispatchers extends Logging { * Has a fluent builder interface for configuring its semantics. */ def newExecutorBasedEventDrivenDispatcher(name: String, throughput: Int, mailboxType: MailboxType) = - new ExecutorBasedEventDrivenDispatcher(name, throughput, mailboxType) + ThreadPoolConfigDispatcherBuilder(config => + new ExecutorBasedEventDrivenDispatcher(name, throughput, THROUGHPUT_DEADLINE_TIME_MILLIS, mailboxType, config),ThreadPoolConfig()) /** @@ -116,30 +118,32 @@ object Dispatchers extends Logging { *

* Has a fluent builder interface for configuring its semantics. */ - def newExecutorBasedEventDrivenDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) = - new ExecutorBasedEventDrivenDispatcher(name, throughput, throughputDeadlineMs, mailboxType) + def newExecutorBasedEventDrivenDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) = + ThreadPoolConfigDispatcherBuilder(config => + new ExecutorBasedEventDrivenDispatcher(name, throughput, throughputDeadlineMs, mailboxType, config),ThreadPoolConfig()) /** * Creates a executor-based event-driven dispatcher with work stealing (TODO: better doc) serving multiple (millions) of actors through a thread pool. *

* Has a fluent builder interface for configuring its semantics. */ - def newExecutorBasedEventDrivenWorkStealingDispatcher(name: String) = new ExecutorBasedEventDrivenWorkStealingDispatcher(name) + def newExecutorBasedEventDrivenWorkStealingDispatcher(name: String): ThreadPoolConfigDispatcherBuilder = + newExecutorBasedEventDrivenWorkStealingDispatcher(name,MAILBOX_TYPE) /** * Creates a executor-based event-driven dispatcher with work stealing (TODO: better doc) serving multiple (millions) of actors through a thread pool. *

* Has a fluent builder interface for configuring its semantics. */ - def newExecutorBasedEventDrivenWorkStealingDispatcher(name: String, mailboxType: MailboxType) = - new ExecutorBasedEventDrivenWorkStealingDispatcher(name, mailboxType = mailboxType) + def newExecutorBasedEventDrivenWorkStealingDispatcher(name: String, mailboxType: MailboxType) = + ThreadPoolConfigDispatcherBuilder(config => new ExecutorBasedEventDrivenWorkStealingDispatcher(name,mailboxType,config),ThreadPoolConfig()) /** * Utility function that tries to load the specified dispatcher config from the akka.conf * or else use the supplied default dispatcher */ def fromConfig(key: String, default: => MessageDispatcher = defaultGlobalDispatcher): MessageDispatcher = - config.getConfigMap(key).flatMap(from).getOrElse(default) + config getConfigMap key flatMap from getOrElse default /* * Creates of obtains a dispatcher from a ConfigMap according to the format below @@ -167,13 +171,10 @@ object Dispatchers extends Logging { lazy val name = cfg.getString("name", newUuid.toString) def configureThreadPool(createDispatcher: => (ThreadPoolConfig) => MessageDispatcher): ThreadPoolConfigDispatcherBuilder = { - val builder = ThreadPoolConfigDispatcherBuilder(createDispatcher,ThreadPoolConfig()) //Create a new builder - //Creates a transformation from builder to builder, if the option isDefined - def conf_?[T](opt: Option[T])(fun: (T) => ThreadPoolConfigDispatcherBuilder => ThreadPoolConfigDispatcherBuilder): - Option[(ThreadPoolConfigDispatcherBuilder) => ThreadPoolConfigDispatcherBuilder] = opt map fun + import ThreadPoolConfigDispatcherBuilder.conf_? //Apply the following options to the config if they are present in the cfg - List( + ThreadPoolConfigDispatcherBuilder(createDispatcher,ThreadPoolConfig()).configure( conf_?(cfg getInt "keep-alive-time" )(time => _.setKeepAliveTime(Duration(time, TIME_UNIT))), conf_?(cfg getDouble "core-pool-size-factor")(factor => _.setCorePoolSizeFromFactor(factor)), conf_?(cfg getDouble "max-pool-size-factor" )(factor => _.setMaxPoolSizeFromFactor(factor)), @@ -185,8 +186,7 @@ object Dispatchers extends Logging { case "discard-oldest" => new DiscardOldestPolicy() case "discard" => new DiscardPolicy() case x => throw new IllegalArgumentException("[%s] is not a valid rejectionPolicy!" format x) - })(policy => _.setRejectionPolicy(policy)) - ).foldLeft(builder)( (c,f) => f.map( _(c) ).getOrElse(c)) //Returns the builder with all the specified options set + })(policy => _.setRejectionPolicy(policy))) } lazy val mailboxType: MailboxType = { diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala index 4e4a1f9c59..6ec5f6963e 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala @@ -82,6 +82,9 @@ class ExecutorBasedEventDrivenDispatcher( def this(_name: String, throughput: Int) = this(_name, throughput, Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS, Dispatchers.MAILBOX_TYPE) // Needed for Java API usage + def this(_name: String, _config: ThreadPoolConfig) = + this(_name, Dispatchers.THROUGHPUT, Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS, Dispatchers.MAILBOX_TYPE, _config) + def this(_name: String) = this(_name, Dispatchers.THROUGHPUT, Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS, Dispatchers.MAILBOX_TYPE) // Needed for Java API usage @@ -90,7 +93,7 @@ class ExecutorBasedEventDrivenDispatcher( private[akka] val active = new Switch(false) private[akka] val threadFactory = new MonitorableThreadFactory(name) - private[akka] val executorService = new AtomicReference[ExecutorService](null) + private[akka] val executorService = new AtomicReference[ExecutorService](config.createLazyExecutorService(threadFactory)) def dispatch(invocation: MessageInvocation) = { val mbox = getMailbox(invocation.receiver) @@ -133,15 +136,10 @@ class ExecutorBasedEventDrivenDispatcher( def start: Unit = if (active.isOff) active switchOn { log.debug("Starting up %s\n\twith throughput [%d]", toString, throughput) - if (executorService.get() eq null) { - val newExecutor = config.createExecutorService(threadFactory) - if (!executorService.compareAndSet(null,newExecutor)) - log.error("Thought the ExecutorService was missing but appeared out of nowhere!") - } } def shutdown: Unit = if (active.isOn) active switchOff { - val old = executorService.getAndSet(null) + val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory)) if (old ne null) { log.debug("Shutting down %s", toString) old.shutdownNow() diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala index f44bd60c50..a9ea028210 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala @@ -4,11 +4,12 @@ package se.scalablesolutions.akka.dispatch -import java.util.concurrent.CopyOnWriteArrayList import jsr166x.{Deque, ConcurrentLinkedDeque, LinkedBlockingDeque} import se.scalablesolutions.akka.actor.{Actor, ActorRef, IllegalActorStateException} import se.scalablesolutions.akka.util.Switch +import java.util.concurrent. {ExecutorService, CopyOnWriteArrayList} +import java.util.concurrent.atomic.AtomicReference /** * An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed @@ -33,27 +34,28 @@ import se.scalablesolutions.akka.util.Switch class ExecutorBasedEventDrivenWorkStealingDispatcher( _name: String, _mailboxType: MailboxType = Dispatchers.MAILBOX_TYPE, - config: ThreadPoolConfig = ThreadPoolConfig()) extends MessageDispatcher with ThreadPoolBuilder { + config: ThreadPoolConfig = ThreadPoolConfig()) extends MessageDispatcher { def this(_name: String, mailboxType: MailboxType) = this(_name, mailboxType,ThreadPoolConfig()) def this(_name: String) = this(_name, Dispatchers.MAILBOX_TYPE,ThreadPoolConfig()) + + //implicit def actorRef2actor(actorRef: ActorRef): Actor = actorRef.actor val mailboxType = Some(_mailboxType) - + val name = "akka:event-driven-work-stealing:dispatcher:" + _name private val active = new Switch(false) - implicit def actorRef2actor(actorRef: ActorRef): Actor = actorRef.actor - /** Type of the actors registered in this dispatcher. */ - private var actorType: Option[Class[_]] = None - + @volatile private var actorType: Option[Class[_]] = None private val pooledActors = new CopyOnWriteArrayList[ActorRef] + private[akka] val threadFactory = new MonitorableThreadFactory(name) + private[akka] val executorService = new AtomicReference[ExecutorService](config.createLazyExecutorService(threadFactory)) /** The index in the pooled actors list which was last used to steal work */ @volatile private var lastThiefIndex = 0 - val name = "akka:event-driven-work-stealing:dispatcher:" + _name + /** * @return the mailbox associated with the actor @@ -65,7 +67,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( def dispatch(invocation: MessageInvocation) = if (active.isOn) { val mbox = getMailbox(invocation.receiver) mbox enqueue invocation - executor execute mbox + executorService.get() execute mbox } else throw new IllegalActorStateException("Can't submit invocations to dispatcher since it's not started") /** @@ -92,6 +94,8 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( mailboxWasProcessed } + def isShutdown = active.isOff + /** * Process the messages in the mailbox of the given actor. * @return @@ -160,10 +164,10 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( private def donateMessage(receiver: ActorRef, thief: ActorRef): Boolean = { val donated = getMailbox(receiver).pollLast if (donated ne null) { - if (donated.senderFuture.isDefined) thief.self.postMessageToMailboxAndCreateFutureResultWithTimeout[Any]( + if (donated.senderFuture.isDefined) thief.postMessageToMailboxAndCreateFutureResultWithTimeout[Any]( donated.message, receiver.timeout, donated.sender, donated.senderFuture) - else if (donated.sender.isDefined) thief.self.postMessageToMailbox(donated.message, donated.sender) - else thief.self.postMessageToMailbox(donated.message, None) + else if (donated.sender.isDefined) thief.postMessageToMailbox(donated.message, donated.sender) + else thief.postMessageToMailbox(donated.message, None) true } else false } @@ -172,10 +176,13 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( log.debug("Starting up %s",toString) } - def shutdown = active switchOff { - log.debug("Shutting down %s", toString) - executor.shutdownNow - uuids.clear + def shutdown: Unit = if (active.isOn) active switchOff { + val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory)) + if (old ne null) { + log.debug("Shutting down %s", toString) + old.shutdownNow() + uuids.clear + } } @@ -186,7 +193,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( def resume(actorRef: ActorRef) { val mbox = getMailbox(actorRef) mbox.suspended.switchOff - executor execute mbox + executorService.get() execute mbox } def ensureNotActive(): Unit = if (active.isOn) throw new IllegalActorStateException( @@ -207,9 +214,8 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( findThief(actorRef).foreach( tryDonateAndProcessMessages(actorRef,_) ) } } - case BoundedMailbox(blocking, capacity, pushTimeOut) => - val cap = if (mailboxCapacity == -1) capacity else mailboxCapacity - new LinkedBlockingDeque[MessageInvocation](cap) with MessageQueue with Runnable { + case BoundedMailbox(blocking, capacity, pushTimeOut) => + new LinkedBlockingDeque[MessageInvocation](capacity) with MessageQueue with Runnable { def enqueue(handle: MessageInvocation): Unit = this.add(handle) def dequeue: MessageInvocation = this.poll() diff --git a/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala index e65d47796e..ef53e81eac 100644 --- a/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala @@ -58,6 +58,9 @@ case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.def flowHandler: ThreadPoolConfig.FlowHandler = ThreadPoolConfig.defaultFlowHandler, queueFactory: ThreadPoolConfig.QueueFactory = ThreadPoolConfig.linkedBlockingQueue()) { + final def createLazyExecutorService(threadFactory: ThreadFactory): ExecutorService = + new LazyExecutorServiceWrapper(createExecutorService(threadFactory)) + final def createExecutorService(threadFactory: ThreadFactory): ExecutorService = { flowHandler match { case Left(rejectHandler) => @@ -76,6 +79,11 @@ trait DispatcherBuilder { def build: MessageDispatcher } +object ThreadPoolConfigDispatcherBuilder { + def conf_?[T](opt: Option[T])(fun: (T) => ThreadPoolConfigDispatcherBuilder => ThreadPoolConfigDispatcherBuilder): + Option[(ThreadPoolConfigDispatcherBuilder) => ThreadPoolConfigDispatcherBuilder] = opt map fun +} + case class ThreadPoolConfigDispatcherBuilder(dispatcherFactory: (ThreadPoolConfig) => MessageDispatcher, config: ThreadPoolConfig) extends DispatcherBuilder { import ThreadPoolConfig._ def build = dispatcherFactory(config) @@ -133,214 +141,18 @@ case class ThreadPoolConfigDispatcherBuilder(dispatcherFactory: (ThreadPoolConfi def setAllowCoreThreadTimeout(allow: Boolean): ThreadPoolConfigDispatcherBuilder = this.copy(config = config.copy(allowCorePoolTimeout = allow)) + + def configure(fs: Option[Function[ThreadPoolConfigDispatcherBuilder,ThreadPoolConfigDispatcherBuilder]]*): + ThreadPoolConfigDispatcherBuilder = fs.foldLeft(this)( (c,f) => f.map( _(c) ).getOrElse(c)) } - -trait ThreadPoolBuilder extends Logging { - val name: String - - private val NR_START_THREADS = 16 - private val NR_MAX_THREADS = 128 - private val KEEP_ALIVE_TIME = 60000L // default is one minute - private val MILLISECONDS = TimeUnit.MILLISECONDS - - private var threadPoolBuilder: ThreadPoolExecutor = _ - private var boundedExecutorBound = -1 - protected var mailboxCapacity = -1 - @volatile private var inProcessOfBuilding = false - private var blockingQueue: BlockingQueue[Runnable] = _ - - protected lazy val threadFactory = new MonitorableThreadFactory(name) - - @volatile var executor: ExecutorService = _ - - def isShutdown = executor.isShutdown - - def buildThreadPool(): ExecutorService = synchronized { - ensureNotActive - inProcessOfBuilding = false - - log.debug("Creating a %s with config [core-pool:%d,max-pool:%d,timeout:%d,allowCoreTimeout:%s,rejectPolicy:%s]", - getClass.getName, - threadPoolBuilder.getCorePoolSize, - threadPoolBuilder.getMaximumPoolSize, - threadPoolBuilder.getKeepAliveTime(MILLISECONDS), - threadPoolBuilder.allowsCoreThreadTimeOut, - threadPoolBuilder.getRejectedExecutionHandler.getClass.getSimpleName) - - if (boundedExecutorBound > 0) { - val boundedExecutor = new BoundedExecutorDecorator(threadPoolBuilder, boundedExecutorBound) - boundedExecutorBound = -1 //Why is this here? - executor = boundedExecutor - } else { - executor = threadPoolBuilder - } - executor - } - - def withNewThreadPoolWithCustomBlockingQueue(queue: BlockingQueue[Runnable]): ThreadPoolBuilder = synchronized { - ensureNotActive - verifyNotInConstructionPhase - blockingQueue = queue - threadPoolBuilder = new ThreadPoolExecutor(NR_START_THREADS, NR_MAX_THREADS, KEEP_ALIVE_TIME, MILLISECONDS, queue) - this - } - - /** - * Creates a new thread pool in which the number of tasks in the pending queue is bounded. Will block when exceeded. - *

- * The 'bound' variable should specify the number equal to the size of the thread pool PLUS the number of queued tasks that should be followed. - */ - def withNewBoundedThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity(bound: Int): ThreadPoolBuilder = synchronized { - ensureNotActive - verifyNotInConstructionPhase - blockingQueue = new LinkedBlockingQueue[Runnable] - threadPoolBuilder = new ThreadPoolExecutor(NR_START_THREADS, NR_MAX_THREADS, KEEP_ALIVE_TIME, MILLISECONDS, blockingQueue, threadFactory) - boundedExecutorBound = bound - this - } - - def withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity: ThreadPoolBuilder = synchronized { - ensureNotActive - verifyNotInConstructionPhase - blockingQueue = new LinkedBlockingQueue[Runnable] - threadPoolBuilder = new ThreadPoolExecutor( - NR_START_THREADS, NR_MAX_THREADS, KEEP_ALIVE_TIME, MILLISECONDS, blockingQueue, threadFactory, new CallerRunsPolicy) - this - } - - def withNewThreadPoolWithLinkedBlockingQueueWithCapacity(capacity: Int): ThreadPoolBuilder = synchronized { - ensureNotActive - verifyNotInConstructionPhase - blockingQueue = new LinkedBlockingQueue[Runnable](capacity) - threadPoolBuilder = new ThreadPoolExecutor( - NR_START_THREADS, NR_MAX_THREADS, KEEP_ALIVE_TIME, MILLISECONDS, blockingQueue, threadFactory, new CallerRunsPolicy) - this - } - - def withNewThreadPoolWithSynchronousQueueWithFairness(fair: Boolean): ThreadPoolBuilder = synchronized { - ensureNotActive - verifyNotInConstructionPhase - blockingQueue = new SynchronousQueue[Runnable](fair) - threadPoolBuilder = new ThreadPoolExecutor( - NR_START_THREADS, NR_MAX_THREADS, KEEP_ALIVE_TIME, MILLISECONDS, blockingQueue, threadFactory, new CallerRunsPolicy) - this - } - - def withNewThreadPoolWithArrayBlockingQueueWithCapacityAndFairness(capacity: Int, fair: Boolean): ThreadPoolBuilder = synchronized { - ensureNotActive - verifyNotInConstructionPhase - blockingQueue = new ArrayBlockingQueue[Runnable](capacity, fair) - threadPoolBuilder = new ThreadPoolExecutor( - NR_START_THREADS, NR_MAX_THREADS, KEEP_ALIVE_TIME, MILLISECONDS, blockingQueue, threadFactory, new CallerRunsPolicy) - this - } - - def configureIfPossible(f: (ThreadPoolBuilder) => Unit): Boolean = synchronized { - if(inProcessOfBuilding) { - f(this) - true - } - else { - log.warning("Tried to configure an already started ThreadPoolBuilder of type [%s]",getClass.getName) - false - } - } - - /** - * Default is 16. - */ - def setCorePoolSize(size: Int): ThreadPoolBuilder = - setThreadPoolExecutorProperty(_.setCorePoolSize(size)) - - /** - * Default is 128. - */ - def setMaxPoolSize(size: Int): ThreadPoolBuilder = - setThreadPoolExecutorProperty(_.setMaximumPoolSize(size)) - - - /** - * Sets the core pool size to (availableProcessors * multipliers).ceil.toInt - */ - def setCorePoolSizeFromFactor(multiplier: Double): ThreadPoolBuilder = - setThreadPoolExecutorProperty(_.setCorePoolSize(procs(multiplier))) - - /** - * Sets the max pool size to (availableProcessors * multipliers).ceil.toInt - */ - def setMaxPoolSizeFromFactor(multiplier: Double): ThreadPoolBuilder = - setThreadPoolExecutorProperty(_.setMaximumPoolSize(procs(multiplier))) - - /** - * Sets the bound, -1 is unbounded - */ - def setExecutorBounds(bounds: Int): Unit = synchronized { - this.boundedExecutorBound = bounds - } - - /** - * Sets the mailbox capacity, -1 is unbounded - */ - def setMailboxCapacity(capacity: Int): Unit = synchronized { - this.mailboxCapacity = capacity - } - - protected def procs(multiplier: Double): Int = - (Runtime.getRuntime.availableProcessors * multiplier).ceil.toInt - - /** - * Default is 60000 (one minute). - */ - def setKeepAliveTimeInMillis(time: Long): ThreadPoolBuilder = - setThreadPoolExecutorProperty(_.setKeepAliveTime(time, MILLISECONDS)) - - /** - * Default ThreadPoolExecutor.CallerRunsPolicy. To allow graceful backing off when pool is overloaded. - */ - def setRejectionPolicy(policy: RejectedExecutionHandler): ThreadPoolBuilder = - setThreadPoolExecutorProperty(_.setRejectedExecutionHandler(policy)) - - /** - * Default false, set to true to conserve thread for potentially unused dispatchers - */ - def setAllowCoreThreadTimeout(allow: Boolean) = - setThreadPoolExecutorProperty(_.allowCoreThreadTimeOut(allow)) - - /** - * Default ThreadPoolExecutor.CallerRunsPolicy. To allow graceful backing off when pool is overloaded. - */ - protected def setThreadPoolExecutorProperty(f: (ThreadPoolExecutor) => Unit): ThreadPoolBuilder = synchronized { - ensureNotActive - verifyInConstructionPhase - f(threadPoolBuilder) - this - } - - - protected def verifyNotInConstructionPhase = { - if (inProcessOfBuilding) throw new IllegalActorStateException("Is already in the process of building a thread pool") - inProcessOfBuilding = true - } - - protected def verifyInConstructionPhase = { - if (!inProcessOfBuilding) throw new IllegalActorStateException( - "Is not in the process of building a thread pool, start building one by invoking one of the 'newThreadPool*' methods") - } - - def ensureNotActive(): Unit -} - - /** * @author Jonas Bonér */ class MonitorableThreadFactory(val name: String) extends ThreadFactory { protected val counter = new AtomicLong - def newThread(runnable: Runnable) = - new MonitorableThread(runnable, name) -// new Thread(runnable, name + "-" + counter.getAndIncrement) + def newThread(runnable: Runnable) = new MonitorableThread(runnable, name) } /** @@ -382,10 +194,10 @@ class MonitorableThread(runnable: Runnable, name: String) /** * @author Jonas Bonér */ -class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extends ExecutorService with Logging { +class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extends ExecutorServiceDelegate with Logging { protected val semaphore = new Semaphore(bound) - def execute(command: Runnable) = { + override def execute(command: Runnable) = { semaphore.acquire try { executor.execute(new Runnable() { @@ -405,8 +217,14 @@ class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extend throw e } } +} + +trait ExecutorServiceDelegate extends ExecutorService { + + def executor: ExecutorService + + def execute(command: Runnable) = executor.execute(command) - // Delegating methods for the ExecutorService interface def shutdown = executor.shutdown def shutdownNow = executor.shutdownNow @@ -431,3 +249,14 @@ class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extend def invokeAny[T](callables: Collection[_ <: Callable[T]], l: Long, timeUnit: TimeUnit) = executor.invokeAny(callables, l, timeUnit) } + +trait LazyExecutorService extends ExecutorServiceDelegate { + + def createExecutor: ExecutorService + + lazy val executor = createExecutor +} + +class LazyExecutorServiceWrapper(executorFactory: => ExecutorService) extends LazyExecutorService { + def createExecutor = executorFactory +} \ No newline at end of file diff --git a/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala b/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala index 5fcaf13173..2ad8ac267b 100644 --- a/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala +++ b/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala @@ -36,7 +36,7 @@ class SupervisorMiscSpec extends WordSpec with MustMatchers { }).start val actor3 = Actor.actorOf(new Actor { - self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("test") + self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("test").build override def postRestart(cause: Throwable) {countDownLatch.countDown} protected def receive = { diff --git a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala index 352ade75f0..1c2670da0d 100644 --- a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala @@ -10,7 +10,7 @@ import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} object ExecutorBasedEventDrivenDispatcherActorSpec { class TestActor extends Actor { - self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(self.uuid.toString) + self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(self.uuid.toString).build def receive = { case "Hello" => self.reply("World") @@ -23,7 +23,7 @@ object ExecutorBasedEventDrivenDispatcherActorSpec { val oneWay = new CountDownLatch(1) } class OneWayTestActor extends Actor { - self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(self.uuid.toString) + self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(self.uuid.toString).build def receive = { case "OneWay" => OneWayTestActor.oneWay.countDown } @@ -68,9 +68,10 @@ class ExecutorBasedEventDrivenDispatcherActorSpec extends JUnitSuite { } @Test def shouldRespectThroughput { - val throughputDispatcher = new ExecutorBasedEventDrivenDispatcher("THROUGHPUT",101,0,Dispatchers.MAILBOX_TYPE, (e) => { - e.setCorePoolSize(1) - }) + val throughputDispatcher = Dispatchers. + newExecutorBasedEventDrivenDispatcher("THROUGHPUT",101,0,Dispatchers.MAILBOX_TYPE). + setCorePoolSize(1). + build val works = new AtomicBoolean(true) val latch = new CountDownLatch(100) @@ -103,10 +104,10 @@ class ExecutorBasedEventDrivenDispatcherActorSpec extends JUnitSuite { @Test def shouldRespectThroughputDeadline { val deadlineMs = 100 - val throughputDispatcher = new ExecutorBasedEventDrivenDispatcher("THROUGHPUT",2,deadlineMs,Dispatchers.MAILBOX_TYPE, (e) => { - e.setCorePoolSize(1) - }) - + val throughputDispatcher = Dispatchers. + newExecutorBasedEventDrivenDispatcher("THROUGHPUT",2,deadlineMs,Dispatchers.MAILBOX_TYPE). + setCorePoolSize(1). + build val works = new AtomicBoolean(true) val latch = new CountDownLatch(1) val start = new CountDownLatch(1) diff --git a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala index 3285e450c6..fe45d79869 100644 --- a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala @@ -11,9 +11,9 @@ import Actor._ import se.scalablesolutions.akka.dispatch.{MessageQueue, Dispatchers} object ExecutorBasedEventDrivenWorkStealingDispatcherSpec { - val delayableActorDispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher") - val sharedActorDispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher") - val parentActorDispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher") + val delayableActorDispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher").build + val sharedActorDispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher").build + val parentActorDispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher").build class DelayableActor(name: String, delay: Int, finishedCounter: CountDownLatch) extends Actor { self.dispatcher = delayableActorDispatcher diff --git a/akka-spring/src/main/scala/DispatcherFactoryBean.scala b/akka-spring/src/main/scala/DispatcherFactoryBean.scala index d273e75011..cfe0e05930 100644 --- a/akka-spring/src/main/scala/DispatcherFactoryBean.scala +++ b/akka-spring/src/main/scala/DispatcherFactoryBean.scala @@ -11,6 +11,7 @@ import se.scalablesolutions.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 /** * Reusable factory method for dispatchers. @@ -24,53 +25,66 @@ object DispatcherFactoryBean { */ def createNewInstance(properties: DispatcherProperties, actorRef: Option[ActorRef] = None): MessageDispatcher = { - def configThreadPool(): ThreadPoolConfig = { - val poolCfg = ThreadPoolConfig() + //Creates a ThreadPoolConfigDispatcherBuilder and applies the configuration to it + def configureThreadPool(createDispatcher: => (ThreadPoolConfig) => MessageDispatcher): ThreadPoolConfigDispatcherBuilder = { if ((properties.threadPool ne null) && (properties.threadPool.queue ne null)) { - properties.threadPool.queue match { - case VAL_BOUNDED_ARRAY_BLOCKING_QUEUE => threadPoolBuilder.withNewThreadPoolWithArrayBlockingQueueWithCapacityAndFairness(properties.threadPool.capacity, properties.threadPool.fairness) - case VAL_UNBOUNDED_LINKED_BLOCKING_QUEUE if properties.threadPool.capacity > -1 => threadPoolBuilder.withNewThreadPoolWithLinkedBlockingQueueWithCapacity(properties.threadPool.capacity) - case VAL_UNBOUNDED_LINKED_BLOCKING_QUEUE if properties.threadPool.capacity <= 0 => threadPoolBuilder.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity - case VAL_BOUNDED_LINKED_BLOCKING_QUEUE => threadPoolBuilder.withNewBoundedThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity(properties.threadPool.bound) - case VAL_SYNCHRONOUS_QUEUE => threadPoolBuilder.withNewThreadPoolWithSynchronousQueueWithFairness(properties.threadPool.fairness) - case _ => throw new IllegalArgumentException("unknown queue type") - } + import ThreadPoolConfigDispatcherBuilder.conf_? + import properties._ + val queueDef = Some(threadPool.queue) + val corePoolSize = if (threadPool.corePoolSize > -1) Some(threadPool.corePoolSize) else None + val maxPoolSize = if (threadPool.maxPoolSize > -1) Some(threadPool.maxPoolSize) else None + val keepAlive = if (threadPool.keepAlive > -1) Some(threadPool.keepAlive) else None + val executorBounds = if (threadPool.bound > -1) Some(threadPool.bound) else None + val flowHandler = threadPool.rejectionPolicy match { + case null | "" => None + case "abort-policy" => Some(new AbortPolicy()) + case "caller-runs-policy" => Some(new CallerRunsPolicy()) + case "discard-oldest-policy" => Some(new DiscardOldestPolicy()) + case "discard-policy" => Some(new DiscardPolicy()) + case x => throw new IllegalArgumentException("Unknown rejection-policy '" + x + "'") + } - if (properties.threadPool.corePoolSize > -1) - threadPoolBuilder.setCorePoolSize(properties.threadPool.corePoolSize) - - if (properties.threadPool.maxPoolSize > -1) - threadPoolBuilder.setMaxPoolSize(properties.threadPool.maxPoolSize) - - if (properties.threadPool.keepAlive > -1) - threadPoolBuilder.setKeepAliveTimeInMillis(properties.threadPool.keepAlive) - - if (properties.threadPool.mailboxCapacity > -1) - threadPoolBuilder.setMailboxCapacity(properties.threadPool.mailboxCapacity) - - if ((properties.threadPool.rejectionPolicy ne null) && (!properties.threadPool.rejectionPolicy.isEmpty)) { - val policy: RejectedExecutionHandler = properties.threadPool.rejectionPolicy match { - case "abort-policy" => new AbortPolicy() - case "caller-runs-policy" => new CallerRunsPolicy() - case "discard-oldest-policy" => new DiscardOldestPolicy() - case "discard-policy" => new DiscardPolicy() - case _ => throw new IllegalArgumentException("Unknown rejection-policy '" + properties.threadPool.rejectionPolicy + "'") - } - threadPoolBuilder.setRejectionPolicy(policy) - } - } else poolCfg + //Apply the following options to the config if they are present in the cfg + ThreadPoolConfigDispatcherBuilder(createDispatcher,ThreadPoolConfig()).configure( + conf_?(queueDef )(definition => definition match { + case VAL_BOUNDED_ARRAY_BLOCKING_QUEUE => + _.withNewThreadPoolWithArrayBlockingQueueWithCapacityAndFairness(threadPool.capacity,threadPool.fairness) + case VAL_UNBOUNDED_LINKED_BLOCKING_QUEUE if threadPool.capacity > -1 => + _.withNewThreadPoolWithLinkedBlockingQueueWithCapacity(threadPool.capacity) + case VAL_UNBOUNDED_LINKED_BLOCKING_QUEUE if threadPool.capacity <= 0 => + _.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity + case VAL_BOUNDED_LINKED_BLOCKING_QUEUE => + _.withNewBoundedThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity(threadPool.bound) + case VAL_SYNCHRONOUS_QUEUE => + _.withNewThreadPoolWithSynchronousQueueWithFairness(threadPool.fairness) + case unknown => + throw new IllegalArgumentException("Unknown queue type " + unknown) + }), + conf_?(keepAlive )(time => _.setKeepAliveTimeInMillis(time)), + conf_?(corePoolSize )(count => _.setCorePoolSize(count)), + conf_?(maxPoolSize )(count => _.setMaxPoolSize(count)), + conf_?(executorBounds)(bounds => _.setExecutorBounds(bounds)), + conf_?(flowHandler )(policy => _.setRejectionPolicy(policy))) + } + else + ThreadPoolConfigDispatcherBuilder(createDispatcher,ThreadPoolConfig()) } - var dispatcher = properties.dispatcherType match { - case EXECUTOR_BASED_EVENT_DRIVEN => new ExecutorBasedEventDrivenDispatcher(properties.name, config = configThreadPool) - case EXECUTOR_BASED_EVENT_DRIVEN_WORK_STEALING => Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher(properties.name) - case THREAD_BASED if actorRef.isEmpty => throw new IllegalArgumentException("Need an ActorRef to create a thread based dispatcher.") - case THREAD_BASED if actorRef.isDefined => Dispatchers.newThreadBasedDispatcher(actorRef.get) - case HAWT => Dispatchers.newHawtDispatcher(properties.aggregate) - case _ => throw new IllegalArgumentException("unknown dispatcher type") + //Create the dispatcher + properties.dispatcherType match { + case EXECUTOR_BASED_EVENT_DRIVEN => + configureThreadPool(poolConfig => new ExecutorBasedEventDrivenDispatcher(properties.name, poolConfig)).build + case EXECUTOR_BASED_EVENT_DRIVEN_WORK_STEALING => + configureThreadPool(poolConfig => new ExecutorBasedEventDrivenWorkStealingDispatcher(properties.name,Dispatchers.MAILBOX_TYPE,poolConfig)).build + case THREAD_BASED if actorRef.isEmpty => + throw new IllegalArgumentException("Need an ActorRef to create a thread based dispatcher.") + case THREAD_BASED if actorRef.isDefined => + Dispatchers.newThreadBasedDispatcher(actorRef.get) + case HAWT => + Dispatchers.newHawtDispatcher(properties.aggregate) + case unknown => + throw new IllegalArgumentException("Unknown dispatcher type " + unknown) } - - dispatcher } } diff --git a/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala b/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala index 51c8d2bd73..a5ae40ff16 100644 --- a/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala +++ b/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala @@ -130,13 +130,18 @@ class DispatcherSpringFeatureTest extends FeatureSpec with ShouldMatchers { * get ThreadPoolExecutor via reflection and assert that dispatcher is correct type */ private def getThreadPoolExecutorAndAssert(dispatcher: MessageDispatcher): ThreadPoolExecutor = { - assert(dispatcher.isInstanceOf[ThreadPoolBuilder]) - val pool = dispatcher.asInstanceOf[ThreadPoolBuilder] - val field = pool.getClass.getDeclaredField("se$scalablesolutions$akka$dispatch$ThreadPoolBuilder$$threadPoolBuilder") - field.setAccessible(true) - val executor = field.get(pool).asInstanceOf[ThreadPoolExecutor] - assert(executor ne null) - executor; + + def unpackExecutorService(e: ExecutorService): ExecutorService = e match { + case b: ExecutorServiceDelegate => unpackExecutorService(b.executor) + case t: ThreadPoolExecutor => t + case e => throw new IllegalStateException("Illegal executor type: " + e) + } + + unpackExecutorService(dispatcher match { + case e: ExecutorBasedEventDrivenDispatcher => e.start; e.executorService.get() + case e: ExecutorBasedEventDrivenWorkStealingDispatcher => e.start; e.executorService.get() + case x => throw new IllegalStateException("Illegal dispatcher type: " + x) + }).asInstanceOf[ThreadPoolExecutor] } } 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 ea6e939386..5d2e9fd315 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 @@ -30,7 +30,7 @@ class TypedActorGuiceConfiguratorSpec extends override def beforeAll { Config.config - val dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("test") + val dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("test").build conf.addExternalGuiceModule(new AbstractModule { def configure = bind(classOf[Ext]).to(classOf[ExtImpl]).in(Scopes.SINGLETON) From 149d346e5f87acf83215b47854e9018cc869bd7a Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sun, 24 Oct 2010 01:18:59 +0200 Subject: [PATCH 03/39] Removing unused code and the isShutdown method --- .../src/main/scala/actor/ActorRef.scala | 1 - .../ExecutorBasedEventDrivenDispatcher.scala | 2 -- ...sedEventDrivenWorkStealingDispatcher.scala | 2 -- .../main/scala/dispatch/HawtDispatcher.scala | 2 -- .../main/scala/dispatch/MessageHandling.scala | 35 +++++++++++-------- .../actor/supervisor/SupervisorMiscSpec.scala | 8 ++--- ...rBasedEventDrivenDispatcherActorSpec.scala | 1 - 7 files changed, 24 insertions(+), 27 deletions(-) diff --git a/akka-actor/src/main/scala/actor/ActorRef.scala b/akka-actor/src/main/scala/actor/ActorRef.scala index 7a6e493f43..f88f40fecf 100644 --- a/akka-actor/src/main/scala/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/actor/ActorRef.scala @@ -781,7 +781,6 @@ class LocalActorRef private[akka] ( "Can't restart an actor that has been shut down with 'stop' or 'exit'") if (!isRunning) { dispatcher.register(this) - dispatcher.start if (isTransactor) transactorConfig = transactorConfig.copy(factory = Some(TransactionFactory(transactorConfig.config, id))) diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala index 6ec5f6963e..49e94afc6f 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala @@ -101,8 +101,6 @@ class ExecutorBasedEventDrivenDispatcher( registerForExecution(mbox) } - def isShutdown = active.isOff - /** * @return the mailbox associated with the actor */ diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala index a9ea028210..53296cce71 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala @@ -94,8 +94,6 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( mailboxWasProcessed } - def isShutdown = active.isOff - /** * Process the messages in the mailbox of the given actor. * @return diff --git a/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala b/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala index 4ca63f64f2..a5bc315204 100644 --- a/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala @@ -150,8 +150,6 @@ class HawtDispatcher(val aggregate: Boolean = true, val parent: DispatchQueue = def shutdown = active switchOff { releaseNonDaemon } - def isShutdown = active.isOff - def dispatch(invocation: MessageInvocation) = if (active.isOn) { mailbox(invocation.receiver).dispatch(invocation) } else { diff --git a/akka-actor/src/main/scala/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/dispatch/MessageHandling.scala index 88d978b311..c84a56e78d 100644 --- a/akka-actor/src/main/scala/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/dispatch/MessageHandling.scala @@ -4,15 +4,12 @@ package se.scalablesolutions.akka.dispatch -import se.scalablesolutions.akka.actor.{Actor, ActorRef, Uuid, ActorInitializationException} -import se.scalablesolutions.akka.util.{SimpleLock, Duration, HashCode, Logging} -import se.scalablesolutions.akka.util.ReflectiveAccess.EnterpriseModule -import se.scalablesolutions.akka.AkkaException +import se.scalablesolutions.akka.actor.{ActorRegistry, ActorRef, Uuid, ActorInitializationException} import org.multiverse.commitbarriers.CountDownCommitBarrier -import java.util.{Queue, List} import java.util.concurrent._ +import se.scalablesolutions.akka.util. {Logging, HashCode} /** * @author Jonas Bonér @@ -61,12 +58,6 @@ final class MessageInvocation(val receiver: ActorRef, trait MessageDispatcher extends MailboxFactory with Logging { protected val uuids = new ConcurrentSkipListSet[Uuid] - - def dispatch(invocation: MessageInvocation): Unit - - def start: Unit - - def shutdown: Unit def register(actorRef: ActorRef) { start @@ -77,15 +68,29 @@ trait MessageDispatcher extends MailboxFactory with Logging { def unregister(actorRef: ActorRef) = { uuids remove actorRef.uuid actorRef.mailbox = null - if (canBeShutDown) shutdown // shut down in the dispatcher's references is zero + if (uuids.isEmpty) shutdown // shut down in the dispatcher's references is zero + } + + def stopAllLinkedActors { + val i = uuids.iterator + while(i.hasNext()) { + val uuid = i.next() + i.remove() + ActorRegistry.actorFor(uuid) match { + case Some(actor) => actor.stop + case None => log.warn("stopAllLinkedActors couldn't find linked actor: " + uuid) + } + } + if(uuids.isEmpty) shutdown } def suspend(actorRef: ActorRef): Unit def resume(actorRef: ActorRef): Unit - - def canBeShutDown: Boolean = uuids.isEmpty - def isShutdown: Boolean + def dispatch(invocation: MessageInvocation): Unit + + protected def start: Unit + protected def shutdown: Unit /** * Returns the size of the mailbox for the specified actor diff --git a/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala b/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala index 2ad8ac267b..17cfc94f83 100644 --- a/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala +++ b/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala @@ -70,10 +70,10 @@ class SupervisorMiscSpec extends WordSpec with MustMatchers { actor4 ! "kill" countDownLatch.await() - assert(!actor1.dispatcher.isShutdown, "dispatcher1 is shutdown") - assert(!actor2.dispatcher.isShutdown, "dispatcher2 is shutdown") - assert(!actor3.dispatcher.isShutdown, "dispatcher3 is shutdown") - assert(!actor4.dispatcher.isShutdown, "dispatcher4 is shutdown") + assert(!actor1.isShutdown, "actor1 is shutdown") + assert(!actor2.isShutdown, "actor2 is shutdown") + assert(!actor3.isShutdown, "actor3 is shutdown") + assert(!actor4.isShutdown, "actor4 is shutdown") } } } diff --git a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala index 1c2670da0d..f2f4787070 100644 --- a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala @@ -98,7 +98,6 @@ class ExecutorBasedEventDrivenDispatcherActorSpec extends JUnitSuite { val result = latch.await(3,TimeUnit.SECONDS) fastOne.stop slowOne.stop - throughputDispatcher.shutdown assert(result === true) } From 53e67d6b3ea910f0596102a888240906b553c70f Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sun, 24 Oct 2010 15:22:28 +0200 Subject: [PATCH 04/39] Fixing race-conditions, now works albeit inefficiently when adding/removing actors rapidly --- .../src/main/scala/actor/ActorRef.scala | 4 +-- .../ExecutorBasedEventDrivenDispatcher.scala | 5 ++-- ...sedEventDrivenWorkStealingDispatcher.scala | 8 ++---- .../main/scala/dispatch/MessageHandling.scala | 25 +++++++++++++------ .../scala/dispatch/ThreadPoolBuilder.scala | 12 +++++---- .../scala/DispatcherSpringFeatureTest.scala | 4 +-- 6 files changed, 32 insertions(+), 26 deletions(-) diff --git a/akka-actor/src/main/scala/actor/ActorRef.scala b/akka-actor/src/main/scala/actor/ActorRef.scala index f88f40fecf..870b74edd2 100644 --- a/akka-actor/src/main/scala/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/actor/ActorRef.scala @@ -780,7 +780,7 @@ class LocalActorRef private[akka] ( if (isShutdown) throw new ActorStartException( "Can't restart an actor that has been shut down with 'stop' or 'exit'") if (!isRunning) { - dispatcher.register(this) + dispatcher.attach(this) if (isTransactor) transactorConfig = transactorConfig.copy(factory = Some(TransactionFactory(transactorConfig.config, id))) @@ -802,7 +802,7 @@ class LocalActorRef private[akka] ( if (isRunning) { receiveTimeout = None cancelReceiveTimeout - dispatcher.unregister(this) + dispatcher.detach(this) transactorConfig = transactorConfig.copy(factory = None) _status = ActorRefInternals.SHUTDOWN actor.postStop diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala index 49e94afc6f..2b0a7cbfe0 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala @@ -132,16 +132,15 @@ class ExecutorBasedEventDrivenDispatcher( case JMSBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("JMSBasedDurableMailbox is not yet supported") } - def start: Unit = if (active.isOff) active switchOn { + protected def start: Unit = active switchOn { log.debug("Starting up %s\n\twith throughput [%d]", toString, throughput) } - def shutdown: Unit = if (active.isOn) active switchOff { + protected def shutdown: Unit = active switchOff { val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory)) if (old ne null) { log.debug("Shutting down %s", toString) old.shutdownNow() - uuids.clear } } diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala index 53296cce71..a6e40d2f50 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala @@ -170,16 +170,15 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( } else false } - def start = active switchOn { + protected def start = active switchOn { log.debug("Starting up %s",toString) } - def shutdown: Unit = if (active.isOn) active switchOff { + protected def shutdown: Unit = active switchOff { val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory)) if (old ne null) { log.debug("Shutting down %s", toString) old.shutdownNow() - uuids.clear } } @@ -194,9 +193,6 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( executorService.get() execute mbox } - def ensureNotActive(): Unit = if (active.isOn) throw new IllegalActorStateException( - "Can't build a new thread pool for a dispatcher that is already up and running") - override val toString = "ExecutorBasedEventDrivenWorkStealingDispatcher[" + name + "]" def createTransientMailbox(actorRef: ActorRef, mailboxType: TransientMailboxType): AnyRef = mailboxType match { diff --git a/akka-actor/src/main/scala/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/dispatch/MessageHandling.scala index c84a56e78d..1e18f057cf 100644 --- a/akka-actor/src/main/scala/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/dispatch/MessageHandling.scala @@ -9,7 +9,7 @@ import se.scalablesolutions.akka.actor.{ActorRegistry, ActorRef, Uuid, ActorInit import org.multiverse.commitbarriers.CountDownCommitBarrier import java.util.concurrent._ -import se.scalablesolutions.akka.util. {Logging, HashCode} +import se.scalablesolutions.akka.util. {ReentrantGuard, Logging, HashCode} /** * @author Jonas Bonér @@ -58,24 +58,33 @@ final class MessageInvocation(val receiver: ActorRef, trait MessageDispatcher extends MailboxFactory with Logging { protected val uuids = new ConcurrentSkipListSet[Uuid] + protected val guard = new ReentrantGuard - def register(actorRef: ActorRef) { - start + final def attach(actorRef: ActorRef): Unit = guard withGuard { + register(actorRef) + } + + final def detach(actorRef: ActorRef): Unit = guard withGuard { + unregister(actorRef) + } + + protected def register(actorRef: ActorRef) { + if (uuids.isEmpty()) start if (actorRef.mailbox eq null) actorRef.mailbox = createMailbox(actorRef) uuids add actorRef.uuid } - def unregister(actorRef: ActorRef) = { - uuids remove actorRef.uuid - actorRef.mailbox = null - if (uuids.isEmpty) shutdown // shut down in the dispatcher's references is zero + protected def unregister(actorRef: ActorRef) = { + if (uuids remove actorRef.uuid) { + actorRef.mailbox = null + if (uuids.isEmpty) shutdown // shut down in the dispatcher's references is zero + } } def stopAllLinkedActors { val i = uuids.iterator while(i.hasNext()) { val uuid = i.next() - i.remove() ActorRegistry.actorFor(uuid) match { case Some(actor) => actor.stop case None => log.warn("stopAllLinkedActors couldn't find linked actor: " + uuid) diff --git a/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala index ef53e81eac..0e1a1ceafb 100644 --- a/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala @@ -9,8 +9,7 @@ import java.util.concurrent._ import atomic.{AtomicLong, AtomicInteger} import ThreadPoolExecutor.CallerRunsPolicy -import se.scalablesolutions.akka.actor.IllegalActorStateException -import se.scalablesolutions.akka.util. {Duration, Logger, Logging} +import se.scalablesolutions.akka.util. {Duration, Logging} object ThreadPoolConfig { type Bounds = Int @@ -194,7 +193,7 @@ class MonitorableThread(runnable: Runnable, name: String) /** * @author Jonas Bonér */ -class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extends ExecutorServiceDelegate with Logging { +class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extends ExecutorServiceDelegate { protected val semaphore = new Semaphore(bound) override def execute(command: Runnable) = { @@ -219,7 +218,7 @@ class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extend } } -trait ExecutorServiceDelegate extends ExecutorService { +trait ExecutorServiceDelegate extends ExecutorService with Logging { def executor: ExecutorService @@ -254,7 +253,10 @@ trait LazyExecutorService extends ExecutorServiceDelegate { def createExecutor: ExecutorService - lazy val executor = createExecutor + lazy val executor = { + log.info("Lazily initializing ExecutorService for ",this) + createExecutor + } } class LazyExecutorServiceWrapper(executorFactory: => ExecutorService) extends LazyExecutorService { diff --git a/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala b/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala index a5ae40ff16..fac5358fc3 100644 --- a/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala +++ b/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala @@ -138,8 +138,8 @@ class DispatcherSpringFeatureTest extends FeatureSpec with ShouldMatchers { } unpackExecutorService(dispatcher match { - case e: ExecutorBasedEventDrivenDispatcher => e.start; e.executorService.get() - case e: ExecutorBasedEventDrivenWorkStealingDispatcher => e.start; e.executorService.get() + case e: ExecutorBasedEventDrivenDispatcher => e.executorService.get() + case e: ExecutorBasedEventDrivenWorkStealingDispatcher => e.executorService.get() case x => throw new IllegalStateException("Illegal dispatcher type: " + x) }).asInstanceOf[ThreadPoolExecutor] } From 990b933d8febaf01f61c383930e657ce69f814a9 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sun, 24 Oct 2010 16:01:00 +0200 Subject: [PATCH 05/39] Moved active flag into MessageDispatcher and let it handle the callbacks, also fixed race in DataFlowSpec --- .../src/main/scala/actor/ActorRef.scala | 4 +- .../ExecutorBasedEventDrivenDispatcher.scala | 9 +-- ...sedEventDrivenWorkStealingDispatcher.scala | 13 ++--- .../main/scala/dispatch/HawtDispatcher.scala | 10 +--- .../main/scala/dispatch/MessageHandling.scala | 57 +++++++++++++++---- .../test/scala/dataflow/DataFlowSpec.scala | 2 +- 6 files changed, 60 insertions(+), 35 deletions(-) diff --git a/akka-actor/src/main/scala/actor/ActorRef.scala b/akka-actor/src/main/scala/actor/ActorRef.scala index 870b74edd2..92a7800128 100644 --- a/akka-actor/src/main/scala/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/actor/ActorRef.scala @@ -964,7 +964,7 @@ class LocalActorRef private[akka] ( message, senderOption, None, remoteAddress.get, timeout, true, this, None, ActorType.ScalaActor) } else { val invocation = new MessageInvocation(this, message, senderOption, None, transactionSet.get) - dispatcher dispatch invocation + dispatcher dispatchMessage invocation } } @@ -985,7 +985,7 @@ class LocalActorRef private[akka] ( else new DefaultCompletableFuture[T](timeout) val invocation = new MessageInvocation( this, message, senderOption, Some(future.asInstanceOf[CompletableFuture[Any]]), transactionSet.get) - dispatcher dispatch invocation + dispatcher dispatchMessage invocation future } } diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala index 2b0a7cbfe0..45b1dfa886 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala @@ -91,11 +91,10 @@ class ExecutorBasedEventDrivenDispatcher( val name = "akka:event-driven:dispatcher:" + _name val mailboxType = Some(_mailboxType) - private[akka] val active = new Switch(false) private[akka] val threadFactory = new MonitorableThreadFactory(name) private[akka] val executorService = new AtomicReference[ExecutorService](config.createLazyExecutorService(threadFactory)) - def dispatch(invocation: MessageInvocation) = { + protected def dispatch(invocation: MessageInvocation) = { val mbox = getMailbox(invocation.receiver) mbox enqueue invocation registerForExecution(mbox) @@ -132,11 +131,9 @@ class ExecutorBasedEventDrivenDispatcher( case JMSBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("JMSBasedDurableMailbox is not yet supported") } - protected def start: Unit = active switchOn { - log.debug("Starting up %s\n\twith throughput [%d]", toString, throughput) - } + protected def start= log.debug("Starting up %s\n\twith throughput [%d]", toString, throughput) - protected def shutdown: Unit = active switchOff { + protected def shutdown { val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory)) if (old ne null) { log.debug("Shutting down %s", toString) diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala index a6e40d2f50..f357ff4ab3 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala @@ -44,7 +44,6 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( val mailboxType = Some(_mailboxType) val name = "akka:event-driven-work-stealing:dispatcher:" + _name - private val active = new Switch(false) /** Type of the actors registered in this dispatcher. */ @volatile private var actorType: Option[Class[_]] = None @@ -55,8 +54,6 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( /** The index in the pooled actors list which was last used to steal work */ @volatile private var lastThiefIndex = 0 - - /** * @return the mailbox associated with the actor */ @@ -64,11 +61,11 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( override def mailboxSize(actorRef: ActorRef) = getMailbox(actorRef).size - def dispatch(invocation: MessageInvocation) = if (active.isOn) { + protected def dispatch(invocation: MessageInvocation) { val mbox = getMailbox(invocation.receiver) mbox enqueue invocation executorService.get() execute mbox - } else throw new IllegalActorStateException("Can't submit invocations to dispatcher since it's not started") + } /** * Try processing the mailbox of the given actor. Fails if the dispatching lock on the actor is already held by @@ -170,11 +167,9 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( } else false } - protected def start = active switchOn { - log.debug("Starting up %s",toString) - } + protected def start = log.debug("Starting up %s",toString) - protected def shutdown: Unit = active switchOff { + protected def shutdown { val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory)) if (old ne null) { log.debug("Shutting down %s", toString) diff --git a/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala b/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala index a5bc315204..50ae2da19e 100644 --- a/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala @@ -142,18 +142,14 @@ object HawtDispatcher { class HawtDispatcher(val aggregate: Boolean = true, val parent: DispatchQueue = globalQueue) extends MessageDispatcher { import HawtDispatcher._ - private val active = new Switch(false) - val mailboxType: Option[MailboxType] = None - def start = active switchOn { retainNonDaemon } + protected def start { retainNonDaemon } - def shutdown = active switchOff { releaseNonDaemon } + protected def shutdown { releaseNonDaemon } - def dispatch(invocation: MessageInvocation) = if (active.isOn) { + protected def dispatch(invocation: MessageInvocation){ mailbox(invocation.receiver).dispatch(invocation) - } else { - log.warning("%s is shut down,\n\tignoring the the messages sent to\n\t%s", toString, invocation.receiver) } // hawtdispatch does not have a way to get queue sizes, getting an accurate diff --git a/akka-actor/src/main/scala/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/dispatch/MessageHandling.scala index 1e18f057cf..95923ebfe5 100644 --- a/akka-actor/src/main/scala/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/dispatch/MessageHandling.scala @@ -4,12 +4,11 @@ package se.scalablesolutions.akka.dispatch -import se.scalablesolutions.akka.actor.{ActorRegistry, ActorRef, Uuid, ActorInitializationException} - import org.multiverse.commitbarriers.CountDownCommitBarrier import java.util.concurrent._ -import se.scalablesolutions.akka.util. {ReentrantGuard, Logging, HashCode} +import se.scalablesolutions.akka.util. {Switch, ReentrantGuard, Logging, HashCode} +import se.scalablesolutions.akka.actor._ /** * @author Jonas Bonér @@ -56,20 +55,34 @@ final class MessageInvocation(val receiver: ActorRef, * @author Jonas Bonér */ trait MessageDispatcher extends MailboxFactory with Logging { - protected val uuids = new ConcurrentSkipListSet[Uuid] protected val guard = new ReentrantGuard + protected val active = new Switch(false) + /** + * Attaches the specified actorRef to this dispatcher + */ final def attach(actorRef: ActorRef): Unit = guard withGuard { register(actorRef) } + /** + * Detaches the specified actorRef from this dispatcher + */ final def detach(actorRef: ActorRef): Unit = guard withGuard { unregister(actorRef) } + private[akka] final def dispatchMessage(invocation: MessageInvocation): Unit = if (active.isOn) { + dispatch(invocation) + } else throw new IllegalActorStateException("Can't submit invocations to dispatcher since it's not started") + protected def register(actorRef: ActorRef) { - if (uuids.isEmpty()) start + if (uuids.isEmpty()) { + active.switchOn { + start + } + } if (actorRef.mailbox eq null) actorRef.mailbox = createMailbox(actorRef) uuids add actorRef.uuid } @@ -77,28 +90,52 @@ trait MessageDispatcher extends MailboxFactory with Logging { protected def unregister(actorRef: ActorRef) = { if (uuids remove actorRef.uuid) { actorRef.mailbox = null - if (uuids.isEmpty) shutdown // shut down in the dispatcher's references is zero + if (uuids.isEmpty){ + active switchOff { + shutdown // shut down in the dispatcher's references is zero + } + } } } + /** + * Traverses the list of actors (uuids) currently being attached to this dispatcher and stops those actors + */ def stopAllLinkedActors { val i = uuids.iterator while(i.hasNext()) { val uuid = i.next() ActorRegistry.actorFor(uuid) match { case Some(actor) => actor.stop - case None => log.warn("stopAllLinkedActors couldn't find linked actor: " + uuid) + case None => + log.error("stopAllLinkedActors couldn't find linked actor: " + uuid) } } - if(uuids.isEmpty) shutdown } - + + /** + * After the call to this method, the dispatcher mustn't begin any new message processing for the specified reference + */ def suspend(actorRef: ActorRef): Unit + + /* + * After the call to this method, the dispatcher must begin any new message processing for the specified reference + */ def resume(actorRef: ActorRef): Unit - def dispatch(invocation: MessageInvocation): Unit + /** + * Will be called when the dispatcher is to queue an invocation for execution + */ + protected def dispatch(invocation: MessageInvocation): Unit + /** + * Called one time every time an actor is attached to this dispatcher and this dispatcher was previously shutdown + */ protected def start: Unit + + /** + * Called one time every time an actor is detached from this dispatcher and this dispatcher has no actors left attached + */ protected def shutdown: Unit /** diff --git a/akka-actor/src/test/scala/dataflow/DataFlowSpec.scala b/akka-actor/src/test/scala/dataflow/DataFlowSpec.scala index d1f663e9f4..d596ecfac1 100644 --- a/akka-actor/src/test/scala/dataflow/DataFlowSpec.scala +++ b/akka-actor/src/test/scala/dataflow/DataFlowSpec.scala @@ -28,8 +28,8 @@ class DataFlowTest extends Spec with ShouldMatchers with BeforeAndAfterAll { val x, y, z = new DataFlowVariable[Int] thread { z << x() + y() - latch.countDown result.set(z()) + latch.countDown } thread { x << 40 } thread { y << 2 } From b745f9839d4346eb3d93075c21b8053ef77e60fb Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sun, 24 Oct 2010 16:31:19 +0200 Subject: [PATCH 06/39] Renamed stopAllLinkedActors to stopAllAttachedActors --- akka-actor/src/main/scala/dispatch/MessageHandling.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/dispatch/MessageHandling.scala index 95923ebfe5..919d4b674d 100644 --- a/akka-actor/src/main/scala/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/dispatch/MessageHandling.scala @@ -101,7 +101,7 @@ trait MessageDispatcher extends MailboxFactory with Logging { /** * Traverses the list of actors (uuids) currently being attached to this dispatcher and stops those actors */ - def stopAllLinkedActors { + def stopAllAttachedActors { val i = uuids.iterator while(i.hasNext()) { val uuid = i.next() From c90580a0265260eedae372a094d5c19467dbf485 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sun, 24 Oct 2010 21:39:39 +0200 Subject: [PATCH 07/39] =?UTF-8?q?Na=C3=AFve=20implementation=20of=20timeou?= =?UTF-8?q?t=20completed?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ExecutorBasedEventDrivenDispatcher.scala | 1 + .../main/scala/dispatch/MessageHandling.scala | 22 ++++++++++++++----- 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala index 45b1dfa886..5cbb6b3c8b 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala @@ -141,6 +141,7 @@ class ExecutorBasedEventDrivenDispatcher( } } + private[akka] def registerForExecution(mbox: MessageQueue with ExecutableMailbox): Unit = if (active.isOn) { if (mbox.suspended.isOff && mbox.dispatcherLock.tryLock()) { try { diff --git a/akka-actor/src/main/scala/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/dispatch/MessageHandling.scala index 919d4b674d..a7bc8165aa 100644 --- a/akka-actor/src/main/scala/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/dispatch/MessageHandling.scala @@ -7,6 +7,7 @@ package se.scalablesolutions.akka.dispatch import org.multiverse.commitbarriers.CountDownCommitBarrier import java.util.concurrent._ +import atomic. {AtomicReference, AtomicLong} import se.scalablesolutions.akka.util. {Switch, ReentrantGuard, Logging, HashCode} import se.scalablesolutions.akka.actor._ @@ -57,6 +58,7 @@ final class MessageInvocation(val receiver: ActorRef, trait MessageDispatcher extends MailboxFactory with Logging { protected val uuids = new ConcurrentSkipListSet[Uuid] protected val guard = new ReentrantGuard + private val scheduledShutdown = new AtomicReference[ScheduledFuture[AnyRef]](null) protected val active = new Switch(false) /** @@ -78,22 +80,30 @@ trait MessageDispatcher extends MailboxFactory with Logging { } else throw new IllegalActorStateException("Can't submit invocations to dispatcher since it's not started") protected def register(actorRef: ActorRef) { - if (uuids.isEmpty()) { + if (actorRef.mailbox eq null) actorRef.mailbox = createMailbox(actorRef) + if (uuids add actorRef.uuid) { + val future = scheduledShutdown.getAndSet(null) + if (future ne null) future.cancel(false) + } + if (active.isOff) { active.switchOn { start } } - if (actorRef.mailbox eq null) actorRef.mailbox = createMailbox(actorRef) - uuids add actorRef.uuid } protected def unregister(actorRef: ActorRef) = { if (uuids remove actorRef.uuid) { actorRef.mailbox = null if (uuids.isEmpty){ - active switchOff { - shutdown // shut down in the dispatcher's references is zero - } + val future = scheduledShutdown.getAndSet(Scheduler.scheduleOnce(() => guard withGuard { + if (uuids.isEmpty()) { + active switchOff { + shutdown // shut down in the dispatcher's references is zero + } + } + }, 1, TimeUnit.SECONDS)) + if (future ne null) future.cancel(false) } } } From a630caecf3830ebd4af0f181ae9f2e21cd365886 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 25 Oct 2010 00:01:31 +0200 Subject: [PATCH 08/39] Rewrote timed shutdown facility, causes less than 5% overhead now --- .../main/scala/dispatch/MessageHandling.scala | 51 ++++++++++++++----- 1 file changed, 37 insertions(+), 14 deletions(-) diff --git a/akka-actor/src/main/scala/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/dispatch/MessageHandling.scala index a7bc8165aa..dd55384151 100644 --- a/akka-actor/src/main/scala/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/dispatch/MessageHandling.scala @@ -7,7 +7,7 @@ package se.scalablesolutions.akka.dispatch import org.multiverse.commitbarriers.CountDownCommitBarrier import java.util.concurrent._ -import atomic. {AtomicReference, AtomicLong} +import atomic. {AtomicInteger, AtomicBoolean, AtomicReference, AtomicLong} import se.scalablesolutions.akka.util. {Switch, ReentrantGuard, Logging, HashCode} import se.scalablesolutions.akka.actor._ @@ -52,13 +52,19 @@ final class MessageInvocation(val receiver: ActorRef, } } +object MessageDispatcher { + val UNSCHEDULED = 0 + val SCHEDULED = 1 + val RESCHEDULED = 2 +} + /** * @author Jonas Bonér */ trait MessageDispatcher extends MailboxFactory with Logging { protected val uuids = new ConcurrentSkipListSet[Uuid] protected val guard = new ReentrantGuard - private val scheduledShutdown = new AtomicReference[ScheduledFuture[AnyRef]](null) + private val shutdownSchedule = new AtomicInteger(MessageDispatcher.UNSCHEDULED) protected val active = new Switch(false) /** @@ -81,10 +87,7 @@ trait MessageDispatcher extends MailboxFactory with Logging { protected def register(actorRef: ActorRef) { if (actorRef.mailbox eq null) actorRef.mailbox = createMailbox(actorRef) - if (uuids add actorRef.uuid) { - val future = scheduledShutdown.getAndSet(null) - if (future ne null) future.cancel(false) - } + uuids add actorRef.uuid if (active.isOff) { active.switchOn { start @@ -96,14 +99,14 @@ trait MessageDispatcher extends MailboxFactory with Logging { if (uuids remove actorRef.uuid) { actorRef.mailbox = null if (uuids.isEmpty){ - val future = scheduledShutdown.getAndSet(Scheduler.scheduleOnce(() => guard withGuard { - if (uuids.isEmpty()) { - active switchOff { - shutdown // shut down in the dispatcher's references is zero - } - } - }, 1, TimeUnit.SECONDS)) - if (future ne null) future.cancel(false) + shutdownSchedule.get() match { + case MessageDispatcher.UNSCHEDULED => + shutdownSchedule.set(MessageDispatcher.SCHEDULED) + Scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS) + case MessageDispatcher.SCHEDULED => + shutdownSchedule.set(MessageDispatcher.RESCHEDULED) + case MessageDispatcher.RESCHEDULED => //Already marked for reschedule + } } } } @@ -123,6 +126,26 @@ trait MessageDispatcher extends MailboxFactory with Logging { } } + private val shutdownAction = new Runnable { + def run = guard withGuard { + shutdownSchedule.get() match { + case MessageDispatcher.RESCHEDULED => + shutdownSchedule.set(MessageDispatcher.SCHEDULED) + Scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS) + case MessageDispatcher.SCHEDULED => + if (uuids.isEmpty()) { + active switchOff { + shutdown // shut down in the dispatcher's references is zero + } + } + shutdownSchedule.set(MessageDispatcher.UNSCHEDULED) + case MessageDispatcher.UNSCHEDULED => //Do nothing + } + } + } + + protected def timeoutMs: Long = 1000 + /** * After the call to this method, the dispatcher mustn't begin any new message processing for the specified reference */ From b075b80db9576d3df10c49a11fc531732cc06340 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 25 Oct 2010 00:38:48 +0200 Subject: [PATCH 09/39] Removed boilerplate, added final optmization --- .../main/scala/dispatch/MessageHandling.scala | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/akka-actor/src/main/scala/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/dispatch/MessageHandling.scala index dd55384151..dbbdc84b5c 100644 --- a/akka-actor/src/main/scala/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/dispatch/MessageHandling.scala @@ -62,9 +62,10 @@ object MessageDispatcher { * @author Jonas Bonér */ trait MessageDispatcher extends MailboxFactory with Logging { + import MessageDispatcher._ protected val uuids = new ConcurrentSkipListSet[Uuid] protected val guard = new ReentrantGuard - private val shutdownSchedule = new AtomicInteger(MessageDispatcher.UNSCHEDULED) + private var shutdownSchedule = UNSCHEDULED //This can be non-volatile since it is protected by guard withGuard protected val active = new Switch(false) /** @@ -99,13 +100,13 @@ trait MessageDispatcher extends MailboxFactory with Logging { if (uuids remove actorRef.uuid) { actorRef.mailbox = null if (uuids.isEmpty){ - shutdownSchedule.get() match { - case MessageDispatcher.UNSCHEDULED => - shutdownSchedule.set(MessageDispatcher.SCHEDULED) + shutdownSchedule match { + case UNSCHEDULED => + shutdownSchedule = SCHEDULED Scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS) - case MessageDispatcher.SCHEDULED => - shutdownSchedule.set(MessageDispatcher.RESCHEDULED) - case MessageDispatcher.RESCHEDULED => //Already marked for reschedule + case SCHEDULED => + shutdownSchedule = RESCHEDULED + case RESCHEDULED => //Already marked for reschedule } } } @@ -128,18 +129,18 @@ trait MessageDispatcher extends MailboxFactory with Logging { private val shutdownAction = new Runnable { def run = guard withGuard { - shutdownSchedule.get() match { - case MessageDispatcher.RESCHEDULED => - shutdownSchedule.set(MessageDispatcher.SCHEDULED) + shutdownSchedule match { + case RESCHEDULED => + shutdownSchedule = SCHEDULED Scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS) - case MessageDispatcher.SCHEDULED => + case SCHEDULED => if (uuids.isEmpty()) { active switchOff { shutdown // shut down in the dispatcher's references is zero } } - shutdownSchedule.set(MessageDispatcher.UNSCHEDULED) - case MessageDispatcher.UNSCHEDULED => //Do nothing + shutdownSchedule = UNSCHEDULED + case UNSCHEDULED => //Do nothing } } } From ceeaffd091a3925beed142df68133bda309f0c5e Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 25 Oct 2010 00:57:17 +0200 Subject: [PATCH 10/39] Fixing #492 --- akka-actor/src/main/scala/actor/Scheduler.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/akka-actor/src/main/scala/actor/Scheduler.scala b/akka-actor/src/main/scala/actor/Scheduler.scala index 5dc57056cc..6775479aa7 100644 --- a/akka-actor/src/main/scala/actor/Scheduler.scala +++ b/akka-actor/src/main/scala/actor/Scheduler.scala @@ -27,7 +27,7 @@ object Scheduler extends Logging { case class SchedulerException(msg: String, e: Throwable) extends RuntimeException(msg, e) - private var service = Executors.newSingleThreadScheduledExecutor(SchedulerThreadFactory) + @volatile private var service = Executors.newSingleThreadScheduledExecutor(SchedulerThreadFactory) log.info("Starting up Scheduler") @@ -108,12 +108,12 @@ object Scheduler extends Logging { } } - def shutdown = { + def shutdown: Unit = synchronized { log.info("Shutting down Scheduler") service.shutdown } - def restart = { + def restart: Unit = synchronized { log.info("Restarting Scheduler") shutdown service = Executors.newSingleThreadScheduledExecutor(SchedulerThreadFactory) From 253f77cc5691b1ef9402dafbefdd2bdedcb5c775 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 25 Oct 2010 01:46:42 +0200 Subject: [PATCH 11/39] Closing ticket #460 --- akka-actor/src/main/scala/actor/Actor.scala | 5 ++++- akka-actor/src/main/scala/actor/UntypedActor.scala | 6 ++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/actor/Actor.scala b/akka-actor/src/main/scala/actor/Actor.scala index c242a2318f..8087ecb889 100644 --- a/akka-actor/src/main/scala/actor/Actor.scala +++ b/akka-actor/src/main/scala/actor/Actor.scala @@ -15,6 +15,7 @@ import java.net.InetSocketAddress import scala.reflect.BeanProperty import se.scalablesolutions.akka.util. {ReflectiveAccess, Logging, Duration} +import se.scalablesolutions.akka.japi.Procedure /** * Implements the Transactor abstraction. E.g. a transactional actor. @@ -44,7 +45,9 @@ abstract class RemoteActor(address: InetSocketAddress) extends Actor { */ @serializable sealed trait LifeCycleMessage -case class HotSwap(code: Actor.Receive) extends LifeCycleMessage +case class HotSwap(code: Actor.Receive) extends LifeCycleMessage { + def this(behavior: Procedure[Any]) = this({ case msg => behavior.apply(msg) }: Actor.Receive) +} case object RevertHotSwap extends LifeCycleMessage diff --git a/akka-actor/src/main/scala/actor/UntypedActor.scala b/akka-actor/src/main/scala/actor/UntypedActor.scala index a1d724a566..91b93b7a88 100644 --- a/akka-actor/src/main/scala/actor/UntypedActor.scala +++ b/akka-actor/src/main/scala/actor/UntypedActor.scala @@ -11,6 +11,7 @@ import se.scalablesolutions.akka.config.Supervision._ import java.net.InetSocketAddress import scala.reflect.BeanProperty +import se.scalablesolutions.akka.japi.Procedure /** * Subclass this abstract class to create a MDB-style untyped actor. @@ -67,6 +68,11 @@ abstract class UntypedActor extends Actor { case msg => onReceive(msg) } + /** + * Java API for become + */ + def become(behavior: Procedure[Any]): Unit = super.become { case msg => behavior.apply(msg) } + @throws(classOf[Exception]) def onReceive(message: Any): Unit } From f948b637282709ad2248d042b153249be8a431c1 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 25 Oct 2010 12:51:13 +0200 Subject: [PATCH 12/39] Adding test harness for ActorModel (Dispatcher), work-in-progress --- .../ExecutorBasedEventDrivenDispatcher.scala | 6 +- ...sedEventDrivenWorkStealingDispatcher.scala | 10 +- .../main/scala/dispatch/HawtDispatcher.scala | 10 +- .../main/scala/dispatch/MailboxHandling.scala | 6 +- .../main/scala/dispatch/MessageHandling.scala | 12 +- .../test/scala/dispatch/ActorModelSpec.scala | 222 ++++++++++++++++++ 6 files changed, 244 insertions(+), 22 deletions(-) create mode 100644 akka-actor/src/test/scala/dispatch/ActorModelSpec.scala diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala index 5cbb6b3c8b..d565f4d55b 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala @@ -94,7 +94,7 @@ class ExecutorBasedEventDrivenDispatcher( private[akka] val threadFactory = new MonitorableThreadFactory(name) private[akka] val executorService = new AtomicReference[ExecutorService](config.createLazyExecutorService(threadFactory)) - protected def dispatch(invocation: MessageInvocation) = { + private[akka] def dispatch(invocation: MessageInvocation) = { val mbox = getMailbox(invocation.receiver) mbox enqueue invocation registerForExecution(mbox) @@ -131,9 +131,9 @@ class ExecutorBasedEventDrivenDispatcher( case JMSBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("JMSBasedDurableMailbox is not yet supported") } - protected def start= log.debug("Starting up %s\n\twith throughput [%d]", toString, throughput) + private[akka] def start = log.debug("Starting up %s\n\twith throughput [%d]", toString, throughput) - protected def shutdown { + private[akka] def shutdown { val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory)) if (old ne null) { log.debug("Shutting down %s", toString) diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala index f357ff4ab3..0f850be6f7 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala @@ -61,7 +61,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( override def mailboxSize(actorRef: ActorRef) = getMailbox(actorRef).size - protected def dispatch(invocation: MessageInvocation) { + private[akka] def dispatch(invocation: MessageInvocation) { val mbox = getMailbox(invocation.receiver) mbox enqueue invocation executorService.get() execute mbox @@ -167,9 +167,9 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( } else false } - protected def start = log.debug("Starting up %s",toString) + private[akka] def start = log.debug("Starting up %s",toString) - protected def shutdown { + private[akka] def shutdown { val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory)) if (old ne null) { log.debug("Shutting down %s", toString) @@ -190,7 +190,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( override val toString = "ExecutorBasedEventDrivenWorkStealingDispatcher[" + name + "]" - def createTransientMailbox(actorRef: ActorRef, mailboxType: TransientMailboxType): AnyRef = mailboxType match { + private[akka] def createTransientMailbox(actorRef: ActorRef, mailboxType: TransientMailboxType): AnyRef = mailboxType match { case UnboundedMailbox(blocking) => // FIXME make use of 'blocking' in work stealer ConcurrentLinkedDeque new ConcurrentLinkedDeque[MessageInvocation] with MessageQueue with Runnable { def enqueue(handle: MessageInvocation): Unit = this.add(handle) @@ -220,7 +220,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( /** * Creates and returns a durable mailbox for the given actor. */ - protected def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef = mailboxType match { + private[akka] def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef = mailboxType match { // FIXME make generic (work for TypedActor as well) case FileBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("FileBasedDurableMailbox is not yet supported for ExecutorBasedEventDrivenWorkStealingDispatcher") case ZooKeeperBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("ZooKeeperBasedDurableMailbox is not yet supported for ExecutorBasedEventDrivenWorkStealingDispatcher") diff --git a/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala b/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala index 50ae2da19e..8f001084b7 100644 --- a/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala @@ -144,11 +144,11 @@ class HawtDispatcher(val aggregate: Boolean = true, val parent: DispatchQueue = val mailboxType: Option[MailboxType] = None - protected def start { retainNonDaemon } + private[akka] def start { retainNonDaemon } - protected def shutdown { releaseNonDaemon } + private[akka] def shutdown { releaseNonDaemon } - protected def dispatch(invocation: MessageInvocation){ + private[akka] def dispatch(invocation: MessageInvocation){ mailbox(invocation.receiver).dispatch(invocation) } @@ -166,12 +166,12 @@ class HawtDispatcher(val aggregate: Boolean = true, val parent: DispatchQueue = def suspend(actorRef: ActorRef) = mailbox(actorRef).suspend def resume(actorRef:ActorRef) = mailbox(actorRef).resume - def createTransientMailbox(actorRef: ActorRef, mailboxType: TransientMailboxType): AnyRef = null.asInstanceOf[AnyRef] + private[akka] def createTransientMailbox(actorRef: ActorRef, mailboxType: TransientMailboxType): AnyRef = null.asInstanceOf[AnyRef] /** * Creates and returns a durable mailbox for the given actor. */ - protected def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef = null.asInstanceOf[AnyRef] + private[akka] def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef = null.asInstanceOf[AnyRef] override def toString = "HawtDispatcher" } diff --git a/akka-actor/src/main/scala/dispatch/MailboxHandling.scala b/akka-actor/src/main/scala/dispatch/MailboxHandling.scala index 68fbebb0e7..d363e76382 100644 --- a/akka-actor/src/main/scala/dispatch/MailboxHandling.scala +++ b/akka-actor/src/main/scala/dispatch/MailboxHandling.scala @@ -90,7 +90,7 @@ trait MailboxFactory { /** * Creates a MessageQueue (Mailbox) with the specified properties. */ - protected def createMailbox(actorRef: ActorRef): AnyRef = + private[akka] def createMailbox(actorRef: ActorRef): AnyRef = mailboxType.getOrElse(throw new IllegalStateException("No mailbox type defined")) match { case mb: TransientMailboxType => createTransientMailbox(actorRef, mb) case mb: DurableMailboxType => createDurableMailbox(actorRef, mb) @@ -99,10 +99,10 @@ trait MailboxFactory { /** * Creates and returns a transient mailbox for the given actor. */ - protected def createTransientMailbox(actorRef: ActorRef, mailboxType: TransientMailboxType): AnyRef + private[akka] def createTransientMailbox(actorRef: ActorRef, mailboxType: TransientMailboxType): AnyRef /** * Creates and returns a durable mailbox for the given actor. */ - protected def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef + private[akka] def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef } \ No newline at end of file diff --git a/akka-actor/src/main/scala/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/dispatch/MessageHandling.scala index dbbdc84b5c..9465d79897 100644 --- a/akka-actor/src/main/scala/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/dispatch/MessageHandling.scala @@ -86,7 +86,7 @@ trait MessageDispatcher extends MailboxFactory with Logging { dispatch(invocation) } else throw new IllegalActorStateException("Can't submit invocations to dispatcher since it's not started") - protected def register(actorRef: ActorRef) { + private[akka] def register(actorRef: ActorRef) { if (actorRef.mailbox eq null) actorRef.mailbox = createMailbox(actorRef) uuids add actorRef.uuid if (active.isOff) { @@ -96,7 +96,7 @@ trait MessageDispatcher extends MailboxFactory with Logging { } } - protected def unregister(actorRef: ActorRef) = { + private[akka] def unregister(actorRef: ActorRef) = { if (uuids remove actorRef.uuid) { actorRef.mailbox = null if (uuids.isEmpty){ @@ -145,7 +145,7 @@ trait MessageDispatcher extends MailboxFactory with Logging { } } - protected def timeoutMs: Long = 1000 + private[akka] def timeoutMs: Long = 1000 /** * After the call to this method, the dispatcher mustn't begin any new message processing for the specified reference @@ -160,17 +160,17 @@ trait MessageDispatcher extends MailboxFactory with Logging { /** * Will be called when the dispatcher is to queue an invocation for execution */ - protected def dispatch(invocation: MessageInvocation): Unit + private[akka] def dispatch(invocation: MessageInvocation): Unit /** * Called one time every time an actor is attached to this dispatcher and this dispatcher was previously shutdown */ - protected def start: Unit + private[akka] def start: Unit /** * Called one time every time an actor is detached from this dispatcher and this dispatcher has no actors left attached */ - protected def shutdown: Unit + private[akka] def shutdown: Unit /** * Returns the size of the mailbox for the specified actor diff --git a/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala b/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala new file mode 100644 index 0000000000..c79da02244 --- /dev/null +++ b/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala @@ -0,0 +1,222 @@ +/** + * Copyright (C) 2009-2010 Scalable Solutions AB + */ +package se.scalablesolutions.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 java.util.concurrent.atomic.AtomicLong +import java.util.concurrent. {ConcurrentHashMap, CountDownLatch, TimeUnit} +import se.scalablesolutions.akka.actor.dispatch.ActorModelSpec.MessageDispatcherInterceptor + +object ActorModelSpec { + + sealed trait ActorModelMessage + case class Reply_?(expect: Any) extends ActorModelMessage + case class Reply(expect: Any) extends ActorModelMessage + case class Forward(to: ActorRef,msg: Any) extends ActorModelMessage + case class CountDown(latch: CountDownLatch) extends ActorModelMessage + case class Increment(counter: AtomicLong) extends ActorModelMessage + case class Await(latch: CountDownLatch) extends ActorModelMessage + case class Meet(acknowledge: CountDownLatch, waitFor: CountDownLatch) extends ActorModelMessage + case object Restart extends ActorModelMessage + + val Ping = "Ping" + val Pong = "Pong" + + class DispatcherActor(dispatcher: MessageDispatcherInterceptor) extends Actor { + self.dispatcher = dispatcher.asInstanceOf[MessageDispatcher] + + def ack { dispatcher.getStats(self).msgsProcessed.incrementAndGet() } + + override def postRestart(reason: Throwable) { + dispatcher.getStats(self).restarts.incrementAndGet() + } + + def receive = { + case Await(latch) => ack; latch.await() + case Meet(sign, wait) => ack; sign.countDown(); wait.await() + case Reply(msg) => ack; self.reply(msg) + case Reply_?(msg) => ack; self.reply_?(msg) + case Forward(to,msg) => ack; to.forward(msg) + case CountDown(latch) => ack; latch.countDown() + case Increment(count) => ack; count.incrementAndGet() + case Restart => ack; throw new Exception("Restart requested") + } + } + + class InterceptorStats { + val suspensions = new AtomicLong(0) + val resumes = new AtomicLong(0) + val registers = new AtomicLong(0) + val unregisters = new AtomicLong(0) + val msgsReceived = new AtomicLong(0) + val msgsProcessed = new AtomicLong(0) + val restarts = new AtomicLong(0) + } + + trait MessageDispatcherInterceptor extends MessageDispatcher { + val stats = new ConcurrentHashMap[ActorRef,InterceptorStats] + val starts = new AtomicLong(0) + val stops = new AtomicLong(0) + + def getStats(actorRef: ActorRef) = { + stats.putIfAbsent(actorRef,new InterceptorStats) + stats.get(actorRef) + } + + abstract override def suspend(actorRef: ActorRef) { + super.suspend(actorRef) + getStats(actorRef).suspensions.incrementAndGet() + } + + abstract override def resume(actorRef: ActorRef) { + super.resume(actorRef) + getStats(actorRef).resumes.incrementAndGet() + } + + private[akka] abstract override def register(actorRef: ActorRef) { + super.register(actorRef) + getStats(actorRef).registers.incrementAndGet() + } + + private[akka] abstract override def unregister(actorRef: ActorRef) { + super.unregister(actorRef) + getStats(actorRef).unregisters.incrementAndGet() + } + + private[akka] abstract override def dispatch(invocation: MessageInvocation) { + super.dispatch(invocation) + getStats(invocation.receiver).msgsReceived.incrementAndGet() + } + + private[akka] abstract override def start { + super.start + starts.incrementAndGet() + } + + private[akka] abstract override def shutdown { + super.shutdown + stops.incrementAndGet() + } + } + + def assertDispatcher(dispatcher: MessageDispatcherInterceptor)( + starts: Long = dispatcher.starts.get(), + stops: Long = dispatcher.stops.get() + ) { + assert(starts === dispatcher.starts.get(), "Dispatcher starts") + assert(stops === dispatcher.stops.get(), "Dispatcher stops") + } + + def assertCountDown(latch: CountDownLatch,wait: Long,hint: AnyRef){ + assert(latch.await(wait,TimeUnit.MILLISECONDS) === true) + } + + def assertNoCountDown(latch: CountDownLatch,wait: Long,hint: AnyRef){ + assert(latch.await(wait,TimeUnit.MILLISECONDS) === false) + } + + def statsFor(actorRef: ActorRef, dispatcher: MessageDispatcher = null) = + dispatcher.asInstanceOf[MessageDispatcherInterceptor].getStats(actorRef) + + def assertRefDefaultZero(actorRef: ActorRef,dispatcher: MessageDispatcher = null)( + suspensions: Long = 0, + resumes: Long = 0, + registers: Long = 0, + unregisters: Long = 0, + msgsReceived: Long = 0, + msgsProcessed: Long = 0, + restarts: Long = 0) { + assertRef(actorRef,dispatcher)( + suspensions, + resumes, + registers, + unregisters, + msgsReceived, + msgsProcessed, + restarts + ) + } + + def assertRef(actorRef: ActorRef,dispatcher: MessageDispatcher = null)( + suspensions: Long = statsFor(actorRef).suspensions.get(), + resumes: Long = statsFor(actorRef).resumes.get(), + registers: Long = statsFor(actorRef).registers.get(), + unregisters: Long = statsFor(actorRef).unregisters.get(), + msgsReceived: Long = statsFor(actorRef).msgsReceived.get(), + msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(), + restarts: Long = statsFor(actorRef).restarts.get() + ) { + val stats = statsFor(actorRef,if (dispatcher eq null) actorRef.dispatcher else dispatcher) + assert(stats.suspensions.get() === suspensions, "Suspensions") + assert(stats.resumes.get() === resumes, "Resumes") + assert(stats.registers.get() === registers, "Registers") + assert(stats.unregisters.get() === unregisters, "Unregisters") + assert(stats.msgsReceived.get() === msgsReceived, "Received") + assert(stats.msgsProcessed.get() === msgsProcessed, "Processed") + assert(stats.restarts.get() === restarts, "Restarts") + } + + def newTestActor(implicit d: MessageDispatcherInterceptor) = actorOf(new DispatcherActor(d)) +} + +abstract class ActorModelSpec extends JUnitSuite { + import ActorModelSpec._ + + protected def newInterceptedDispatcher: MessageDispatcherInterceptor + + @Test def dispatcherShouldDynamicallyHandleItsOwnLifeCycle { + implicit val dispatcher = newInterceptedDispatcher + val a = newTestActor + assertDispatcher(dispatcher)(starts = 0, stops = 0) + a.start + assertDispatcher(dispatcher)(starts = 1, stops = 0) + a.stop + Thread.sleep(dispatcher.timeoutMs + 100) + assertDispatcher(dispatcher)(starts = 1, stops = 1) + assertRef(a,dispatcher)( + suspensions = 0, + resumes = 0, + registers = 1, + unregisters = 1, + msgsReceived = 0, + msgsProcessed = 0, + restarts = 0 + ) + } + + @Test def dispatcherShouldProcessMessagesOneAtATime { + implicit val dispatcher = newInterceptedDispatcher + val a = newTestActor + val start,step1,step2,oneAtATime = new CountDownLatch(1) + val counter = new AtomicLong(0) + a.start + + a ! CountDown(start) + assertCountDown(start,3000, "Should process first message within 3 seconds") + assertRefDefaultZero(a)(registers = 1, msgsReceived = 1, msgsProcessed = 1) + + a ! Meet(step1,step2) + assertCountDown(step1,3000, "Didn't process the Meet message in 3 seocnds") + assertRefDefaultZero(a)(registers = 1, msgsReceived = 2, msgsProcessed = 2) + + a ! CountDown(oneAtATime) + assertNoCountDown(oneAtATime,500,"Processed message when not allowed to") + step2.countDown() + assertCountDown(oneAtATime,500,"Processed message when allowed") + assertRefDefaultZero(a)(registers = 1, msgsReceived = 3, msgsProcessed = 3) + + a.stop + assertRefDefaultZero(a)(registers = 1, unregisters = 1, msgsReceived = 3, msgsProcessed = 3) + } +} + +class ExecutorBasedEventDrivenDispatcherModelTest extends ActorModelSpec { + def newInterceptedDispatcher = + new ExecutorBasedEventDrivenDispatcher("foo") with MessageDispatcherInterceptor +} \ No newline at end of file From 58a7eb71e749d3bc328e065652cdebfdf7163b10 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 25 Oct 2010 13:09:14 +0200 Subject: [PATCH 13/39] Added test for dispatcher parallelism --- .../test/scala/dispatch/ActorModelSpec.scala | 25 ++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) diff --git a/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala b/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala index c79da02244..ae27faba09 100644 --- a/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala @@ -165,7 +165,7 @@ object ActorModelSpec { def newTestActor(implicit d: MessageDispatcherInterceptor) = actorOf(new DispatcherActor(d)) } -abstract class ActorModelSpec extends JUnitSuite { +abstract class ActorModelSpec(val supportsMoreThanOneActor: Boolean) extends JUnitSuite { import ActorModelSpec._ protected def newInterceptedDispatcher: MessageDispatcherInterceptor @@ -194,7 +194,6 @@ abstract class ActorModelSpec extends JUnitSuite { implicit val dispatcher = newInterceptedDispatcher val a = newTestActor val start,step1,step2,oneAtATime = new CountDownLatch(1) - val counter = new AtomicLong(0) a.start a ! CountDown(start) @@ -207,6 +206,8 @@ abstract class ActorModelSpec extends JUnitSuite { a ! CountDown(oneAtATime) assertNoCountDown(oneAtATime,500,"Processed message when not allowed to") + assertRefDefaultZero(a)(registers = 1, msgsReceived = 3, msgsProcessed = 2) + step2.countDown() assertCountDown(oneAtATime,500,"Processed message when allowed") assertRefDefaultZero(a)(registers = 1, msgsReceived = 3, msgsProcessed = 3) @@ -214,9 +215,27 @@ abstract class ActorModelSpec extends JUnitSuite { a.stop assertRefDefaultZero(a)(registers = 1, unregisters = 1, msgsReceived = 3, msgsProcessed = 3) } + + @Test def dispatcherShouldProcessMessagesInParallel: Unit = if (supportsMoreThanOneActor) { + implicit val dispatcher = newInterceptedDispatcher + val a, b = newTestActor.start + val aStart,aStop,bParallel = new CountDownLatch(1) + + a ! Meet(aStart,aStop) + assertCountDown(aStart,3000, "Should process first message within 3 seconds") + + b ! CountDown(bParallel) + assertCountDown(bParallel, 3000, "Should process other actors in parallel") + + aStop.countDown() + a.stop() + b.stop() + assertRefDefaultZero(a)(registers = 1, unregisters = 1, msgsReceived = 1, msgsProcessed = 1) + assertRefDefaultZero(b)(registers = 1, unregisters = 1, msgsReceived = 1, msgsProcessed = 1) + } } -class ExecutorBasedEventDrivenDispatcherModelTest extends ActorModelSpec { +class ExecutorBasedEventDrivenDispatcherModelTest extends ActorModelSpec(supportsMoreThanOneActor = true) { def newInterceptedDispatcher = new ExecutorBasedEventDrivenDispatcher("foo") with MessageDispatcherInterceptor } \ No newline at end of file From 5e984d22c3c1f045e77cb701b5eeb486d2ee913a Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 25 Oct 2010 13:32:40 +0200 Subject: [PATCH 14/39] Added tests for suspend/resume --- .../test/scala/dispatch/ActorModelSpec.scala | 35 +++++++++++++++++-- 1 file changed, 33 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala b/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala index ae27faba09..dcb3ed8270 100644 --- a/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala @@ -228,11 +228,42 @@ abstract class ActorModelSpec(val supportsMoreThanOneActor: Boolean) extends JUn assertCountDown(bParallel, 3000, "Should process other actors in parallel") aStop.countDown() - a.stop() - b.stop() + a.stop + b.stop assertRefDefaultZero(a)(registers = 1, unregisters = 1, msgsReceived = 1, msgsProcessed = 1) assertRefDefaultZero(b)(registers = 1, unregisters = 1, msgsReceived = 1, msgsProcessed = 1) } + + @Test def dispatcherShouldSuspendAndResumeAFailingNonSupervisedPermanentActor { + implicit val dispatcher = newInterceptedDispatcher + val a = newTestActor.start + val done = new CountDownLatch(1) + a ! Restart + a ! CountDown(done) + assertCountDown(done, 3000, "Should be suspended+resumed and done with next message within 3 seconds") + a.stop + assertRefDefaultZero(a)(registers = 1,unregisters = 1, msgsReceived = 2, + msgsProcessed = 2, suspensions = 1, resumes = 1) + } + + @Test def dispatcherShouldNotProcessMessagesForASuspendedActor { + implicit val dispatcher = newInterceptedDispatcher + val a = newTestActor.start + val done = new CountDownLatch(1) + dispatcher.suspend(a) + a ! CountDown(done) + assertNoCountDown(done, 1000, "Should not process messages while suspended") + assertRefDefaultZero(a)(registers = 1, msgsReceived = 1, suspensions = 1) + + dispatcher.resume(a) + assertCountDown(done, 3000, "Should resume processing of messages when resumed") + assertRefDefaultZero(a)(registers = 1, msgsReceived = 1, msgsProcessed = 1, + suspensions = 1, resumes = 1) + + a.stop + assertRefDefaultZero(a)(registers = 1,unregisters = 1, msgsReceived = 1, msgsProcessed = 1, + suspensions = 1, resumes = 1) + } } class ExecutorBasedEventDrivenDispatcherModelTest extends ActorModelSpec(supportsMoreThanOneActor = true) { From 2c4304fdc91ee5e93b8d79e55875d453ad8411f9 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 25 Oct 2010 13:52:27 +0200 Subject: [PATCH 15/39] Added the ActorModel specification to HawtDispatcher and EBEDWSD --- ...torBasedEventDrivenWorkStealingDispatcher.scala | 4 ++-- .../src/test/scala/dispatch/ActorModelSpec.scala | 14 +++++++++++--- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala index 0f850be6f7..933139ceda 100644 --- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala @@ -230,13 +230,13 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher( case JMSBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("JMSBasedDurableMailbox is not yet supported for ExecutorBasedEventDrivenWorkStealingDispatcher") } - override def register(actorRef: ActorRef) = { + private[akka] override def register(actorRef: ActorRef) = { verifyActorsAreOfSameType(actorRef) pooledActors add actorRef super.register(actorRef) } - override def unregister(actorRef: ActorRef) = { + private[akka] override def unregister(actorRef: ActorRef) = { pooledActors remove actorRef super.unregister(actorRef) } diff --git a/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala b/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala index dcb3ed8270..efe362d5c8 100644 --- a/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala @@ -165,7 +165,7 @@ object ActorModelSpec { def newTestActor(implicit d: MessageDispatcherInterceptor) = actorOf(new DispatcherActor(d)) } -abstract class ActorModelSpec(val supportsMoreThanOneActor: Boolean) extends JUnitSuite { +abstract class ActorModelSpec extends JUnitSuite { import ActorModelSpec._ protected def newInterceptedDispatcher: MessageDispatcherInterceptor @@ -216,7 +216,7 @@ abstract class ActorModelSpec(val supportsMoreThanOneActor: Boolean) extends JUn assertRefDefaultZero(a)(registers = 1, unregisters = 1, msgsReceived = 3, msgsProcessed = 3) } - @Test def dispatcherShouldProcessMessagesInParallel: Unit = if (supportsMoreThanOneActor) { + @Test def dispatcherShouldProcessMessagesInParallel: Unit = { implicit val dispatcher = newInterceptedDispatcher val a, b = newTestActor.start val aStart,aStop,bParallel = new CountDownLatch(1) @@ -266,7 +266,15 @@ abstract class ActorModelSpec(val supportsMoreThanOneActor: Boolean) extends JUn } } -class ExecutorBasedEventDrivenDispatcherModelTest extends ActorModelSpec(supportsMoreThanOneActor = true) { +class ExecutorBasedEventDrivenDispatcherModelTest extends ActorModelSpec { def newInterceptedDispatcher = new ExecutorBasedEventDrivenDispatcher("foo") with MessageDispatcherInterceptor +} + +class HawtDispatcherModelTest extends ActorModelSpec { + def newInterceptedDispatcher = new HawtDispatcher(false) with MessageDispatcherInterceptor +} + +class ExecutorBasedEventDrivenWorkStealingDispatcherModelTest extends ActorModelSpec { + def newInterceptedDispatcher = new ExecutorBasedEventDrivenWorkStealingDispatcher("foo") with MessageDispatcherInterceptor } \ No newline at end of file From 7850f0bb68895927839c71f81f3800ab297f6cce Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 25 Oct 2010 14:26:12 +0200 Subject: [PATCH 16/39] Adding a flooding test to reproduce error reported by user --- .../test/scala/dispatch/ActorModelSpec.scala | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala b/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala index efe362d5c8..1761f0195a 100644 --- a/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ActorModelSpec.scala @@ -23,6 +23,7 @@ object ActorModelSpec { case class Increment(counter: AtomicLong) extends ActorModelMessage case class Await(latch: CountDownLatch) extends ActorModelMessage case class Meet(acknowledge: CountDownLatch, waitFor: CountDownLatch) extends ActorModelMessage + case class CountDownNStop(latch: CountDownLatch) extends ActorModelMessage case object Restart extends ActorModelMessage val Ping = "Ping" @@ -45,6 +46,7 @@ object ActorModelSpec { case Forward(to,msg) => ack; to.forward(msg) case CountDown(latch) => ack; latch.countDown() case Increment(count) => ack; count.incrementAndGet() + case CountDownNStop(l)=> ack; l.countDown; self.stop case Restart => ack; throw new Exception("Restart requested") } } @@ -264,6 +266,23 @@ abstract class ActorModelSpec extends JUnitSuite { assertRefDefaultZero(a)(registers = 1,unregisters = 1, msgsReceived = 1, msgsProcessed = 1, suspensions = 1, resumes = 1) } + + @Test def dispatcherShouldHandleWavesOfActors { + implicit val dispatcher = newInterceptedDispatcher + + def flood(num: Int) { + val cachedMessage = CountDownNStop(new CountDownLatch(num)) + (1 to num) foreach { + _ => newTestActor.start ! cachedMessage + } + assertCountDown(cachedMessage.latch,10000, "Should process " + num + " countdowns") + } + for(run <- 1 to 3) { + flood(10000) + Thread.sleep(dispatcher.timeoutMs * 2) + assertDispatcher(dispatcher)(starts = run, stops = run) + } + } } class ExecutorBasedEventDrivenDispatcherModelTest extends ActorModelSpec { From fde1baa9a62407aafa7e5e412678fee8c8cd2f21 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 25 Oct 2010 15:32:54 +0200 Subject: [PATCH 17/39] Closing ticket #471 --- .../src/main/scala/actor/TypedActor.scala | 13 ++++++++++++- .../akka/actor/SimpleJavaPojo.java | 2 ++ .../akka/actor/SimpleJavaPojoImpl.java | 5 +++++ .../scala/actor/typed-actor/TypedActorSpec.scala | 6 ++++++ 4 files changed, 25 insertions(+), 1 deletion(-) diff --git a/akka-typed-actor/src/main/scala/actor/TypedActor.scala b/akka-typed-actor/src/main/scala/actor/TypedActor.scala index 0aeb127fdc..69e7d57c48 100644 --- a/akka-typed-actor/src/main/scala/actor/TypedActor.scala +++ b/akka-typed-actor/src/main/scala/actor/TypedActor.scala @@ -24,6 +24,9 @@ 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, + * 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. * * Here is an example of usage (in Java): *

@@ -731,6 +734,9 @@ object TypedActor extends Logging {
   private[akka] def returnsFuture_?(methodRtti: MethodRtti): Boolean =
     classOf[Future[_]].isAssignableFrom(methodRtti.getMethod.getReturnType)
 
+  private[akka] def returnsOption_?(methodRtti: MethodRtti): Boolean =
+    classOf[se.scalablesolutions.akka.japi.Option[_]].isAssignableFrom(methodRtti.getMethod.getReturnType)
+
   private[akka] def supervise(faultHandlingStrategy: FaultHandlingStrategy, components: List[Supervise]): Supervisor =
     Supervisor(SupervisorConfig(faultHandlingStrategy, components))
 
@@ -818,7 +824,12 @@ 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}
+      (actorRef.!!(joinPoint, timeout)(senderActorRef)).as[JOption[AnyRef]] match {
+        case None => JOption.none[AnyRef]
+        case Some(x) => if(x.isDefined) x else JOption.some[AnyRef](null)
+      }
     } else {
       val result = (actorRef.!!(joinPoint, timeout)(senderActorRef)).as[AnyRef]
       if (result.isDefined) result.get
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 d3a18abbd9..8cc029aa89 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
@@ -3,11 +3,13 @@ package se.scalablesolutions.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;
 
 public interface SimpleJavaPojo {
   public Object getSender();
   public Object getSenderFuture();
   public Future square(int value);
+  public Option middleName();
   public void setName(String name);
   public String getName();
   public void throwException();
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 c02d266ce8..219d3806d5 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
@@ -3,6 +3,7 @@ package se.scalablesolutions.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;
 
 public class SimpleJavaPojoImpl extends TypedActor implements SimpleJavaPojo {
 
@@ -29,6 +30,10 @@ public class SimpleJavaPojoImpl extends TypedActor implements SimpleJavaPojo {
     return getContext().getSenderFuture().get();
   }
 
+  public Option middleName() {
+      return Option.some("foo");
+  }
+
   public void setName(String name) {
     this.name = name;
   }
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 13c8c8e1fa..ff23a8601c 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
@@ -10,6 +10,7 @@ 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 se.scalablesolutions.akka.dispatch.DefaultCompletableFuture
 import TypedActorSpec._
@@ -79,6 +80,11 @@ class TypedActorSpec extends
       future.result.get should equal (100)
     }
 
+    it("should return none instead of exception") {
+      val middleName = simplePojo.middleName();
+      assert(middleName === Option.some("foo"))
+    }
+
     it("should accept constructor arguments") {
       val pojo1 = TypedActor.newInstance(classOf[MyTypedActor], new MyTypedActorWithConstructorArgsImpl("test", 1L))
       assert(pojo1.sendRequestReply("hello") === "hello test 1")

From 8ce57c0bc012006498544fac3e3dc30c6a5fb753 Mon Sep 17 00:00:00 2001
From: Viktor Klang 
Date: Mon, 25 Oct 2010 16:12:10 +0200
Subject: [PATCH 18/39] added more tests and fixed corner case to TypedActor
 Option return value

---
 akka-typed-actor/src/main/scala/actor/TypedActor.scala   | 3 ++-
 .../se/scalablesolutions/akka/actor/SimpleJavaPojo.java  | 2 +-
 .../scalablesolutions/akka/actor/SimpleJavaPojoImpl.java | 4 ++--
 .../test/scala/actor/typed-actor/TypedActorSpec.scala    | 9 +++++++--
 4 files changed, 12 insertions(+), 6 deletions(-)

diff --git a/akka-typed-actor/src/main/scala/actor/TypedActor.scala b/akka-typed-actor/src/main/scala/actor/TypedActor.scala
index 69e7d57c48..279c5b4d4e 100644
--- a/akka-typed-actor/src/main/scala/actor/TypedActor.scala
+++ b/akka-typed-actor/src/main/scala/actor/TypedActor.scala
@@ -828,7 +828,8 @@ private[akka] abstract class ActorAspect {
         import se.scalablesolutions.akka.japi.{Option => JOption}
       (actorRef.!!(joinPoint, timeout)(senderActorRef)).as[JOption[AnyRef]] match {
         case None => JOption.none[AnyRef]
-        case Some(x) => if(x.isDefined) x else JOption.some[AnyRef](null)
+        case Some(x) if ((x eq null) || x.isEmpty) => JOption.some[AnyRef](null)
+        case Some(x) if(x.isDefined) => x
       }
     } else {
       val result = (actorRef.!!(joinPoint, timeout)(senderActorRef)).as[AnyRef]
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 8cc029aa89..f4aafa6e1d 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
@@ -9,7 +9,7 @@ public interface SimpleJavaPojo {
   public Object getSender();
   public Object getSenderFuture();
   public Future square(int value);
-  public Option middleName();
+  public Option passThru(Option returnValue);
   public void setName(String name);
   public String getName();
   public void throwException();
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 219d3806d5..103d84de2d 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
@@ -30,8 +30,8 @@ public class SimpleJavaPojoImpl extends TypedActor implements SimpleJavaPojo {
     return getContext().getSenderFuture().get();
   }
 
-  public Option middleName() {
-      return Option.some("foo");
+  public Option passThru(Option returnValue) {
+      return returnValue;
   }
 
   public void setName(String name) {
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 ff23a8601c..219f96f3e2 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
@@ -81,8 +81,13 @@ class TypedActorSpec extends
     }
 
     it("should return none instead of exception") {
-      val middleName = simplePojo.middleName();
-      assert(middleName === Option.some("foo"))
+      val someVal = Option.some("foo")
+      val noneVal = Option.none[String]
+      val nullVal = null:Option[String]
+
+      assert(simplePojo.passThru(someVal) === someVal)
+      assert(simplePojo.passThru(noneVal) === Option.some(null))
+      assert(simplePojo.passThru(nullVal) === Option.some(null))
     }
 
     it("should accept constructor arguments") {

From 5354f776353a8b3a0c561e808db0c19855432e3c Mon Sep 17 00:00:00 2001
From: Viktor Klang 
Date: Mon, 25 Oct 2010 16:28:18 +0200
Subject: [PATCH 19/39] Updating Netty to 3.2.3, closing ticket #495

---
 project/build/AkkaProject.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/project/build/AkkaProject.scala b/project/build/AkkaProject.scala
index 444871306a..5b93e30044 100644
--- a/project/build/AkkaProject.scala
+++ b/project/build/AkkaProject.scala
@@ -209,7 +209,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
 
     lazy val multiverse = "org.multiverse" % "multiverse-alpha" % MULTIVERSE_VERSION % "compile" intransitive
 
-    lazy val netty = "org.jboss.netty" % "netty" % "3.2.2.Final" % "compile"
+    lazy val netty = "org.jboss.netty" % "netty" % "3.2.3.Final" % "compile"
 
     lazy val protobuf = "com.google.protobuf" % "protobuf-java" % "2.3.0" % "compile"
 

From bd364a204aee5c01ee0c52d48abf24120016ba7b Mon Sep 17 00:00:00 2001
From: Viktor Klang 
Date: Mon, 25 Oct 2010 17:15:32 +0200
Subject: [PATCH 20/39] Handling Interrupts for ThreadBasedDispatcher, EBEDD
 and EBEDWSD

---
 .../scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala   | 2 +-
 .../ExecutorBasedEventDrivenWorkStealingDispatcher.scala      | 4 +++-
 2 files changed, 4 insertions(+), 2 deletions(-)

diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala
index d565f4d55b..3622984e69 100644
--- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala
+++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala
@@ -178,7 +178,7 @@ trait ExecutableMailbox extends Runnable { self: MessageQueue =>
   
   final def run = {
     val reschedule = try {
-      processMailbox()
+      try { processMailbox() } catch { case ie: InterruptedException => true }
     } finally {
       dispatcherLock.unlock()
     }
diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
index 933139ceda..9e95384cf5 100644
--- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
+++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
@@ -95,7 +95,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(
    * Process the messages in the mailbox of the given actor.
    * @return
    */
-  private def processMailbox(mailbox: MessageQueue): Boolean = {
+  private def processMailbox(mailbox: MessageQueue): Boolean = try {
     if (mailbox.suspended.isOn)
         return false
 
@@ -107,6 +107,8 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(
       messageInvocation = mailbox.dequeue
     }
     true
+  } catch {
+    case ie: InterruptedException => false
   }
 
   private def findThief(receiver: ActorRef): Option[ActorRef] = {

From a35dccd3a84601f72a75cabaef41fce6625e5a1a Mon Sep 17 00:00:00 2001
From: Viktor Klang 
Date: Mon, 25 Oct 2010 18:20:29 +0200
Subject: [PATCH 21/39] Making ThreadBasedDispatcher Unbounded if no capacity
 specced and fix a possible mem leak in it

---
 .../src/main/scala/dispatch/Dispatchers.scala |  4 ++-
 .../dispatch/ThreadBasedDispatcher.scala      | 30 ++++++++++++-------
 .../src/main/scala/remote/RemoteServer.scala  |  2 +-
 3 files changed, 24 insertions(+), 12 deletions(-)

diff --git a/akka-actor/src/main/scala/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/dispatch/Dispatchers.scala
index d7592d49b7..cd06fd8857 100644
--- a/akka-actor/src/main/scala/dispatch/Dispatchers.scala
+++ b/akka-actor/src/main/scala/dispatch/Dispatchers.scala
@@ -77,11 +77,12 @@ object Dispatchers extends Logging {
    * 

* E.g. each actor consumes its own thread. */ - def newThreadBasedDispatcher(actor: ActorRef) = new ThreadBasedDispatcher(actor, BoundedMailbox(true)) + def newThreadBasedDispatcher(actor: ActorRef) = new ThreadBasedDispatcher(actor) /** * Creates an thread based dispatcher serving a single actor through the same single thread. * Uses the default timeout + * If capacity is negative, it's Integer.MAX_VALUE *

* E.g. each actor consumes its own thread. */ @@ -89,6 +90,7 @@ object Dispatchers extends Logging { /** * Creates an thread based dispatcher serving a single actor through the same single thread. + * If capacity is negative, it's Integer.MAX_VALUE *

* E.g. each actor consumes its own thread. */ diff --git a/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala b/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala index 247399818f..8ec4421c3e 100644 --- a/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala +++ b/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala @@ -10,30 +10,40 @@ import se.scalablesolutions.akka.util.Duration import java.util.Queue import java.util.concurrent.{ConcurrentLinkedQueue, BlockingQueue, TimeUnit, LinkedBlockingQueue} +import se.scalablesolutions.akka.actor +import java.util.concurrent.atomic.AtomicReference /** * Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue. * * @author Jonas Bonér */ -class ThreadBasedDispatcher(private val actor: ActorRef, _mailboxType: MailboxType) +class ThreadBasedDispatcher(_actor: ActorRef, _mailboxType: MailboxType) extends ExecutorBasedEventDrivenDispatcher( - actor.getClass.getName + ":" + actor.uuid, - Dispatchers.THROUGHPUT, - -1, - _mailboxType, - ThreadBasedDispatcher.oneThread) { + _actor.uuid.toString,Dispatchers.THROUGHPUT,-1,_mailboxType,ThreadBasedDispatcher.oneThread) { - def this(actor: ActorRef) = this(actor, BoundedMailbox(true)) // For Java API + private[akka] val owner = new AtomicReference[ActorRef](_actor) - def this(actor: ActorRef, capacity: Int) = this(actor, BoundedMailbox(true, capacity)) + def this(actor: ActorRef) = + this(actor, UnboundedMailbox(true)) // For Java API - def this(actor: ActorRef, capacity: Int, pushTimeOut: Duration) = this(actor, BoundedMailbox(true, capacity, pushTimeOut)) + def this(actor: ActorRef, capacity: Int) = + this(actor, BoundedMailbox(true, capacity)) //For Java API + + def this(actor: ActorRef, capacity: Int, pushTimeOut: Duration) = //For Java API + this(actor, BoundedMailbox(true, capacity, pushTimeOut)) override def register(actorRef: ActorRef) = { - if (actorRef != actor) throw new IllegalArgumentException("Cannot register to anyone but " + actor) + val actor = owner.get() + if ((actor ne null) && actorRef != actor) throw new IllegalArgumentException("Cannot register to anyone but " + actor) + owner.compareAndSet(null,actorRef) //Register if unregistered super.register(actorRef) } + + override def unregister(actorRef: ActorRef) = { + super.unregister(actorRef) + owner.compareAndSet(actorRef,null) //Unregister (prevent memory leak) + } } object ThreadBasedDispatcher { diff --git a/akka-remote/src/main/scala/remote/RemoteServer.scala b/akka-remote/src/main/scala/remote/RemoteServer.scala index deb3c05b87..f3c29c62f9 100644 --- a/akka-remote/src/main/scala/remote/RemoteServer.scala +++ b/akka-remote/src/main/scala/remote/RemoteServer.scala @@ -618,7 +618,7 @@ class RemoteServerHandler( log.info("Creating a new remote actor [%s:%s]", name, uuid) val clazz = if (applicationLoader.isDefined) applicationLoader.get.loadClass(name) else Class.forName(name) - val actorRef = Actor.actorOf(clazz.newInstance.asInstanceOf[Actor]) + val actorRef = Actor.actorOf(clazz.asInstanceOf[Class[_ <: Actor]]) actorRef.uuid = uuidFrom(uuid.getHigh,uuid.getLow) actorRef.id = id actorRef.timeout = timeout From 5c9fab80d357ce94f60a0c454b47aa0243d3805d Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 25 Oct 2010 18:21:22 +0200 Subject: [PATCH 22/39] Fixing a cranky compiler whine on a match statement --- akka-typed-actor/src/main/scala/actor/TypedActor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-typed-actor/src/main/scala/actor/TypedActor.scala b/akka-typed-actor/src/main/scala/actor/TypedActor.scala index 279c5b4d4e..762557d5bd 100644 --- a/akka-typed-actor/src/main/scala/actor/TypedActor.scala +++ b/akka-typed-actor/src/main/scala/actor/TypedActor.scala @@ -829,7 +829,7 @@ private[akka] abstract class ActorAspect { (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) - case Some(x) if(x.isDefined) => x + case Some(x) => x } } else { val result = (actorRef.!!(joinPoint, timeout)(senderActorRef)).as[AnyRef] From d87a71506633531fb369bb8667fe4521fd4f8cde Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bon=C3=A9r?= Date: Tue, 26 Oct 2010 09:29:04 +0200 Subject: [PATCH 23/39] Fixed bug in startLink and friends + Added cryptographically secure cookie generator --- .../src/main/scala/actor/ActorRef.scala | 49 ++++++------------- akka-actor/src/main/scala/util/Crypt.scala | 36 ++++++++++++++ akka-actor/src/main/scala/util/Helpers.scala | 18 +------ .../src/main/scala/RedisStorageBackend.scala | 2 +- scripts/generate_secure_cookie.sh | 27 ++++++++++ 5 files changed, 81 insertions(+), 51 deletions(-) create mode 100644 akka-actor/src/main/scala/util/Crypt.scala create mode 100755 scripts/generate_secure_cookie.sh diff --git a/akka-actor/src/main/scala/actor/ActorRef.scala b/akka-actor/src/main/scala/actor/ActorRef.scala index 7a6e493f43..48552a6a6f 100644 --- a/akka-actor/src/main/scala/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/actor/ActorRef.scala @@ -787,7 +787,7 @@ class LocalActorRef private[akka] ( _status = ActorRefInternals.RUNNING - //If we are not currently creating this ActorRef instance + // If we are not currently creating this ActorRef instance if ((actorInstance ne null) && (actorInstance.get ne null)) initializeActorInstance @@ -854,11 +854,8 @@ class LocalActorRef private[akka] ( * To be invoked from within the actor itself. */ def startLink(actorRef: ActorRef): Unit = guard.withGuard { - try { - link(actorRef) - } finally { - actorRef.start - } + link(actorRef) + actorRef.start } /** @@ -868,12 +865,9 @@ class LocalActorRef private[akka] ( */ def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int): Unit = guard.withGuard { ensureRemotingEnabled - try { - actorRef.makeRemote(hostname, port) - link(actorRef) - } finally { - actorRef.start - } + actorRef.makeRemote(hostname, port) + link(actorRef) + actorRef.start } /** @@ -905,11 +899,8 @@ class LocalActorRef private[akka] ( */ def spawnLink(clazz: Class[_ <: Actor]): ActorRef = guard.withGuard { val actor = Actor.actorOf(clazz) - try { - link(actor) - } finally { - actor.start - } + link(actor) + actor.start actor } @@ -921,12 +912,9 @@ class LocalActorRef private[akka] ( def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = guard.withGuard { ensureRemotingEnabled val actor = Actor.actorOf(clazz) - try { - actor.makeRemote(hostname, port) - link(actor) - } finally { - actor.start - } + actor.makeRemote(hostname, port) + link(actor) + actor.start actor } @@ -995,8 +983,7 @@ class LocalActorRef private[akka] ( * Callback for the dispatcher. This is the single entry point to the user Actor implementation. */ protected[akka] def invoke(messageHandle: MessageInvocation): Unit = guard.withGuard { - if (isShutdown) - Actor.log.warning("Actor [%s] is shut down,\n\tignoring message [%s]", toString, messageHandle) + if (isShutdown) Actor.log.warning("Actor [%s] is shut down,\n\tignoring message [%s]", toString, messageHandle) else { currentMessage = messageHandle try { @@ -1005,8 +992,7 @@ class LocalActorRef private[akka] ( case e => Actor.log.error(e, "Could not invoke actor [%s]", this) throw e - } - finally { + } finally { currentMessage = null //TODO: Don't reset this, we might want to resend the message } } @@ -1032,8 +1018,7 @@ class LocalActorRef private[akka] ( protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = { val isUnrestartable = if (maxNrOfRetries.isEmpty && withinTimeRange.isEmpty) { //Immortal false - } - else if (withinTimeRange.isEmpty) { // restrict number of restarts + } else if (withinTimeRange.isEmpty) { // restrict number of restarts maxNrOfRetriesCount += 1 //Increment number of retries maxNrOfRetriesCount > maxNrOfRetries.get } else { // cannot restart more than N within M timerange @@ -1042,10 +1027,8 @@ class LocalActorRef private[akka] ( val now = System.currentTimeMillis val retries = maxNrOfRetriesCount //We are within the time window if it isn't the first restart, or if the window hasn't closed - val insideWindow = if (windowStart == 0) - false - else - (now - windowStart) <= withinTimeRange.get + val insideWindow = if (windowStart == 0) false + else (now - windowStart) <= withinTimeRange.get //The actor is dead if it dies X times within the window of restart val unrestartable = insideWindow && retries > maxNrOfRetries.getOrElse(1) diff --git a/akka-actor/src/main/scala/util/Crypt.scala b/akka-actor/src/main/scala/util/Crypt.scala new file mode 100644 index 0000000000..e8cd2f82dd --- /dev/null +++ b/akka-actor/src/main/scala/util/Crypt.scala @@ -0,0 +1,36 @@ +/** + * Copyright (C) 2009-2010 Scalable Solutions AB + */ + +package se.scalablesolutions.akka.util + +import java.security.{MessageDigest, SecureRandom} + +/** + * @author Jonas Bonér + */ +object Crypt { + lazy val random = SecureRandom.getInstance("SHA1PRNG") + + def generateSecureCookie: String = { + val bytes = Array.make(32, 0.byteValue) + random.nextBytes(bytes) + getMD5For(bytes) + } + + def getMD5For(s: String): String = getMD5For(s.getBytes("ASCII")) + + def getMD5For(b: Array[Byte]): String = { + val digest = MessageDigest.getInstance("MD5") + digest.update(b) + val bytes = digest.digest + + val sb = new StringBuilder + val hex = "0123456789ABCDEF" + bytes.foreach { b => + val n = b.asInstanceOf[Int] + sb.append(hex.charAt((n & 0xF) >> 4)).append(hex.charAt(n & 0xF)) + } + sb.toString + } +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/util/Helpers.scala b/akka-actor/src/main/scala/util/Helpers.scala index 394b39e101..75ba61620a 100644 --- a/akka-actor/src/main/scala/util/Helpers.scala +++ b/akka-actor/src/main/scala/util/Helpers.scala @@ -4,8 +4,6 @@ package se.scalablesolutions.akka.util -import java.security.MessageDigest - /** * @author Jonas Bonér */ @@ -22,20 +20,6 @@ object Helpers extends Logging { bytes } - def getMD5For(s: String) = { - val digest = MessageDigest.getInstance("MD5") - digest.update(s.getBytes("ASCII")) - val bytes = digest.digest - - val sb = new StringBuilder - val hex = "0123456789ABCDEF" - bytes.foreach(b => { - val n = b.asInstanceOf[Int] - sb.append(hex.charAt((n & 0xF) >> 4)).append(hex.charAt(n & 0xF)) - }) - sb.toString - } - /** * Convenience helper to cast the given Option of Any to an Option of the given type. Will throw a ClassCastException * if the actual type is not assignable from the given one. @@ -57,4 +41,4 @@ object Helpers extends Logging { log.warning(e, "Cannot narrow %s to expected type %s!", o, implicitly[Manifest[T]].erasure.getName) None } -} +} \ No newline at end of file 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 6f2052f0bd..dbb63f972b 100644 --- a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala +++ b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala @@ -32,7 +32,7 @@ object CommonsCodec { import CommonsCodec._ import CommonsCodec.Base64StringEncoder._ - + /** * A module for supporting Redis based persistence. *

diff --git a/scripts/generate_secure_cookie.sh b/scripts/generate_secure_cookie.sh new file mode 100755 index 0000000000..12ca10a940 --- /dev/null +++ b/scripts/generate_secure_cookie.sh @@ -0,0 +1,27 @@ +#!/bin/sh +exec scala "$0" "$@" +!# + +/** + * Copyright (C) 2009-2010 Scalable Solutions AB + */ +import java.security.{MessageDigest, SecureRandom} + +lazy val random = SecureRandom.getInstance("SHA1PRNG") + +val buffer = Array.make(32, 0.byteValue) +random.nextBytes(buffer) + +val digest = MessageDigest.getInstance("MD5") +digest.update(buffer) +val bytes = digest.digest + +val sb = new StringBuilder +val hex = "0123456789ABCDEF" +bytes.foreach { b => + val n = b.asInstanceOf[Int] + sb.append(hex.charAt((n & 0xF) >> 4)).append(hex.charAt(n & 0xF)) +} + +println("Cryptographically secure cookie:") +println(sb.toString) From 8fd6361c4485739085f2746969c1321fc0dbd2b6 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 26 Oct 2010 12:49:25 +0200 Subject: [PATCH 24/39] BREAKAGE: switching from se.scalablesolutions.akka to akka for all packages --- akka-actor/src/main/scala/actor/Actor.scala | 16 +- .../src/main/scala/actor/ActorRef.scala | 20 +- .../src/main/scala/actor/ActorRegistry.scala | 6 +- akka-actor/src/main/scala/actor/Agent.scala | 8 +- .../actor/BootableActorLoaderService.scala | 6 +- akka-actor/src/main/scala/actor/FSM.scala | 6 +- .../src/main/scala/actor/Implicits.scala | 2 +- .../src/main/scala/actor/Scheduler.scala | 6 +- .../src/main/scala/actor/Supervisor.scala | 10 +- .../src/main/scala/actor/UntypedActor.scala | 10 +- akka-actor/src/main/scala/config/Config.scala | 10 +- .../src/main/scala/config/Configuration.scala | 26 +- .../src/main/scala/config/Configurator.scala | 4 +- .../main/scala/config/SupervisionConfig.scala | 6 +- .../scala/dataflow/DataFlowVariable.scala | 12 +- .../src/main/scala/dispatch/Dispatchers.scala | 12 +- .../ExecutorBasedEventDrivenDispatcher.scala | 10 +- ...sedEventDrivenWorkStealingDispatcher.scala | 12 +- .../src/main/scala/dispatch/Future.scala | 8 +- .../main/scala/dispatch/HawtDispatcher.scala | 6 +- .../main/scala/dispatch/MailboxHandling.scala | 10 +- .../main/scala/dispatch/MessageHandling.scala | 6 +- .../dispatch/ThreadBasedDispatcher.scala | 10 +- .../scala/dispatch/ThreadPoolBuilder.scala | 4 +- akka-actor/src/main/scala/japi/JavaAPI.scala | 2 +- .../src/main/scala/routing/Iterators.scala | 4 +- .../src/main/scala/routing/Listeners.scala | 4 +- .../src/main/scala/routing/Routers.scala | 4 +- .../src/main/scala/routing/Routing.scala | 6 +- akka-actor/src/main/scala/stm/Ref.scala | 4 +- .../src/main/scala/stm/Transaction.scala | 14 +- .../main/scala/stm/TransactionFactory.scala | 6 +- .../scala/stm/TransactionFactoryBuilder.scala | 4 +- .../scala/stm/TransactionManagement.scala | 22 +- .../src/main/scala/stm/TransactionalMap.scala | 4 +- .../main/scala/stm/TransactionalVector.scala | 4 +- .../src/main/scala/stm/global/Atomic.scala | 6 +- .../src/main/scala/stm/global/GlobalStm.scala | 6 +- .../src/main/scala/stm/global/package.scala | 2 +- .../src/main/scala/stm/local/Atomic.scala | 6 +- .../src/main/scala/stm/local/LocalStm.scala | 6 +- .../src/main/scala/stm/local/package.scala | 2 +- .../src/main/scala/stm/transactional.scala | 10 +- akka-actor/src/main/scala/util/Address.scala | 2 +- .../src/main/scala/util/AkkaException.scala | 6 +- akka-actor/src/main/scala/util/Bootable.scala | 2 +- akka-actor/src/main/scala/util/Duration.scala | 8 +- akka-actor/src/main/scala/util/HashCode.scala | 2 +- akka-actor/src/main/scala/util/Helpers.scala | 2 +- .../main/scala/util/ListenerManagement.scala | 4 +- akka-actor/src/main/scala/util/LockUtil.scala | 2 +- akka-actor/src/main/scala/util/Logging.scala | 2 +- .../main/scala/util/ReflectiveAccess.scala | 34 +- .../akka/config/SupervisionConfig.java | 6 +- .../akka/japi/JavaAPITestBase.java | 2 +- .../akka/stm/Address.java | 2 +- .../akka/stm/CounterExample.java | 6 +- .../akka/stm/JavaStmTests.java | 6 +- .../akka/stm/RefExample.java | 6 +- .../akka/stm/StmExamples.java | 6 +- .../akka/stm/TransactionFactoryExample.java | 6 +- .../akka/stm/TransactionalMapExample.java | 6 +- .../akka/stm/TransactionalVectorExample.java | 6 +- .../scalablesolutions => }/akka/stm/User.java | 2 +- akka-actor/src/test/scala/Messages.scala | 2 +- .../ActorFireForgetRequestReplySpec.scala | 6 +- .../test/scala/actor/actor/ActorRefSpec.scala | 4 +- .../test/scala/actor/actor/AgentSpec.scala | 2 +- .../src/test/scala/actor/actor/Bench.scala | 4 +- .../test/scala/actor/actor/FSMActorSpec.scala | 2 +- .../scala/actor/actor/ForwardActorSpec.scala | 2 +- .../test/scala/actor/actor/HotSwapSpec.scala | 2 +- .../actor/actor/ReceiveTimeoutSpec.scala | 2 +- .../scala/actor/actor/TransactorSpec.scala | 4 +- .../supervisor/RestartStrategySpec.scala | 4 +- .../supervisor/SupervisorHierarchySpec.scala | 4 +- .../actor/supervisor/SupervisorMiscSpec.scala | 6 +- .../actor/supervisor/SupervisorSpec.scala | 6 +- .../test/scala/dataflow/DataFlowSpec.scala | 6 +- .../test/scala/dispatch/ActorModelSpec.scala | 10 +- .../test/scala/dispatch/DispatchersSpec.scala | 4 +- ...rBasedEventDrivenDispatcherActorSpec.scala | 6 +- ...BasedEventDrivenDispatcherActorsSpec.scala | 4 +- ...ventDrivenWorkStealingDispatcherSpec.scala | 6 +- .../src/test/scala/dispatch/FutureSpec.scala | 4 +- .../dispatch/HawtDispatcherActorSpec.scala | 6 +- .../dispatch/HawtDispatcherEchoServer.scala | 8 +- .../scala/dispatch/MailboxConfigSpec.scala | 8 +- .../scala/dispatch/ThreadBasedActorSpec.scala | 6 +- .../dispatch/ThreadBasedDispatcherSpec.scala | 4 +- .../src/test/scala/japi/JavaAPITest.scala | 2 +- .../test/scala/misc/ActorRegistrySpec.scala | 2 +- .../src/test/scala/misc/SchedulerSpec.scala | 4 +- .../src/test/scala/routing/RoutingSpec.scala | 12 +- .../src/test/scala/stm/JavaStmSpec.scala | 4 +- akka-actor/src/test/scala/stm/RefSpec.scala | 4 +- akka-actor/src/test/scala/stm/StmSpec.scala | 8 +- .../src/test/scala/ticket/Ticket001Spec.scala | 2 +- .../akka/amqp/ExampleSessionJava.java | 18 +- .../se/scalablesolutions/akka/amqp/AMQP.scala | 12 +- .../akka/amqp/AMQPMessage.scala | 4 +- .../akka/amqp/ConsumerActor.scala | 6 +- .../akka/amqp/ExampleSession.scala | 8 +- .../akka/amqp/ExchangeType.scala | 2 +- .../akka/amqp/FaultTolerantChannelActor.scala | 6 +- .../amqp/FaultTolerantConnectionActor.scala | 8 +- .../akka/amqp/ProducerActor.scala | 4 +- .../scalablesolutions/akka/amqp/rpc/RPC.scala | 10 +- .../akka/amqp/rpc/RpcClientActor.scala | 4 +- .../akka/amqp/rpc/RpcServerActor.scala | 4 +- ...MQPConnectionRecoveryTestIntegration.scala | 8 +- ...nsumerChannelRecoveryTestIntegration.scala | 10 +- ...merConnectionRecoveryTestIntegration.scala | 8 +- ...umerManualAcknowledgeTestIntegration.scala | 10 +- ...PConsumerManualRejectTestIntegration.scala | 10 +- .../AMQPConsumerMessageTestIntegration.scala | 10 +- ...oducerChannelRecoveryTestIntegration.scala | 8 +- ...cerConnectionRecoveryTestIntegration.scala | 8 +- .../AMQPProducerMessageTestIntegration.scala | 8 +- ...tobufProducerConsumerTestIntegration.scala | 8 +- .../AMQPRpcClientServerTestIntegration.scala | 10 +- .../test/AMQPRpcProtobufTestIntegration.scala | 8 +- .../test/AMQPRpcStringTestIntegration.scala | 6 +- ...tringProducerConsumerTestIntegration.scala | 6 +- .../akka/amqp/test/AMQPTest.scala | 4 +- .../akka/camel/consume.java | 4 +- .../main/scala/CamelContextLifecycle.scala | 8 +- akka-camel/src/main/scala/CamelService.scala | 12 +- akka-camel/src/main/scala/Consumer.scala | 4 +- .../src/main/scala/ConsumerPublisher.scala | 16 +- akka-camel/src/main/scala/Message.scala | 4 +- akka-camel/src/main/scala/Producer.scala | 4 +- .../main/scala/component/ActorComponent.scala | 24 +- .../scala/component/TypedActorComponent.scala | 6 +- .../akka/camel/MessageJavaTestBase.java | 8 +- .../akka/camel/SampleRemoteTypedConsumer.java | 4 +- .../camel/SampleRemoteTypedConsumerImpl.java | 4 +- .../camel/SampleRemoteUntypedConsumer.java | 4 +- .../akka/camel/SampleTypedActor.java | 2 +- .../akka/camel/SampleTypedActorImpl.java | 4 +- .../akka/camel/SampleTypedConsumer.java | 4 +- .../akka/camel/SampleTypedConsumerImpl.java | 4 +- .../akka/camel/SampleTypedSingleConsumer.java | 4 +- .../camel/SampleTypedSingleConsumerImpl.java | 4 +- .../akka/camel/SampleUntypedActor.java | 4 +- .../akka/camel/SampleUntypedConsumer.java | 4 +- .../camel/SampleUntypedConsumerBlocking.java | 2 +- .../SampleUntypedForwardingProducer.java | 2 +- .../camel/SampleUntypedReplyingProducer.java | 2 +- .../scala/CamelContextLifecycleTest.scala | 2 +- .../test/scala/CamelExchangeAdapterTest.scala | 2 +- .../test/scala/CamelMessageAdapterTest.scala | 2 +- .../test/scala/CamelServiceManagerTest.scala | 4 +- .../test/scala/ConsumerRegisteredTest.scala | 4 +- akka-camel/src/test/scala/ConsumerTest.scala | 6 +- .../src/test/scala/MessageJavaTest.scala | 2 +- .../src/test/scala/MessageScalaTest.scala | 2 +- .../src/test/scala/ProducerFeatureTest.scala | 6 +- .../src/test/scala/PublishRequestorTest.scala | 8 +- .../src/test/scala/RemoteConsumerTest.scala | 8 +- .../scala/UntypedProducerFeatureTest.scala | 6 +- .../component/ActorComponentFeatureTest.scala | 10 +- .../scala/component/ActorComponentTest.scala | 4 +- .../scala/component/ActorProducerTest.scala | 10 +- .../TypedActorComponentFeatureTest.scala | 6 +- .../src/test/scala/support/TestSupport.scala | 6 +- .../src/main/scala/AkkaBroadcaster.scala | 8 +- .../scala/AkkaClusterBroadcastFilter.scala | 6 +- .../src/main/scala/AkkaCometServlet.scala | 6 +- akka-http/src/main/scala/AkkaLoader.scala | 6 +- .../src/main/scala/DefaultAkkaLoader.scala | 16 +- .../src/main/scala/EmbeddedAppServer.scala | 10 +- akka-http/src/main/scala/Initializer.scala | 14 +- akka-http/src/main/scala/ListWriter.scala | 4 +- akka-http/src/main/scala/Security.scala | 10 +- akka-http/src/test/scala/AllTest.scala | 2 +- akka-http/src/test/scala/SecuritySpec.scala | 6 +- .../scala/AtomikosTransactionService.scala | 6 +- akka-jta/src/main/scala/JTA.scala | 12 +- .../src/main/scala/TransactionContext.scala | 6 +- .../src/main/scala/TransactionProtocol.scala | 4 +- .../src/test/scala/ReflectiveAccessSpec.scala | 4 +- .../src/main/resources/features.xml | 6 +- akka-kernel/src/main/scala/Kernel.scala | 6 +- .../src/main/scala/CassandraSession.scala | 12 +- .../src/main/scala/CassandraStorage.scala | 8 +- .../main/scala/CassandraStorageBackend.scala | 12 +- .../scala/CassandraPersistentActorSpec.scala | 4 +- .../src/main/scala/KVStorageBackend.scala | 16 +- .../src/main/scala/Pool.scala | 2 +- .../src/main/scala/Storage.scala | 10 +- .../src/main/scala/StorageBackend.scala | 2 +- .../test/scala/MapStorageBackendTest.scala | 6 +- .../test/scala/QueueStorageBackendTest.scala | 4 +- .../test/scala/RefStorageBackendTest.scala | 4 +- .../scala/SortedSetStorageBackendTest.scala | 4 +- .../src/test/scala/Ticket343Test.scala | 10 +- .../test/scala/VectorStorageBackendTest.scala | 4 +- .../src/main/scala/CouchDBStorage.scala | 8 +- .../main/scala/CouchDBStorageBackend.scala | 10 +- .../scala/CouchDBPersistentActorSpec.scala | 4 +- .../scala/CouchDBStorageBackendSpec.scala | 6 +- .../src/main/scala/HbaseStorage.scala | 8 +- .../src/main/scala/HbaseStorageBackend.scala | 12 +- ...sePersistentActorSpecTestIntegration.scala | 4 +- .../HbaseStorageSpecTestIntegration.scala | 2 +- .../HbaseTicket343SpecTestIntegration.scala | 10 +- .../SimpleHbaseSpecTestIntegration.scala | 4 +- .../src/main/scala/MongoStorage.scala | 8 +- .../src/main/scala/MongoStorageBackend.scala | 10 +- .../test/scala/MongoPersistentActorSpec.scala | 4 +- .../src/test/scala/MongoStorageSpec.scala | 2 +- .../src/test/scala/MongoTicket343Spec.scala | 10 +- .../src/main/scala/RedisPubSubServer.scala | 4 +- .../src/main/scala/RedisStorage.scala | 8 +- .../src/main/scala/RedisStorageBackend.scala | 10 +- .../scala/RedisInconsistentSizeBugTest.scala | 12 +- .../test/scala/RedisPersistentActorSpec.scala | 6 +- .../src/test/scala/RedisPersistentQSpec.scala | 4 +- .../scala/RedisPersistentSortedSetSpec.scala | 4 +- .../test/scala/RedisStorageBackendSpec.scala | 6 +- .../src/test/scala/RedisTicket343Spec.scala | 12 +- .../src/main/scala/RiakStorage.scala | 8 +- .../src/main/scala/RiakStorageBackend.scala | 8 +- .../RiakStorageBackendCompatibilityTest.scala | 4 +- .../scala/RiakTicket343TestIntegration.scala | 4 +- .../src/main/scala/VoldemortStorage.scala | 8 +- .../main/scala/VoldemortStorageBackend.scala | 6 +- .../src/test/scala/EmbeddedVoldemort.scala | 4 +- .../scala/VoldemortPersistentActorSuite.scala | 4 +- ...emortStorageBackendCompatibilityTest.scala | 4 +- .../scala/VoldemortStorageBackendSuite.scala | 6 +- .../test/scala/VoldemortTicket343Test.scala | 4 +- .../akka/remote/protocol/RemoteProtocol.java | 1460 ++++++++--------- .../src/main/protocol/RemoteProtocol.proto | 2 +- .../remote/BootableRemoteActorService.scala | 8 +- .../src/main/scala/remote/Cluster.scala | 16 +- .../scala/remote/JGroupsClusterActor.scala | 4 +- .../main/scala/remote/MessageSerializer.scala | 8 +- .../src/main/scala/remote/RemoteClient.scala | 20 +- .../src/main/scala/remote/RemoteServer.scala | 20 +- .../src/main/scala/serialization/Binary.scala | 2 +- .../scala/serialization/Compression.scala | 2 +- .../scala/serialization/Serializable.scala | 2 +- .../serialization/SerializationProtocol.scala | 20 +- .../main/scala/serialization/Serializer.scala | 2 +- .../main/scala/serialization/package.scala | 2 +- .../akka/actor/ProtobufProtocol.java | 246 +-- .../akka/actor/RemoteTypedActorOne.java | 2 +- .../akka/actor/RemoteTypedActorOneImpl.java | 4 +- .../akka/actor/RemoteTypedActorTwo.java | 2 +- .../akka/actor/RemoteTypedActorTwoImpl.java | 4 +- .../src/test/protocol/ProtobufProtocol.proto | 2 +- akka-remote/src/test/scala/Messages.scala | 4 +- .../ClientInitiatedRemoteActorSpec.scala | 10 +- .../scala/remote/RemoteSupervisorSpec.scala | 12 +- .../RemoteTransactionalTypedActorSpec.scala | 8 +- .../scala/remote/RemoteTypedActorSpec.scala | 10 +- .../ServerInitiatedRemoteActorSample.scala | 8 +- .../ServerInitiatedRemoteActorSpec.scala | 20 +- .../ServerInitiatedRemoteTypedActorSpec.scala | 6 +- .../src/test/scala/remote/ShutdownSpec.scala | 4 +- ...rotobufActorMessageSerializationSpec.scala | 6 +- .../ScalaJSONSerializableSpec.scala | 4 +- .../ScalaJSONSerializerSpec.scala | 4 +- .../SerializableTypeClassActorSpec.scala | 6 +- .../scala/serialization/SerializerSpec.scala | 2 +- .../scala/serialization/Ticket435Spec.scala | 6 +- .../TypedActorSerializationSpec.scala | 10 +- .../UntypedActorSerializationSpec.scala | 6 +- .../src/test/scala/ticket/Ticket001Spec.scala | 2 +- .../src/test/scala/ticket/Ticket434Spec.scala | 12 +- .../src/main/scala/Ants.scala | 1 - .../src/main/java/sample/camel/BeanImpl.java | 2 +- .../sample/camel/RemoteTypedConsumer1.java | 2 +- .../camel/RemoteTypedConsumer1Impl.java | 2 +- .../sample/camel/RemoteTypedConsumer2.java | 2 +- .../camel/RemoteTypedConsumer2Impl.java | 2 +- .../java/sample/camel/TypedConsumer1.java | 2 +- .../java/sample/camel/TypedConsumer1Impl.java | 2 +- .../java/sample/camel/TypedConsumer2.java | 2 +- .../java/sample/camel/TypedConsumer2Impl.java | 2 +- .../src/main/scala/Actors.scala | 6 +- .../src/main/scala/Boot.scala | 8 +- .../src/main/scala/ClientApplication.scala | 8 +- .../src/main/scala/ServerApplication.scala | 8 +- .../main/scala/StandaloneApplication.scala | 4 +- .../scala/HttpConcurrencyTestStress.scala | 12 +- akka-samples/akka-sample-chat/README | 2 +- .../src/main/scala/ChatServer.scala | 16 +- .../main/scala/DiningHakkersOnBecome.scala | 4 +- .../src/main/scala/DiningHakkersOnFsm.scala | 2 +- .../src/main/scala/osgiExample.scala | 2 +- .../src/main/scala/RedisPubSub.scala | 4 +- .../ClientManagedRemoteActorSample.scala | 8 +- .../ServerManagedRemoteActorSample.scala | 8 +- .../src/main/java/sample/rest/java/Boot.java | 4 +- .../java/PersistentSimpleServiceImpl.java | 6 +- .../java/sample/rest/java/ReceiverImpl.java | 2 +- .../sample/rest/java/SimpleServiceImpl.java | 6 +- .../src/main/scala/SimpleService.scala | 16 +- .../src/main/scala/SimpleService.scala | 14 +- .../src/main/webapp/WEB-INF/web.xml | 2 +- akka-sbt-plugin/project/build.properties | 2 +- .../src/main/scala/AkkaProject.scala | 4 +- .../scala/ActorBeanDefinitionParser.scala | 2 +- .../src/main/scala/ActorFactoryBean.scala | 10 +- akka-spring/src/main/scala/ActorParser.scala | 4 +- .../src/main/scala/ActorProperties.scala | 2 +- .../src/main/scala/AkkaNamespaceHandler.scala | 2 +- .../scala/AkkaSpringConfigurationTags.scala | 2 +- .../CamelServiceBeanDefinitionParser.scala | 4 +- .../main/scala/CamelServiceFactoryBean.scala | 4 +- ...onfiggyPropertyPlaceholderConfigurer.scala | 2 +- .../DispatcherBeanDefinitionParser.scala | 2 +- .../main/scala/DispatcherFactoryBean.scala | 10 +- .../src/main/scala/DispatcherProperties.scala | 2 +- .../src/main/scala/PropertyEntries.scala | 2 +- .../src/main/scala/StringReflect.scala | 2 +- .../SupervisionBeanDefinitionParser.scala | 6 +- .../main/scala/SupervisionFactoryBean.scala | 8 +- .../akka/spring/Pojo.java | 4 +- .../akka/spring/PojoInf.java | 2 +- .../akka/spring/SampleBean.java | 4 +- .../akka/spring/SampleBeanIntf.java | 2 +- .../akka/spring/SampleRoute.java | 2 +- .../akka/spring/foo/Bar.java | 4 +- .../akka/spring/foo/Foo.java | 4 +- .../akka/spring/foo/IBar.java | 2 +- .../akka/spring/foo/IFoo.java | 2 +- .../akka/spring/foo/IMyPojo.java | 2 +- .../akka/spring/foo/MyPojo.java | 4 +- .../akka/spring/foo/PingActor.java | 6 +- .../akka/spring/foo/PongActor.java | 4 +- .../akka/spring/foo/StatefulPojo.java | 12 +- akka-spring/src/test/resources/appContext.xml | 18 +- .../appContextCamelServiceCustom.xml | 6 +- .../src/test/resources/dispatcher-config.xml | 26 +- .../src/test/resources/property-config.xml | 2 +- .../test/resources/server-managed-config.xml | 20 +- .../src/test/resources/supervisor-config.xml | 32 +- .../src/test/resources/typed-actor-config.xml | 36 +- .../test/resources/untyped-actor-config.xml | 10 +- .../src/test/scala/ActorFactoryBeanTest.scala | 10 +- .../scala/CamelServiceSpringFeatureTest.scala | 6 +- ...ggyPropertyPlaceholderConfigurerSpec.scala | 6 +- .../DispatcherBeanDefinitionParserTest.scala | 2 +- .../scala/DispatcherFactoryBeanTest.scala | 6 +- .../scala/DispatcherSpringFeatureTest.scala | 8 +- akka-spring/src/test/scala/ScalaDom.scala | 2 +- .../SupervisionBeanDefinitionParserTest.scala | 4 +- .../scala/SupervisionFactoryBeanTest.scala | 8 +- .../scala/SupervisorSpringFeatureTest.scala | 10 +- .../TypedActorBeanDefinitionParserTest.scala | 8 +- .../scala/TypedActorSpringFeatureTest.scala | 10 +- .../scala/UntypedActorSpringFeatureTest.scala | 14 +- .../akka/config/DependencyBinding.java | 2 +- .../akka/config/TypedActorGuiceModule.java | 2 +- .../src/main/resources/META-INF/aop.xml | 2 +- .../src/main/scala/actor/TypedActor.scala | 20 +- .../scala/config/TypedActorConfigurator.scala | 2 +- .../config/TypedActorGuiceConfigurator.scala | 10 +- .../se/scalablesolutions/akka/actor/Bar.java | 2 +- .../scalablesolutions/akka/actor/BarImpl.java | 4 +- .../se/scalablesolutions/akka/actor/Ext.java | 2 +- .../scalablesolutions/akka/actor/ExtImpl.java | 2 +- .../se/scalablesolutions/akka/actor/Foo.java | 2 +- .../scalablesolutions/akka/actor/FooImpl.java | 4 +- .../actor/NestedTransactionalTypedActor.java | 2 +- .../NestedTransactionalTypedActorImpl.java | 6 +- .../akka/actor/SamplePojo.java | 2 +- .../akka/actor/SamplePojoImpl.java | 4 +- .../akka/actor/SimpleJavaPojo.java | 10 +- .../akka/actor/SimpleJavaPojoCaller.java | 4 +- .../akka/actor/SimpleJavaPojoCallerImpl.java | 6 +- .../akka/actor/SimpleJavaPojoImpl.java | 10 +- .../akka/actor/TransactionalTypedActor.java | 2 +- .../actor/TransactionalTypedActorImpl.java | 10 +- .../akka/actor/TypedActorFailer.java | 2 +- .../akka/actor/TypedActorFailerImpl.java | 4 +- .../NestedTransactionalTypedActorSpec.scala | 4 +- ...artNestedTransactionalTypedActorSpec.scala | 12 +- .../RestartTransactionalTypedActorSpec.scala | 8 +- .../TransactionalTypedActorSpec.scala | 4 +- .../typed-actor/TypedActorContextSpec.scala | 4 +- .../TypedActorGuiceConfiguratorSpec.scala | 10 +- .../typed-actor/TypedActorLifecycleSpec.scala | 8 +- .../actor/typed-actor/TypedActorSpec.scala | 6 +- .../TypedActorUtilFunctionsSpec.scala | 2 +- config/akka-reference.conf | 4 +- config/microkernel-server.xml | 2 +- project/build.properties | 2 +- project/build/AkkaProject.scala | 10 +- 393 files changed, 2100 insertions(+), 2101 deletions(-) rename akka-actor/src/test/java/{se/scalablesolutions => }/akka/config/SupervisionConfig.java (81%) rename akka-actor/src/test/java/{se/scalablesolutions => }/akka/japi/JavaAPITestBase.java (96%) rename akka-actor/src/test/java/{se/scalablesolutions => }/akka/stm/Address.java (85%) rename akka-actor/src/test/java/{se/scalablesolutions => }/akka/stm/CounterExample.java (82%) rename akka-actor/src/test/java/{se/scalablesolutions => }/akka/stm/JavaStmTests.java (94%) rename akka-actor/src/test/java/{se/scalablesolutions => }/akka/stm/RefExample.java (85%) rename akka-actor/src/test/java/{se/scalablesolutions => }/akka/stm/StmExamples.java (74%) rename akka-actor/src/test/java/{se/scalablesolutions => }/akka/stm/TransactionFactoryExample.java (87%) rename akka-actor/src/test/java/{se/scalablesolutions => }/akka/stm/TransactionalMapExample.java (87%) rename akka-actor/src/test/java/{se/scalablesolutions => }/akka/stm/TransactionalVectorExample.java (87%) rename akka-actor/src/test/java/{se/scalablesolutions => }/akka/stm/User.java (83%) rename akka-amqp/src/main/java/{se/scalablesolutions => }/akka/amqp/ExampleSessionJava.java (95%) rename akka-amqp/src/test/scala/{se/scalablesolutions => }/akka/amqp/test/AMQPConnectionRecoveryTestIntegration.scala (88%) rename akka-amqp/src/test/scala/{se/scalablesolutions => }/akka/amqp/test/AMQPConsumerChannelRecoveryTestIntegration.scala (90%) rename akka-amqp/src/test/scala/{se/scalablesolutions => }/akka/amqp/test/AMQPConsumerConnectionRecoveryTestIntegration.scala (94%) rename akka-amqp/src/test/scala/{se/scalablesolutions => }/akka/amqp/test/AMQPConsumerManualAcknowledgeTestIntegration.scala (90%) rename akka-amqp/src/test/scala/{se/scalablesolutions => }/akka/amqp/test/AMQPConsumerManualRejectTestIntegration.scala (86%) rename akka-amqp/src/test/scala/{se/scalablesolutions => }/akka/amqp/test/AMQPConsumerMessageTestIntegration.scala (83%) rename akka-amqp/src/test/scala/{se/scalablesolutions => }/akka/amqp/test/AMQPProducerChannelRecoveryTestIntegration.scala (86%) rename akka-amqp/src/test/scala/{se/scalablesolutions => }/akka/amqp/test/AMQPProducerConnectionRecoveryTestIntegration.scala (86%) rename akka-amqp/src/test/scala/{se/scalablesolutions => }/akka/amqp/test/AMQPProducerMessageTestIntegration.scala (85%) rename akka-amqp/src/test/scala/{se/scalablesolutions => }/akka/amqp/test/AMQPProtobufProducerConsumerTestIntegration.scala (86%) rename akka-amqp/src/test/scala/{se/scalablesolutions => }/akka/amqp/test/AMQPRpcClientServerTestIntegration.scala (89%) rename akka-amqp/src/test/scala/{se/scalablesolutions => }/akka/amqp/test/AMQPRpcProtobufTestIntegration.scala (86%) rename akka-amqp/src/test/scala/{se/scalablesolutions => }/akka/amqp/test/AMQPRpcStringTestIntegration.scala (89%) rename akka-amqp/src/test/scala/{se/scalablesolutions => }/akka/amqp/test/AMQPStringProducerConsumerTestIntegration.scala (89%) rename akka-amqp/src/test/scala/{se/scalablesolutions => }/akka/amqp/test/AMQPTest.scala (77%) rename akka-camel/src/main/java/{se/scalablesolutions => }/akka/camel/consume.java (79%) rename akka-remote/src/main/java/{se/scalablesolutions => }/akka/remote/protocol/RemoteProtocol.java (69%) rename akka-remote/src/test/java/{se/scalablesolutions => }/akka/actor/ProtobufProtocol.java (71%) rename akka-remote/src/test/java/{se/scalablesolutions => }/akka/actor/RemoteTypedActorOne.java (77%) rename akka-remote/src/test/java/{se/scalablesolutions => }/akka/actor/RemoteTypedActorOneImpl.java (89%) rename akka-remote/src/test/java/{se/scalablesolutions => }/akka/actor/RemoteTypedActorTwo.java (77%) rename akka-remote/src/test/java/{se/scalablesolutions => }/akka/actor/RemoteTypedActorTwoImpl.java (89%) rename akka-spring/src/test/java/{se/scalablesolutions => }/akka/spring/Pojo.java (92%) rename akka-spring/src/test/java/{se/scalablesolutions => }/akka/spring/PojoInf.java (86%) rename akka-spring/src/test/java/{se/scalablesolutions => }/akka/spring/SampleBean.java (80%) rename akka-spring/src/test/java/{se/scalablesolutions => }/akka/spring/SampleBeanIntf.java (70%) rename akka-spring/src/test/java/{se/scalablesolutions => }/akka/spring/SampleRoute.java (84%) rename akka-spring/src/test/java/{se/scalablesolutions => }/akka/spring/foo/Bar.java (77%) rename akka-spring/src/test/java/{se/scalablesolutions => }/akka/spring/foo/Foo.java (54%) rename akka-spring/src/test/java/{se/scalablesolutions => }/akka/spring/foo/IBar.java (54%) rename akka-spring/src/test/java/{se/scalablesolutions => }/akka/spring/foo/IFoo.java (82%) rename akka-spring/src/test/java/{se/scalablesolutions => }/akka/spring/foo/IMyPojo.java (86%) rename akka-spring/src/test/java/{se/scalablesolutions => }/akka/spring/foo/MyPojo.java (85%) rename akka-spring/src/test/java/{se/scalablesolutions => }/akka/spring/foo/PingActor.java (91%) rename akka-spring/src/test/java/{se/scalablesolutions => }/akka/spring/foo/PongActor.java (80%) rename akka-spring/src/test/java/{se/scalablesolutions => }/akka/spring/foo/StatefulPojo.java (80%) rename akka-typed-actor/src/main/java/{se/scalablesolutions => }/akka/config/DependencyBinding.java (91%) rename akka-typed-actor/src/main/java/{se/scalablesolutions => }/akka/config/TypedActorGuiceModule.java (95%) 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 92a7800128..cfedcf9256 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._ @@ -156,7 +156,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 6b9385ca4e..89acfb6e4c 100644 --- a/akka-actor/src/main/scala/actor/Agent.scala +++ b/akka-actor/src/main/scala/actor/Agent.scala @@ -2,11 +2,11 @@ * 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 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 0bdc04fc48..43eeae2f56 100644 --- a/akka-actor/src/main/scala/actor/FSM.scala +++ b/akka-actor/src/main/scala/actor/FSM.scala @@ -2,10 +2,10 @@ * 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.stm.local._ +import akka.stm.Ref +import akka.stm.local._ import java.util.concurrent.{ScheduledFuture, TimeUnit} 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 e97347754b..fe762aefdf 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 ea06ebb4ec..5d4ebe5440 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/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 394b39e101..4154bfadba 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 import java.security.MessageDigest 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 909713194b..c6ae768bad 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 e4515bd3da..8d0a404da0 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 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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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/se/scalablesolutions/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..8d93b11388 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 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..4b0f245479 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] 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..001d6015a2 100644 --- a/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorageBackend.scala +++ b/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorageBackend.scala @@ -2,13 +2,13 @@ * 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._ 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-common/src/main/scala/KVStorageBackend.scala b/akka-persistence/akka-persistence-common/src/main/scala/KVStorageBackend.scala index 25c1b0b1f1..dd1ca60938 100644 --- a/akka-persistence/akka-persistence-common/src/main/scala/KVStorageBackend.scala +++ b/akka-persistence/akka-persistence-common/src/main/scala/KVStorageBackend.scala @@ -2,13 +2,13 @@ * 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.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 java.lang.String import collection.JavaConversions @@ -16,7 +16,7 @@ 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 akka.persistence.common.PersistentMapBinary.COrdering._ import collection.immutable._ @@ -452,7 +452,7 @@ private[akka] trait KVStorageBackend extends MapStorageBackend[Array[Byte], Arra } def fromBytes(bytes: Array[Byte]): SortedSet[Array[Byte]] = { - import se.scalablesolutions.akka.persistence.common.PersistentMapBinary.COrdering._ + import akka.persistence.common.PersistentMapBinary.COrdering._ var set = new TreeSet[Array[Byte]] if (bytes.length > IntSerializer.bytesPerInt) { 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..e7660805bb 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 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..e5833e1091 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 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..fa1648aef1 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._ 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..a5589d92a0 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 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 3be2657540..b80af0aa22 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 4d118850f0..253eeba2b0 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 6f2052f0bd..6d0dc52eea 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..de1773013f 100644 --- a/akka-persistence/akka-persistence-riak/src/main/scala/RiakStorageBackend.scala +++ b/akka-persistence/akka-persistence-riak/src/main/scala/RiakStorageBackend.scala @@ -2,16 +2,16 @@ * 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} 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..3c9a632d59 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 { 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..775fd59dca 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 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..5bf27398d6 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,11 @@ -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.util.{Logging} import collection.immutable.TreeSet import VoldemortStorageBackendSuite._ import scala.None 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..3cdcc09032 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 { 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 e5265ea396..dc44a063cf 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,35 +3612,35 @@ 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); } 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; @@ -3656,7 +3656,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; @@ -3683,7 +3683,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); } getUnknownFields().writeTo(output); @@ -3719,7 +3719,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); } @@ -3728,41 +3728,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)) { @@ -3771,7 +3771,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 { @@ -3782,12 +3782,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 { @@ -3797,25 +3797,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; } @@ -3824,7 +3824,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; } @@ -3834,24 +3834,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( @@ -3860,7 +3860,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."); @@ -3869,22 +3869,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()); } @@ -3905,7 +3905,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_); } @@ -3935,7 +3935,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()); } @@ -3944,7 +3944,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()); } @@ -3953,7 +3953,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()); } @@ -3966,7 +3966,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()); } @@ -3975,7 +3975,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()); } @@ -3984,7 +3984,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; @@ -3998,10 +3998,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(); } @@ -4009,16 +4009,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; } @@ -4027,7 +4027,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; } @@ -4035,10 +4035,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(); } @@ -4046,16 +4046,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; } @@ -4064,7 +4064,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; } @@ -4072,10 +4072,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(); } @@ -4083,16 +4083,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; } @@ -4101,7 +4101,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; } @@ -4127,10 +4127,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(); } @@ -4138,16 +4138,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; } @@ -4156,7 +4156,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; } @@ -4164,10 +4164,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(); } @@ -4175,16 +4175,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; } @@ -4193,52 +4193,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; @@ -4253,7 +4253,7 @@ public final class RemoteProtocol { static { defaultInstance = new RemoteRequestProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -4279,41 +4279,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; @@ -4331,21 +4331,21 @@ 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); } 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; @@ -4361,7 +4361,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; @@ -4388,7 +4388,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); } getUnknownFields().writeTo(output); @@ -4424,7 +4424,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); } @@ -4433,41 +4433,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)) { @@ -4476,7 +4476,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 { @@ -4487,12 +4487,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 { @@ -4502,25 +4502,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; } @@ -4529,7 +4529,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; } @@ -4539,24 +4539,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( @@ -4565,7 +4565,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."); @@ -4574,22 +4574,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()); } @@ -4610,7 +4610,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_); } @@ -4640,7 +4640,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()); } @@ -4649,7 +4649,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()); } @@ -4658,7 +4658,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()); } @@ -4667,7 +4667,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()); } @@ -4684,7 +4684,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; @@ -4698,10 +4698,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(); } @@ -4709,16 +4709,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; } @@ -4727,7 +4727,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; } @@ -4735,10 +4735,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(); } @@ -4746,16 +4746,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; } @@ -4764,7 +4764,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; } @@ -4772,10 +4772,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(); } @@ -4783,16 +4783,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; } @@ -4801,7 +4801,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; } @@ -4809,10 +4809,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(); } @@ -4820,16 +4820,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; } @@ -4838,7 +4838,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; } @@ -4879,47 +4879,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; @@ -4934,7 +4934,7 @@ public final class RemoteProtocol { static { defaultInstance = new RemoteReplyProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -4960,12 +4960,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; @@ -5021,41 +5021,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)) { @@ -5064,7 +5064,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 { @@ -5075,12 +5075,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 { @@ -5090,25 +5090,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; } @@ -5117,7 +5117,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; } @@ -5127,24 +5127,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( @@ -5153,27 +5153,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()); } @@ -5259,7 +5259,7 @@ public final class RemoteProtocol { static { defaultInstance = new UuidProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -5285,12 +5285,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; @@ -5346,41 +5346,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)) { @@ -5389,7 +5389,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 { @@ -5400,12 +5400,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 { @@ -5415,25 +5415,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; } @@ -5442,7 +5442,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; } @@ -5452,24 +5452,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( @@ -5478,27 +5478,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()); } @@ -5590,7 +5590,7 @@ public final class RemoteProtocol { static { defaultInstance = new MetadataEntryProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -5616,23 +5616,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; @@ -5663,41 +5663,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)) { @@ -5706,7 +5706,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 { @@ -5717,12 +5717,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 { @@ -5732,25 +5732,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; } @@ -5759,7 +5759,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; } @@ -5769,24 +5769,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( @@ -5795,27 +5795,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()); } @@ -5846,7 +5846,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 { @@ -5863,10 +5863,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(); } @@ -5876,7 +5876,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; } @@ -5885,7 +5885,7 @@ public final class RemoteProtocol { static { defaultInstance = new LifeCycleProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -5911,12 +5911,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; @@ -5972,41 +5972,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)) { @@ -6015,7 +6015,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 { @@ -6026,12 +6026,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 { @@ -6041,25 +6041,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; } @@ -6068,7 +6068,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; } @@ -6078,24 +6078,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( @@ -6104,27 +6104,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()); } @@ -6213,7 +6213,7 @@ public final class RemoteProtocol { static { defaultInstance = new AddressProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -6239,12 +6239,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; @@ -6300,41 +6300,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)) { @@ -6343,7 +6343,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 { @@ -6354,12 +6354,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 { @@ -6369,25 +6369,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; } @@ -6396,7 +6396,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; } @@ -6406,24 +6406,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( @@ -6432,27 +6432,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()); } @@ -6544,7 +6544,7 @@ public final class RemoteProtocol { static { defaultInstance = new ExceptionProtocol(true); - se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit(); + akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -6682,8 +6682,8 @@ public final class RemoteProtocol { "ializationSchemeType\022\010\n\004JAVA\020\001\022\013\n\007SBINAR", "Y\020\002\022\016\n\nSCALA_JSON\020\003\022\r\n\tJAVA_JSON\020\004\022\014\n\010PR" + "OTOBUF\020\005*-\n\rLifeCycleType\022\r\n\tPERMANENT\020\001" + - "\022\r\n\tTEMPORARY\020\002B-\n)se.scalablesolutions." + - "akka.remote.protocolH\001" + "\022\r\n\tTEMPORARY\020\002B\030\n\024akka.remote.protocolH" + + "\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -6696,112 +6696,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", }, - 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", }, - 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 40c5756e04..ad32c83660 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 e39b83a503..3a9278267c 100644 --- a/akka-remote/src/main/scala/remote/RemoteClient.scala +++ b/akka-remote/src/main/scala/remote/RemoteClient.scala @@ -2,15 +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.{Exit, Actor, ActorRef, ActorType, RemoteActorRef, IllegalActorStateException} -import se.scalablesolutions.akka.dispatch.{DefaultCompletableFuture, CompletableFuture} -import se.scalablesolutions.akka.actor.{Uuid,newUuid,uuidFrom} -import se.scalablesolutions.akka.config.Config._ -import se.scalablesolutions.akka.serialization.RemoteActorSerialization._ -import se.scalablesolutions.akka.AkkaException +import 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._ import group.DefaultChannelGroup @@ -29,8 +29,8 @@ import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable.{HashSet, HashMap} import scala.reflect.BeanProperty -import se.scalablesolutions.akka.actor._ -import se.scalablesolutions.akka.util._ +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 f3c29c62f9..2fa16e7b32 100644 --- a/akka-remote/src/main/scala/remote/RemoteServer.scala +++ b/akka-remote/src/main/scala/remote/RemoteServer.scala @@ -2,23 +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} -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.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._ 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 922db92ad4..887048c995 100644 --- a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala +++ b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala @@ -2,18 +2,18 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -package se.scalablesolutions.akka.serialization +package akka.serialization -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, MessageSerializer} -import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.{ActorType => ActorTypeProtocol, _} +import akka.stm.global._ +import akka.stm.TransactionManagement._ +import akka.stm.TransactionManagement +import akka.dispatch.MessageInvocation +import akka.remote.{RemoteServer, MessageSerializer} +import 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/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 b0cbc5ec08..0f532af1b8 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 71ece9792e..e2f7f0c2fc 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-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 8348de2134..979545bfeb 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._ /* 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 2bde073df8..e91f73f84f 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/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 0ca834b8d4..025a1a8552 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
@@ -144,7 +144,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/microkernel-server.xml b/config/microkernel-server.xml
index d7b8087428..9efec8e287 100644
--- a/config/microkernel-server.xml
+++ b/config/microkernel-server.xml
@@ -73,7 +73,7 @@
              
                /
                
-	             se.scalablesolutions.akka.comet.AkkaServlet
+	             akka.comet.AkkaServlet
 	             /*
 	           
 	         
diff --git a/project/build.properties b/project/build.properties
index 4b70e4a190..a0dd769ee9 100644
--- a/project/build.properties
+++ b/project/build.properties
@@ -1,4 +1,4 @@
-project.organization=se.scalablesolutions.akka
+project.organization=akka
 project.name=akka
 project.version=1.0-SNAPSHOT
 scala.version=2.8.0
diff --git a/project/build/AkkaProject.scala b/project/build/AkkaProject.scala
index 5b93e30044..e18b421dc2 100644
--- a/project/build/AkkaProject.scala
+++ b/project/build/AkkaProject.scala
@@ -304,7 +304,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
   // Miscellaneous
   // -------------------------------------------------------------------------------------------------------------------
 
-  override def mainClass = Some("se.scalablesolutions.akka.kernel.Main")
+  override def mainClass = Some("akka.kernel.Main")
 
   override def packageOptions =
     manifestClassPath.map(cp => ManifestAttributes(
@@ -393,7 +393,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
       val artifactRE(path, artifactId, artifactVersion) = absPath
       val command = "mvn install:install-file" +
                     " -Dfile=" + absPath +
-                    " -DgroupId=se.scalablesolutions.akka" +
+                    " -DgroupId=akka" +
                     " -DartifactId=" + artifactId +
                     " -Dversion=" + version +
                     " -Dpackaging=jar -DgeneratePom=true"
@@ -720,7 +720,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
       "org.aopalliance.*;version=1.0.0",
 
       // Provided by other bundles
-      "!se.scalablesolutions.akka.*",
+      "!akka.*",
       "!com.google.inject.*",
       "!javax.transaction.*",
       "!javax.ws.rs.*",
@@ -841,7 +841,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
 
   class AkkaSampleOSGiProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) with BNDPlugin {
     val osgi_core = Dependencies.osgi_core
-    override lazy val bndBundleActivator = Some("se.scalablesolutions.akka.sample.osgi.Activator")
+    override lazy val bndBundleActivator = Some("akka.sample.osgi.Activator")
     override lazy val bndExportPackage = Nil // Necessary because of mixing-in AkkaDefaultProject which exports all ...akka.* packages!
   }
 
@@ -948,5 +948,5 @@ trait DeployProject { self: BasicScalaProject =>
 }
 
 trait OSGiProject extends BNDPlugin { self: DefaultProject =>
-  override def bndExportPackage = Seq("se.scalablesolutions.akka.*;version=%s".format(projectVersion.value))
+  override def bndExportPackage = Seq("akka.*;version=%s".format(projectVersion.value))
 }

From 103969f1f51972086b252e4c2839cef3a1cdcd73 Mon Sep 17 00:00:00 2001
From: Viktor Klang 
Date: Tue, 26 Oct 2010 13:32:44 +0200
Subject: [PATCH 25/39] Fixing missing renames of se.scalablesolutions

---
 config/logback-test.xml | 2 +-
 config/logback.xml      | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/config/logback-test.xml b/config/logback-test.xml
index 0671694e3f..f678c406ae 100644
--- a/config/logback-test.xml
+++ b/config/logback-test.xml
@@ -15,7 +15,7 @@
       [%4p] [%d{ISO8601}] [%t] %c{1}: %m%n
     
   
-  
+  
   
   
   
diff --git a/config/logback.xml b/config/logback.xml
index 3e6ba75548..28b4fb5504 100644
--- a/config/logback.xml
+++ b/config/logback.xml
@@ -22,7 +22,7 @@
       ./logs/akka.log.%d{yyyy-MM-dd-HH}
     
   
-  
+  
   
     
     

From deed6c4465a9466219dab92eb83e2f97f7183c42 Mon Sep 17 00:00:00 2001
From: Viktor Klang 
Date: Tue, 26 Oct 2010 13:40:23 +0200
Subject: [PATCH 26/39] Fixing Akka Camel with the new package

---
 .../main/scala/{se/scalablesolutions => }/akka/amqp/AMQP.scala  | 0
 .../{se/scalablesolutions => }/akka/amqp/AMQPMessage.scala      | 0
 .../{se/scalablesolutions => }/akka/amqp/ConsumerActor.scala    | 0
 .../{se/scalablesolutions => }/akka/amqp/ExampleSession.scala   | 0
 .../{se/scalablesolutions => }/akka/amqp/ExchangeType.scala     | 0
 .../akka/amqp/FaultTolerantChannelActor.scala                   | 0
 .../akka/amqp/FaultTolerantConnectionActor.scala                | 0
 .../{se/scalablesolutions => }/akka/amqp/ProducerActor.scala    | 0
 .../scala/{se/scalablesolutions => }/akka/amqp/rpc/RPC.scala    | 0
 .../scalablesolutions => }/akka/amqp/rpc/RpcClientActor.scala   | 0
 .../scalablesolutions => }/akka/amqp/rpc/RpcServerActor.scala   | 0
 .../META-INF/services/org/apache/camel/component/actor          | 2 +-
 .../META-INF/services/org/apache/camel/component/typed-actor    | 2 +-
 .../scalablesolutions => }/akka/camel/MessageJavaTestBase.java  | 0
 .../akka/camel/SampleRemoteTypedConsumer.java                   | 0
 .../akka/camel/SampleRemoteTypedConsumerImpl.java               | 0
 .../akka/camel/SampleRemoteUntypedConsumer.java                 | 0
 .../{se/scalablesolutions => }/akka/camel/SampleTypedActor.java | 0
 .../scalablesolutions => }/akka/camel/SampleTypedActorImpl.java | 0
 .../scalablesolutions => }/akka/camel/SampleTypedConsumer.java  | 0
 .../akka/camel/SampleTypedConsumerImpl.java                     | 0
 .../akka/camel/SampleTypedSingleConsumer.java                   | 0
 .../akka/camel/SampleTypedSingleConsumerImpl.java               | 0
 .../scalablesolutions => }/akka/camel/SampleUntypedActor.java   | 0
 .../akka/camel/SampleUntypedConsumer.java                       | 0
 .../akka/camel/SampleUntypedConsumerBlocking.java               | 0
 .../akka/camel/SampleUntypedForwardingProducer.java             | 0
 .../akka/camel/SampleUntypedReplyingProducer.java               | 0
 akka-samples/akka-sample-ants/src/main/spde/Ants.spde           | 2 +-
 akka-spring/src/main/resources/META-INF/spring.handlers         | 2 +-
 akka-spring/src/main/resources/META-INF/spring.schemas          | 2 +-
 .../{se/scalablesolutions => }/akka/spring/akka-0.10.xsd        | 0
 .../scalablesolutions => }/akka/spring/akka-1.0-SNAPSHOT.xsd    | 0
 33 files changed, 5 insertions(+), 5 deletions(-)
 rename akka-amqp/src/main/scala/{se/scalablesolutions => }/akka/amqp/AMQP.scala (100%)
 rename akka-amqp/src/main/scala/{se/scalablesolutions => }/akka/amqp/AMQPMessage.scala (100%)
 rename akka-amqp/src/main/scala/{se/scalablesolutions => }/akka/amqp/ConsumerActor.scala (100%)
 rename akka-amqp/src/main/scala/{se/scalablesolutions => }/akka/amqp/ExampleSession.scala (100%)
 rename akka-amqp/src/main/scala/{se/scalablesolutions => }/akka/amqp/ExchangeType.scala (100%)
 rename akka-amqp/src/main/scala/{se/scalablesolutions => }/akka/amqp/FaultTolerantChannelActor.scala (100%)
 rename akka-amqp/src/main/scala/{se/scalablesolutions => }/akka/amqp/FaultTolerantConnectionActor.scala (100%)
 rename akka-amqp/src/main/scala/{se/scalablesolutions => }/akka/amqp/ProducerActor.scala (100%)
 rename akka-amqp/src/main/scala/{se/scalablesolutions => }/akka/amqp/rpc/RPC.scala (100%)
 rename akka-amqp/src/main/scala/{se/scalablesolutions => }/akka/amqp/rpc/RpcClientActor.scala (100%)
 rename akka-amqp/src/main/scala/{se/scalablesolutions => }/akka/amqp/rpc/RpcServerActor.scala (100%)
 rename akka-camel/src/test/java/{se/scalablesolutions => }/akka/camel/MessageJavaTestBase.java (100%)
 rename akka-camel/src/test/java/{se/scalablesolutions => }/akka/camel/SampleRemoteTypedConsumer.java (100%)
 rename akka-camel/src/test/java/{se/scalablesolutions => }/akka/camel/SampleRemoteTypedConsumerImpl.java (100%)
 rename akka-camel/src/test/java/{se/scalablesolutions => }/akka/camel/SampleRemoteUntypedConsumer.java (100%)
 rename akka-camel/src/test/java/{se/scalablesolutions => }/akka/camel/SampleTypedActor.java (100%)
 rename akka-camel/src/test/java/{se/scalablesolutions => }/akka/camel/SampleTypedActorImpl.java (100%)
 rename akka-camel/src/test/java/{se/scalablesolutions => }/akka/camel/SampleTypedConsumer.java (100%)
 rename akka-camel/src/test/java/{se/scalablesolutions => }/akka/camel/SampleTypedConsumerImpl.java (100%)
 rename akka-camel/src/test/java/{se/scalablesolutions => }/akka/camel/SampleTypedSingleConsumer.java (100%)
 rename akka-camel/src/test/java/{se/scalablesolutions => }/akka/camel/SampleTypedSingleConsumerImpl.java (100%)
 rename akka-camel/src/test/java/{se/scalablesolutions => }/akka/camel/SampleUntypedActor.java (100%)
 rename akka-camel/src/test/java/{se/scalablesolutions => }/akka/camel/SampleUntypedConsumer.java (100%)
 rename akka-camel/src/test/java/{se/scalablesolutions => }/akka/camel/SampleUntypedConsumerBlocking.java (100%)
 rename akka-camel/src/test/java/{se/scalablesolutions => }/akka/camel/SampleUntypedForwardingProducer.java (100%)
 rename akka-camel/src/test/java/{se/scalablesolutions => }/akka/camel/SampleUntypedReplyingProducer.java (100%)
 rename akka-spring/src/main/resources/{se/scalablesolutions => }/akka/spring/akka-0.10.xsd (100%)
 rename akka-spring/src/main/resources/{se/scalablesolutions => }/akka/spring/akka-1.0-SNAPSHOT.xsd (100%)

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 100%
rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQP.scala
rename to akka-amqp/src/main/scala/akka/amqp/AMQP.scala
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 100%
rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQPMessage.scala
rename to akka-amqp/src/main/scala/akka/amqp/AMQPMessage.scala
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 100%
rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala
rename to akka-amqp/src/main/scala/akka/amqp/ConsumerActor.scala
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 100%
rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala
rename to akka-amqp/src/main/scala/akka/amqp/ExampleSession.scala
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 100%
rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExchangeType.scala
rename to akka-amqp/src/main/scala/akka/amqp/ExchangeType.scala
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 100%
rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala
rename to akka-amqp/src/main/scala/akka/amqp/FaultTolerantChannelActor.scala
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 100%
rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala
rename to akka-amqp/src/main/scala/akka/amqp/FaultTolerantConnectionActor.scala
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 100%
rename from akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ProducerActor.scala
rename to akka-amqp/src/main/scala/akka/amqp/ProducerActor.scala
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 100%
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
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 100%
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
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 100%
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
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/test/java/se/scalablesolutions/akka/camel/MessageJavaTestBase.java b/akka-camel/src/test/java/akka/camel/MessageJavaTestBase.java
similarity index 100%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/MessageJavaTestBase.java
rename to akka-camel/src/test/java/akka/camel/MessageJavaTestBase.java
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 100%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteTypedConsumer.java
rename to akka-camel/src/test/java/akka/camel/SampleRemoteTypedConsumer.java
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 100%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteTypedConsumerImpl.java
rename to akka-camel/src/test/java/akka/camel/SampleRemoteTypedConsumerImpl.java
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 100%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteUntypedConsumer.java
rename to akka-camel/src/test/java/akka/camel/SampleRemoteUntypedConsumer.java
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 100%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedActor.java
rename to akka-camel/src/test/java/akka/camel/SampleTypedActor.java
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 100%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedActorImpl.java
rename to akka-camel/src/test/java/akka/camel/SampleTypedActorImpl.java
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 100%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedConsumer.java
rename to akka-camel/src/test/java/akka/camel/SampleTypedConsumer.java
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 100%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedConsumerImpl.java
rename to akka-camel/src/test/java/akka/camel/SampleTypedConsumerImpl.java
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 100%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedSingleConsumer.java
rename to akka-camel/src/test/java/akka/camel/SampleTypedSingleConsumer.java
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 100%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedSingleConsumerImpl.java
rename to akka-camel/src/test/java/akka/camel/SampleTypedSingleConsumerImpl.java
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 100%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedActor.java
rename to akka-camel/src/test/java/akka/camel/SampleUntypedActor.java
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 100%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedConsumer.java
rename to akka-camel/src/test/java/akka/camel/SampleUntypedConsumer.java
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 100%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedConsumerBlocking.java
rename to akka-camel/src/test/java/akka/camel/SampleUntypedConsumerBlocking.java
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 100%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedForwardingProducer.java
rename to akka-camel/src/test/java/akka/camel/SampleUntypedForwardingProducer.java
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 100%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedReplyingProducer.java
rename to akka-camel/src/test/java/akka/camel/SampleUntypedReplyingProducer.java
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-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

From 4a43c933a1eb9d88af1f8e40642d7f2ec77cd9bc Mon Sep 17 00:00:00 2001
From: Viktor Klang 
Date: Tue, 26 Oct 2010 14:23:55 +0200
Subject: [PATCH 27/39] Switching to non-SSL repo for jBoss

---
 akka-sbt-plugin/src/main/scala/AkkaProject.scala | 2 +-
 project/build/AkkaProject.scala                  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/akka-sbt-plugin/src/main/scala/AkkaProject.scala b/akka-sbt-plugin/src/main/scala/AkkaProject.scala
index 2bde073df8..82ccbe401a 100644
--- a/akka-sbt-plugin/src/main/scala/AkkaProject.scala
+++ b/akka-sbt-plugin/src/main/scala/AkkaProject.scala
@@ -4,7 +4,7 @@ object AkkaRepositories {
   val AkkaRepo               = MavenRepository("Akka Repository", "http://scalablesolutions.se/akka/repository")
   val CodehausRepo           = MavenRepository("Codehaus Repo", "http://repository.codehaus.org")
   val GuiceyFruitRepo        = MavenRepository("GuiceyFruit Repo", "http://guiceyfruit.googlecode.com/svn/repo/releases/")
-  val JBossRepo              = MavenRepository("JBoss Repo", "https://repository.jboss.org/nexus/content/groups/public/")
+  val JBossRepo              = MavenRepository("JBoss Repo", "http://repository.jboss.org/nexus/content/groups/public/")
   val JavaNetRepo            = MavenRepository("java.net Repo", "http://download.java.net/maven/2")
   val SonatypeSnapshotRepo   = MavenRepository("Sonatype OSS Repo", "http://oss.sonatype.org/content/repositories/releases")
   val SunJDMKRepo            = MavenRepository("Sun JDMK Repo", "http://wp5.e-taxonomy.eu/cdmlib/mavenrepo")
diff --git a/project/build/AkkaProject.scala b/project/build/AkkaProject.scala
index 5b93e30044..6add9542ee 100644
--- a/project/build/AkkaProject.scala
+++ b/project/build/AkkaProject.scala
@@ -72,7 +72,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
     lazy val EmbeddedRepo         = MavenRepository("Embedded Repo", (info.projectPath / "embedded-repo").asURL.toString)
     lazy val FusesourceSnapshotRepo = MavenRepository("Fusesource Snapshots", "http://repo.fusesource.com/nexus/content/repositories/snapshots")
     lazy val GuiceyFruitRepo      = MavenRepository("GuiceyFruit Repo", "http://guiceyfruit.googlecode.com/svn/repo/releases/")
-    lazy val JBossRepo            = MavenRepository("JBoss Repo", "https://repository.jboss.org/nexus/content/groups/public/")
+    lazy val JBossRepo            = MavenRepository("JBoss Repo", "http://repository.jboss.org/nexus/content/groups/public/")
     lazy val JavaNetRepo          = MavenRepository("java.net Repo", "http://download.java.net/maven/2")
     lazy val SonatypeSnapshotRepo = MavenRepository("Sonatype OSS Repo", "http://oss.sonatype.org/content/repositories/releases")
     lazy val SunJDMKRepo          = MavenRepository("Sun JDMK Repo", "http://wp5.e-taxonomy.eu/cdmlib/mavenrepo")

From e300b76fbe61c5939bd52940727d7fbb7f859e32 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Jonas=20Bon=C3=A9r?= 
Date: Tue, 26 Oct 2010 12:04:32 +0200
Subject: [PATCH 28/39] Added Erlang-style secure cookie authentication for
 remote client/server

---
 akka-actor/src/main/scala/config/Config.scala |   4 -
 akka-actor/src/main/scala/util/Crypt.scala    |  45 +++---
 akka-actor/src/main/scala/util/Helpers.scala  |  56 ++++++++
 .../akka/remote/protocol/RemoteProtocol.java  | 131 +++++++++++++++---
 .../src/main/protocol/RemoteProtocol.proto    |   2 +
 .../src/main/scala/remote/RemoteClient.scala  |  10 +-
 .../src/main/scala/remote/RemoteServer.scala  |  63 ++++++---
 .../serialization/SerializationProtocol.scala |  10 +-
 config/akka-reference.conf                    |   6 +-
 scripts/generate_secure_cookie.sh             |   4 +-
 10 files changed, 262 insertions(+), 69 deletions(-)

diff --git a/akka-actor/src/main/scala/config/Config.scala b/akka-actor/src/main/scala/config/Config.scala
index e97347754b..16ed6d76c4 100644
--- a/akka-actor/src/main/scala/config/Config.scala
+++ b/akka-actor/src/main/scala/config/Config.scala
@@ -27,10 +27,6 @@ object ConfigLogger extends Logging
 object Config {
   val VERSION = "1.0-SNAPSHOT"
 
-  // Set Multiverse options for max speed
-  System.setProperty("org.multiverse.MuliverseConstants.sanityChecks", "false")
-  System.setProperty("org.multiverse.api.GlobalStmInstance.factorymethod", "org.multiverse.stms.alpha.AlphaStm.createFast")
-
   val HOME = {
     val envHome = System.getenv("AKKA_HOME") match {
       case null | "" | "." => None
diff --git a/akka-actor/src/main/scala/util/Crypt.scala b/akka-actor/src/main/scala/util/Crypt.scala
index e8cd2f82dd..fad9e4c281 100644
--- a/akka-actor/src/main/scala/util/Crypt.scala
+++ b/akka-actor/src/main/scala/util/Crypt.scala
@@ -9,28 +9,37 @@ import java.security.{MessageDigest, SecureRandom}
 /**
  * @author Jonas Bonér
  */
-object Crypt {
+object Crypt extends Logging {
+  val hex = "0123456789ABCDEF"
+  val lineSeparator = System.getProperty("line.separator")
+
   lazy val random = SecureRandom.getInstance("SHA1PRNG")
 
+  def md5(text: String): String        = md5(unifyLineSeparator(text).getBytes("ASCII"))
+
+  def md5(bytes: Array[Byte]): String  = digest(bytes, MessageDigest.getInstance("MD5"))
+                                             
+  def sha1(text: String): String       = sha1(unifyLineSeparator(text).getBytes("ASCII"))                                     
+
+  def sha1(bytes: Array[Byte]): String = digest(bytes, MessageDigest.getInstance("SHA1"))
+
   def generateSecureCookie: String = {
-    val bytes = Array.make(32, 0.byteValue)
+    log.info("Generating secure cookie...")
+    val bytes = Array.fill(32)(0.byteValue)
     random.nextBytes(bytes)
-    getMD5For(bytes)
+    sha1(bytes)
   }
 
-  def getMD5For(s: String): String = getMD5For(s.getBytes("ASCII"))
-
-  def getMD5For(b: Array[Byte]): String = {
-    val digest = MessageDigest.getInstance("MD5")
-    digest.update(b)
-    val bytes = digest.digest
-
-    val sb = new StringBuilder
-    val hex = "0123456789ABCDEF"
-    bytes.foreach { b =>
-      val n = b.asInstanceOf[Int]
-      sb.append(hex.charAt((n & 0xF) >> 4)).append(hex.charAt(n & 0xF))
-    }
-    sb.toString
+  def digest(bytes: Array[Byte], md: MessageDigest): String = {
+    md.update(bytes)
+    hexify(md.digest)
+  }                               
+  
+  def hexify(bytes: Array[Byte]): String = {
+    val builder = new StringBuilder
+    bytes.foreach { byte => builder.append(hex.charAt((byte & 0xF) >> 4)).append(hex.charAt(byte & 0xF)) }
+    builder.toString
   }
-}
\ No newline at end of file
+
+  private def unifyLineSeparator(text: String): String = text.replaceAll(lineSeparator, "\n")
+}
diff --git a/akka-actor/src/main/scala/util/Helpers.scala b/akka-actor/src/main/scala/util/Helpers.scala
index 75ba61620a..b9fa5d10b6 100644
--- a/akka-actor/src/main/scala/util/Helpers.scala
+++ b/akka-actor/src/main/scala/util/Helpers.scala
@@ -20,6 +20,10 @@ object Helpers extends Logging {
     bytes
   }
 
+  def bytesToInt(bytes: Array[Byte], offset: Int): Int = {
+    (0 until 4).foldLeft(0)((value, index) => value + ((bytes(index + offset) & 0x000000FF) << ((4 - 1 - index) * 8)))
+  }
+
   /**
    * Convenience helper to cast the given Option of Any to an Option of the given type. Will throw a ClassCastException
    * if the actual type is not assignable from the given one.
@@ -41,4 +45,56 @@ object Helpers extends Logging {
         log.warning(e, "Cannot narrow %s to expected type %s!", o, implicitly[Manifest[T]].erasure.getName)
         None
     }
+  
+  /**
+   * Reference that can hold either a typed value or an exception.
+   *
+   * Usage:
+   * 
+   * scala> ResultOrError(1)
+   * res0: ResultOrError[Int] = ResultOrError@a96606
+   *
+   * scala> res0()
+    res1: Int = 1
+   *
+   * scala> res0() = 3
+   *
+   * scala> res0()
+   * res3: Int = 3
+   * 
+   * scala> res0() = { println("Hello world"); 3}
+   * Hello world
+   *
+   * scala> res0()
+   * res5: Int = 3
+   *  
+   * scala> res0() = error("Lets see what happens here...")
+   *
+   * scala> res0()
+   * java.lang.RuntimeException: Lets see what happens here...
+   * 	at ResultOrError.apply(Helper.scala:11)
+   * 	at .(:6)
+   * 	at .()
+   * 	at Re...
+   * 
+ */ + class ResultOrError[R](result: R){ + private[this] var contents: Either[R, Throwable] = Left(result) + + def update(value: => R) = { + contents = try { + Left(value) + } catch { + case (error : Throwable) => Right(error) + } + } + + def apply() = contents match { + case Left(result) => result + case Right(error) => throw error.fillInStackTrace + } + } + object ResultOrError { + def apply[R](result: R) = new ResultOrError(result) + } } \ No newline at end of file diff --git a/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java b/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java index e5265ea396..60f5004861 100644 --- a/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java +++ b/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java @@ -3635,6 +3635,13 @@ public final class RemoteProtocol { return metadata_.get(index); } + // optional string cookie = 8; + public static final int COOKIE_FIELD_NUMBER = 8; + private boolean hasCookie; + private java.lang.String cookie_ = ""; + public boolean hasCookie() { return hasCookie; } + 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(); @@ -3686,6 +3693,9 @@ public final class RemoteProtocol { for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { output.writeMessage(7, element); } + if (hasCookie()) { + output.writeString(8, getCookie()); + } getUnknownFields().writeTo(output); } @@ -3723,6 +3733,10 @@ public final class RemoteProtocol { size += com.google.protobuf.CodedOutputStream .computeMessageSize(7, element); } + if (hasCookie()) { + size += com.google.protobuf.CodedOutputStream + .computeStringSize(8, getCookie()); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -3909,6 +3923,9 @@ public final class RemoteProtocol { } result.metadata_.addAll(other.metadata_); } + if (other.hasCookie()) { + setCookie(other.getCookie()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -3989,6 +4006,10 @@ public final class RemoteProtocol { addMetadata(subBuilder.buildPartial()); break; } + case 66: { + setCookie(input.readString()); + break; + } } } } @@ -4248,6 +4269,27 @@ public final class RemoteProtocol { return this; } + // optional string cookie = 8; + public boolean hasCookie() { + return result.hasCookie(); + } + public java.lang.String getCookie() { + return result.getCookie(); + } + public Builder setCookie(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + result.hasCookie = true; + result.cookie_ = value; + return this; + } + public Builder clearCookie() { + result.hasCookie = false; + result.cookie_ = getDefaultInstance().getCookie(); + return this; + } + // @@protoc_insertion_point(builder_scope:RemoteRequestProtocol) } @@ -4341,6 +4383,13 @@ public final class RemoteProtocol { return metadata_.get(index); } + // optional string cookie = 8; + public static final int COOKIE_FIELD_NUMBER = 8; + private boolean hasCookie; + private java.lang.String cookie_ = ""; + public boolean hasCookie() { return hasCookie; } + 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(); @@ -4391,6 +4440,9 @@ public final class RemoteProtocol { for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { output.writeMessage(7, element); } + if (hasCookie()) { + output.writeString(8, getCookie()); + } getUnknownFields().writeTo(output); } @@ -4428,6 +4480,10 @@ public final class RemoteProtocol { size += com.google.protobuf.CodedOutputStream .computeMessageSize(7, element); } + if (hasCookie()) { + size += com.google.protobuf.CodedOutputStream + .computeStringSize(8, getCookie()); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -4614,6 +4670,9 @@ public final class RemoteProtocol { } result.metadata_.addAll(other.metadata_); } + if (other.hasCookie()) { + setCookie(other.getCookie()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -4689,6 +4748,10 @@ public final class RemoteProtocol { addMetadata(subBuilder.buildPartial()); break; } + case 66: { + setCookie(input.readString()); + break; + } } } } @@ -4929,6 +4992,27 @@ public final class RemoteProtocol { return this; } + // optional string cookie = 8; + public boolean hasCookie() { + return result.hasCookie(); + } + public java.lang.String getCookie() { + return result.getCookie(); + } + public Builder setCookie(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + result.hasCookie = true; + result.cookie_ = value; + return this; + } + public Builder clearCookie() { + result.hasCookie = false; + result.cookie_ = getDefaultInstance().getCookie(); + return this; + } + // @@protoc_insertion_point(builder_scope:RemoteReplyProtocol) } @@ -6657,33 +6741,34 @@ public final class RemoteProtocol { "\004\022\035\n\tactorType\030\004 \002(\0162\n.ActorType\022/\n\016type" + "dActorInfo\030\005 \001(\0132\027.TypedActorInfoProtoco" + "l\022\n\n\002id\030\006 \001(\t\";\n\026TypedActorInfoProtocol\022" + - "\021\n\tinterface\030\001 \002(\t\022\016\n\006method\030\002 \002(\t\"\212\002\n\025R" + + "\021\n\tinterface\030\001 \002(\t\022\016\n\006method\030\002 \002(\t\"\232\002\n\025R" + "emoteRequestProtocol\022\033\n\004uuid\030\001 \002(\0132\r.Uui" + "dProtocol\022!\n\007message\030\002 \002(\0132\020.MessageProt", "ocol\022%\n\tactorInfo\030\003 \002(\0132\022.ActorInfoProto" + "col\022\020\n\010isOneWay\030\004 \002(\010\022%\n\016supervisorUuid\030" + "\005 \001(\0132\r.UuidProtocol\022\'\n\006sender\030\006 \001(\0132\027.R" + "emoteActorRefProtocol\022(\n\010metadata\030\007 \003(\0132" + - "\026.MetadataEntryProtocol\"\364\001\n\023RemoteReplyP" + - "rotocol\022\033\n\004uuid\030\001 \002(\0132\r.UuidProtocol\022!\n\007" + - "message\030\002 \001(\0132\020.MessageProtocol\022%\n\texcep" + - "tion\030\003 \001(\0132\022.ExceptionProtocol\022%\n\016superv" + - "isorUuid\030\004 \001(\0132\r.UuidProtocol\022\017\n\007isActor" + - "\030\005 \002(\010\022\024\n\014isSuccessful\030\006 \002(\010\022(\n\010metadata", - "\030\007 \003(\0132\026.MetadataEntryProtocol\")\n\014UuidPr" + - "otocol\022\014\n\004high\030\001 \002(\004\022\013\n\003low\030\002 \002(\004\"3\n\025Met" + - "adataEntryProtocol\022\013\n\003key\030\001 \002(\t\022\r\n\005value" + - "\030\002 \002(\014\"6\n\021LifeCycleProtocol\022!\n\tlifeCycle" + - "\030\001 \002(\0162\016.LifeCycleType\"1\n\017AddressProtoco" + - "l\022\020\n\010hostname\030\001 \002(\t\022\014\n\004port\030\002 \002(\r\"7\n\021Exc" + - "eptionProtocol\022\021\n\tclassname\030\001 \002(\t\022\017\n\007mes" + - "sage\030\002 \002(\t*=\n\tActorType\022\017\n\013SCALA_ACTOR\020\001" + - "\022\016\n\nJAVA_ACTOR\020\002\022\017\n\013TYPED_ACTOR\020\003*]\n\027Ser" + - "ializationSchemeType\022\010\n\004JAVA\020\001\022\013\n\007SBINAR", - "Y\020\002\022\016\n\nSCALA_JSON\020\003\022\r\n\tJAVA_JSON\020\004\022\014\n\010PR" + - "OTOBUF\020\005*-\n\rLifeCycleType\022\r\n\tPERMANENT\020\001" + - "\022\r\n\tTEMPORARY\020\002B-\n)se.scalablesolutions." + - "akka.remote.protocolH\001" + "\026.MetadataEntryProtocol\022\016\n\006cookie\030\010 \001(\t\"" + + "\204\002\n\023RemoteReplyProtocol\022\033\n\004uuid\030\001 \002(\0132\r." + + "UuidProtocol\022!\n\007message\030\002 \001(\0132\020.MessageP" + + "rotocol\022%\n\texception\030\003 \001(\0132\022.ExceptionPr" + + "otocol\022%\n\016supervisorUuid\030\004 \001(\0132\r.UuidPro" + + "tocol\022\017\n\007isActor\030\005 \002(\010\022\024\n\014isSuccessful\030\006", + " \002(\010\022(\n\010metadata\030\007 \003(\0132\026.MetadataEntryPr" + + "otocol\022\016\n\006cookie\030\010 \001(\t\")\n\014UuidProtocol\022\014" + + "\n\004high\030\001 \002(\004\022\013\n\003low\030\002 \002(\004\"3\n\025MetadataEnt" + + "ryProtocol\022\013\n\003key\030\001 \002(\t\022\r\n\005value\030\002 \002(\014\"6" + + "\n\021LifeCycleProtocol\022!\n\tlifeCycle\030\001 \002(\0162\016" + + ".LifeCycleType\"1\n\017AddressProtocol\022\020\n\010hos" + + "tname\030\001 \002(\t\022\014\n\004port\030\002 \002(\r\"7\n\021ExceptionPr" + + "otocol\022\021\n\tclassname\030\001 \002(\t\022\017\n\007message\030\002 \002" + + "(\t*=\n\tActorType\022\017\n\013SCALA_ACTOR\020\001\022\016\n\nJAVA" + + "_ACTOR\020\002\022\017\n\013TYPED_ACTOR\020\003*]\n\027Serializati", + "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" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -6751,7 +6836,7 @@ public final class RemoteProtocol { 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", }, + 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); internal_static_RemoteReplyProtocol_descriptor = @@ -6759,7 +6844,7 @@ public final class RemoteProtocol { 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", }, + 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); internal_static_UuidProtocol_descriptor = diff --git a/akka-remote/src/main/protocol/RemoteProtocol.proto b/akka-remote/src/main/protocol/RemoteProtocol.proto index 40c5756e04..ce694141a0 100644 --- a/akka-remote/src/main/protocol/RemoteProtocol.proto +++ b/akka-remote/src/main/protocol/RemoteProtocol.proto @@ -102,6 +102,7 @@ message RemoteRequestProtocol { optional UuidProtocol supervisorUuid = 5; optional RemoteActorRefProtocol sender = 6; repeated MetadataEntryProtocol metadata = 7; + optional string cookie = 8; } /** @@ -115,6 +116,7 @@ message RemoteReplyProtocol { required bool isActor = 5; required bool isSuccessful = 6; repeated MetadataEntryProtocol metadata = 7; + optional string cookie = 8; } /** diff --git a/akka-remote/src/main/scala/remote/RemoteClient.scala b/akka-remote/src/main/scala/remote/RemoteClient.scala index e39b83a503..1ddf57869e 100644 --- a/akka-remote/src/main/scala/remote/RemoteClient.scala +++ b/akka-remote/src/main/scala/remote/RemoteClient.scala @@ -59,7 +59,13 @@ class RemoteClientException private[akka](message: String, @BeanProperty val cli * @author Jonas Bonér */ object RemoteClient extends Logging { - val READ_TIMEOUT = Duration(config.getInt("akka.remote.client.read-timeout", 1), TIME_UNIT) + val SECURE_COOKIE: Option[String] = { + val cookie = config.getString("akka.remote.secure-cookie", "") + if (cookie == "") None + else Some(cookie) + } + + val READ_TIMEOUT = Duration(config.getInt("akka.remote.client.read-timeout", 1), TIME_UNIT) val RECONNECT_DELAY = Duration(config.getInt("akka.remote.client.reconnect-delay", 5), TIME_UNIT) private val remoteClients = new HashMap[String, RemoteClient] @@ -269,7 +275,7 @@ class RemoteClient private[akka] ( typedActorInfo: Option[Tuple2[String, String]], actorType: ActorType): Option[CompletableFuture[T]] = { send(createRemoteRequestProtocolBuilder( - actorRef, message, isOneWay, senderOption, typedActorInfo, actorType).build, senderFuture) + actorRef, message, isOneWay, senderOption, typedActorInfo, actorType, RemoteClient.SECURE_COOKIE).build, senderFuture) } def send[T]( diff --git a/akka-remote/src/main/scala/remote/RemoteServer.scala b/akka-remote/src/main/scala/remote/RemoteServer.scala index f3c29c62f9..0d39be263a 100644 --- a/akka-remote/src/main/scala/remote/RemoteServer.scala +++ b/akka-remote/src/main/scala/remote/RemoteServer.scala @@ -16,6 +16,7 @@ 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._ @@ -61,21 +62,30 @@ import scala.reflect.BeanProperty object RemoteNode extends RemoteServer /** - * For internal use only. - * Holds configuration variables, remote actors, remote typed actors and remote servers. + * For internal use only. Holds configuration variables, remote actors, remote typed actors and remote servers. * * @author Jonas Bonér */ -object -RemoteServer { +object RemoteServer { val UUID_PREFIX = "uuid:" - val HOSTNAME = config.getString("akka.remote.server.hostname", "localhost") - val PORT = config.getInt("akka.remote.server.port", 9999) + val SECURE_COOKIE: Option[String] = { + val cookie = config.getString("akka.remote.secure-cookie", "") + if (cookie == "") None + else Some(cookie) + } + val REQUIRE_COOKIE = { + val requireCookie = config.getBool("akka.remote.server.require-cookie", true) + if (RemoteServer.SECURE_COOKIE.isEmpty) throw new ConfigurationException( + "Configuration option 'akka.remote.server.require-cookie' is turned on but no secure cookie is defined in 'akka.remote.secure-cookie'.") + requireCookie + } + + val HOSTNAME = config.getString("akka.remote.server.hostname", "localhost") + val PORT = config.getInt("akka.remote.server.port", 9999) val CONNECTION_TIMEOUT_MILLIS = Duration(config.getInt("akka.remote.server.connection-timeout", 1), TIME_UNIT) - - val COMPRESSION_SCHEME = config.getString("akka.remote.compression-scheme", "zlib") - val ZLIB_COMPRESSION_LEVEL = { + val COMPRESSION_SCHEME = config.getString("akka.remote.compression-scheme", "zlib") + val ZLIB_COMPRESSION_LEVEL = { val level = config.getInt("akka.remote.zlib-compression-level", 6) if (level < 1 && level > 9) throw new IllegalArgumentException( "zlib compression level has to be within 1-9, with 1 being fastest and 9 being the most compressed") @@ -128,7 +138,6 @@ RemoteServer { private[akka] def unregister(hostname: String, port: Int) = guard.withWriteGuard { remoteServers.remove(Address(hostname, port)) } - } /** @@ -389,7 +398,7 @@ class RemoteServerPipelineFactory( val lenPrep = new LengthFieldPrepender(4) val protobufDec = new ProtobufDecoder(RemoteRequestProtocol.getDefaultInstance) val protobufEnc = new ProtobufEncoder - val (enc,dec) = RemoteServer.COMPRESSION_SCHEME match { + val (enc, dec) = RemoteServer.COMPRESSION_SCHEME match { case "zlib" => (join(new ZlibEncoder(RemoteServer.ZLIB_COMPRESSION_LEVEL)), join(new ZlibDecoder)) case _ => (join(), join()) } @@ -411,6 +420,7 @@ class RemoteServerHandler( val server: RemoteServer) extends SimpleChannelUpstreamHandler with Logging { import RemoteServer._ val AW_PROXY_PREFIX = "$$ProxiedByAW".intern + val CHANNEL_INIT = "channel-init".intern applicationLoader.foreach(MessageSerializer.setClassLoader(_)) @@ -437,6 +447,7 @@ class RemoteServerHandler( } else { server.notifyListeners(RemoteServerClientConnected(server)) } + if (RemoteServer.REQUIRE_COOKIE) ctx.setAttachment(CHANNEL_INIT) } override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { @@ -445,8 +456,7 @@ class RemoteServerHandler( } override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = { - if (event.isInstanceOf[ChannelStateEvent] && - event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) { + if (event.isInstanceOf[ChannelStateEvent] && event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) { log.debug(event.toString) } super.handleUpstream(ctx, event) @@ -456,7 +466,9 @@ class RemoteServerHandler( val message = event.getMessage if (message eq null) throw new IllegalActorStateException("Message in remote MessageEvent is null: " + event) if (message.isInstanceOf[RemoteRequestProtocol]) { - handleRemoteRequestProtocol(message.asInstanceOf[RemoteRequestProtocol], event.getChannel) + val requestProtocol = message.asInstanceOf[RemoteRequestProtocol] + authenticateRemoteClient(requestProtocol, ctx) + handleRemoteRequestProtocol(requestProtocol, event.getChannel) } } @@ -491,8 +503,11 @@ class RemoteServerHandler( case RemoteActorSystemMessage.Stop => actorRef.stop case _ => // then match on user defined messages if (request.getIsOneWay) actorRef.!(message)(sender) - else actorRef.postMessageToMailboxAndCreateFutureResultWithTimeout(message,request.getActorInfo.getTimeout,None,Some( - new DefaultCompletableFuture[AnyRef](request.getActorInfo.getTimeout){ + else actorRef.postMessageToMailboxAndCreateFutureResultWithTimeout( + message, + request.getActorInfo.getTimeout, + None, + Some(new DefaultCompletableFuture[AnyRef](request.getActorInfo.getTimeout){ override def onComplete(result: AnyRef) { log.debug("Returning result from actor invocation [%s]", result) val replyBuilder = RemoteReplyProtocol.newBuilder @@ -677,4 +692,20 @@ class RemoteServerHandler( if (request.hasSupervisorUuid) replyBuilder.setSupervisorUuid(request.getSupervisorUuid) replyBuilder.build } + + private def authenticateRemoteClient(request: RemoteRequestProtocol, ctx: ChannelHandlerContext) = { + if (RemoteServer.REQUIRE_COOKIE) { + val attachment = ctx.getAttachment + if ((attachment ne null) && + attachment.isInstanceOf[String] && + attachment.asInstanceOf[String] == CHANNEL_INIT) { + val clientAddress = ctx.getChannel.getRemoteAddress.toString + if (!request.hasCookie) throw new SecurityException( + "The remote client [" + clientAddress + "] does not have a secure cookie.") + if (!(request.getCookie == RemoteServer.SECURE_COOKIE.get)) throw new SecurityException( + "The remote client [" + clientAddress + "] secure cookie is not the same as remote server secure cookie") + log.info("Remote client [%s] successfully authenticated using secure cookie", clientAddress) + } + } + } } diff --git a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala index 922db92ad4..8383607f8c 100644 --- a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala +++ b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala @@ -8,7 +8,7 @@ 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, MessageSerializer} +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._ @@ -132,7 +132,8 @@ object ActorSerialization { false, actorRef.getSender, None, - ActorType.ScalaActor).build) + ActorType.ScalaActor, + RemoteClient.SECURE_COOKIE).build) requestProtocols.foreach(rp => builder.addMessages(rp)) } @@ -261,7 +262,8 @@ object RemoteActorSerialization { isOneWay: Boolean, senderOption: Option[ActorRef], typedActorInfo: Option[Tuple2[String, String]], - actorType: ActorType): + actorType: ActorType, + secureCookie: Option[String]): RemoteRequestProtocol.Builder = { import actorRef._ @@ -292,6 +294,8 @@ object RemoteActorSerialization { .setActorInfo(actorInfo) .setIsOneWay(isOneWay) + secureCookie.foreach(requestBuilder.setCookie(_)) + val id = registerSupervisorAsRemoteActor if (id.isDefined) requestBuilder.setSupervisorUuid( UuidProtocol.newBuilder diff --git a/config/akka-reference.conf b/config/akka-reference.conf index 0ca834b8d4..5017a18fed 100644 --- a/config/akka-reference.conf +++ b/config/akka-reference.conf @@ -7,7 +7,7 @@ akka { version = "1.0-SNAPSHOT" # Akka version, checked against the runtime version of Akka. - + time-unit = "seconds" # Default timeout time unit for all timeout properties throughout the config # These boot classes are loaded (and created) automatically when the Akka Microkernel boots up @@ -109,6 +109,9 @@ akka { } remote { + + secure-cookie = "050E0A0D0D06010A00000900040D060F0C09060B" # generate your own with '$AKKA_HOME/scripts/generate_secure_cookie.sh' or using 'Crypt.generateSecureCookie' + compression-scheme = "zlib" # Options: "zlib" (lzf to come), leave out for no compression zlib-compression-level = 6 # Options: 0-9 (1 being fastest and 9 being the most compressed), default is 6 @@ -133,6 +136,7 @@ akka { hostname = "localhost" # The hostname or IP that clients should connect to port = 9999 # The port clients should connect to connection-timeout = 1 + require-cookie = on } client { diff --git a/scripts/generate_secure_cookie.sh b/scripts/generate_secure_cookie.sh index 12ca10a940..bd3813793e 100755 --- a/scripts/generate_secure_cookie.sh +++ b/scripts/generate_secure_cookie.sh @@ -9,10 +9,10 @@ import java.security.{MessageDigest, SecureRandom} lazy val random = SecureRandom.getInstance("SHA1PRNG") -val buffer = Array.make(32, 0.byteValue) +val buffer = Array.fill(32)(0.byteValue) random.nextBytes(buffer) -val digest = MessageDigest.getInstance("MD5") +val digest = MessageDigest.getInstance("SHA1") digest.update(buffer) val bytes = digest.digest From 2b46fcee5066a01afb7667f7bd712a6fe4033e87 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 26 Oct 2010 15:53:34 +0200 Subject: [PATCH 29/39] Added support for remote agent --- akka-actor/src/main/scala/actor/Agent.scala | 63 +++++++++++++------ .../test/scala/remote/RemoteAgentSpec.scala | 37 +++++++++++ 2 files changed, 82 insertions(+), 18 deletions(-) create mode 100644 akka-remote/src/test/scala/remote/RemoteAgentSpec.scala diff --git a/akka-actor/src/main/scala/actor/Agent.scala b/akka-actor/src/main/scala/actor/Agent.scala index 6b9385ca4e..00dceba21c 100644 --- a/akka-actor/src/main/scala/actor/Agent.scala +++ b/akka-actor/src/main/scala/actor/Agent.scala @@ -9,6 +9,7 @@ import se.scalablesolutions.akka.AkkaException import se.scalablesolutions.akka.japi.{ Function => JFunc, Procedure => JProc } import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.CountDownLatch +import se.scalablesolutions.akka.config.RemoteAddress class AgentException private[akka](message: String) extends AkkaException(message) @@ -100,11 +101,20 @@ class AgentException private[akka](message: String) extends AkkaException(messag * @author Viktor Klang * @author Jonas Bonér */ -sealed class Agent[T] private (initialValue: T) { +sealed class Agent[T] private (initialValue: T, remote: Option[RemoteAddress] = None) { + import Agent._ import Actor._ - - private val dispatcher = actorOf(new AgentDispatcher[T](initialValue)).start + val dispatcher = remote match { + case Some(address) => + val d = actorOf(new AgentDispatcher[T]()) + d.makeRemote(remote.get.hostname,remote.get.port) + d.start + d ! Value(initialValue) + d + case None => + actorOf(new AgentDispatcher(initialValue)).start + } /** * Submits a request to read the internal state. @@ -117,11 +127,9 @@ sealed class Agent[T] private (initialValue: T) { if (dispatcher.isTransactionInScope) throw new AgentException( "Can't call Agent.get within an enclosing transaction."+ "\n\tWould block indefinitely.\n\tPlease refactor your code.") - val ref = new AtomicReference[T] - val latch = new CountDownLatch(1) - sendProc((v: T) => {ref.set(v); latch.countDown}) - latch.await - ref.get + val f = (dispatcher.!!![T](Read,java.lang.Long.MAX_VALUE)).await + if (f.exception.isDefined) throw f.exception.get + else f.result.getOrElse(throw new IllegalStateException("Agent remote request timed out")) } /** @@ -185,13 +193,13 @@ sealed class Agent[T] private (initialValue: T) { * Applies function with type 'T => B' to the agent's internal state and then returns a new agent with the result. * Does not change the value of the agent (this). */ - final def map[B](f: (T) => B): Agent[B] = Agent(f(get)) + final def map[B](f: (T) => B): Agent[B] = Agent(f(get),remote) /** * Applies function with type 'T => B' to the agent's internal state and then returns a new agent with the result. * Does not change the value of the agent (this). */ - final def flatMap[B](f: (T) => Agent[B]): Agent[B] = Agent(f(get)()) + final def flatMap[B](f: (T) => Agent[B]): Agent[B] = Agent(f(get)(),remote) /** * Applies function with type 'T => B' to the agent's internal state. @@ -204,14 +212,14 @@ sealed class Agent[T] private (initialValue: T) { * Does not change the value of the agent (this). * Java API */ - final def map[B](f: JFunc[T,B]): Agent[B] = Agent(f(get)) + final def map[B](f: JFunc[T,B]): Agent[B] = Agent(f(get),remote) /** * Applies function with type 'T => B' to the agent's internal state and then returns a new agent with the result. * Does not change the value of the agent (this). * Java API */ - final def flatMap[B](f: JFunc[T,Agent[B]]): Agent[B] = Agent(f(get)()) + final def flatMap[B](f: JFunc[T,Agent[B]]): Agent[B] = Agent(f(get)(),remote) /** * Applies procedure with type T to the agent's internal state. @@ -235,18 +243,33 @@ sealed class Agent[T] private (initialValue: T) { * @author Jonas Bonér */ object Agent { - + import Actor._ /* * The internal messages for passing around requests. */ private[akka] case class Value[T](value: T) private[akka] case class Function[T](fun: ((T) => T)) private[akka] case class Procedure[T](fun: ((T) => Unit)) + private[akka] case object Read /** * Creates a new Agent of type T with the initial value of value. */ - def apply[T](value: T): Agent[T] = new Agent(value) + def apply[T](value: T): Agent[T] = + apply(value,None) + + /** + * Creates an Agent backed by a client managed Actor if Some(remoteAddress) + * or a local agent if None + */ + def apply[T](value: T, remoteAddress: Option[RemoteAddress]): Agent[T] = + new Agent[T](value,remoteAddress) + + /** + * Creates an Agent backed by a client managed Actor + */ + def apply[T](value: T, remoteAddress: RemoteAddress): Agent[T] = + apply(value,Some(remoteAddress)) } /** @@ -254,12 +277,15 @@ object Agent { * * @author Jonas Bonér */ -final class AgentDispatcher[T] private[akka] (initialValue: T) extends Transactor { +final class AgentDispatcher[T] private (ref: Ref[T]) extends Transactor { import Agent._ - import Actor._ - log.debug("Starting up Agent [%s]", self.uuid) - private val value = Ref[T](initialValue) + private[akka] def this(initialValue: T) = this(Ref(initialValue)) + private[akka] def this() = this(Ref[T]()) + + private val value = ref + + log.debug("Starting up Agent [%s]", self.uuid) /** * Periodically handles incoming messages. @@ -267,6 +293,7 @@ final class AgentDispatcher[T] private[akka] (initialValue: T) extends Transacto def receive = { case Value(v: T) => swap(v) + case Read => self.reply_?(value.get()) case Function(fun: (T => T)) => swap(fun(value.getOrWait)) case Procedure(proc: (T => Unit)) => diff --git a/akka-remote/src/test/scala/remote/RemoteAgentSpec.scala b/akka-remote/src/test/scala/remote/RemoteAgentSpec.scala new file mode 100644 index 0000000000..02a93e949b --- /dev/null +++ b/akka-remote/src/test/scala/remote/RemoteAgentSpec.scala @@ -0,0 +1,37 @@ +package se.scalablesolutions.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} + + +class RemoteAgentSpec extends JUnitSuite { + var server: RemoteServer = _ + + val HOSTNAME = "localhost" + val PORT = 9992 + + @Before def startServer { + val s = new RemoteServer() + s.start(HOSTNAME, PORT) + server = s + Thread.sleep(1000) + } + + @After def stopServer { + val s = server + server = null + s.shutdown + RemoteClient.shutdownAll + } + + @Test def remoteAgentShouldInitializeProperly { + val a = Agent(10,RemoteAddress(HOSTNAME,PORT)) + assert(a() == 10, "Remote agent should have the proper initial value") + a(20) + assert(a() == 20, "Remote agent should be updated properly") + a.close + } +} \ No newline at end of file From 9b59bffa9bc7cb0c78342f0a7360c7671cf34594 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 26 Oct 2010 16:40:09 +0200 Subject: [PATCH 30/39] Adding possibility to take naps between scans for finished future, closing ticket #449 --- akka-actor/src/main/scala/dispatch/Future.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/dispatch/Future.scala b/akka-actor/src/main/scala/dispatch/Future.scala index ea06ebb4ec..57ea36d5d0 100644 --- a/akka-actor/src/main/scala/dispatch/Future.scala +++ b/akka-actor/src/main/scala/dispatch/Future.scala @@ -37,10 +37,16 @@ object Futures { def awaitAll(futures: List[Future[_]]): Unit = futures.foreach(_.await) - def awaitOne(futures: List[Future[_]]): Future[_] = { + /** + * Returns the First Future that is completed + * if no Future is completed, awaitOne optionally sleeps "sleepMs" millis and then re-scans + */ + def awaitOne(futures: List[Future[_]], sleepMs: Long = 0): Future[_] = { var future: Option[Future[_]] = None do { future = futures.find(_.isCompleted) + if (sleepMs > 0 && future.isEmpty) + Thread.sleep(sleepMs) } while (future.isEmpty) future.get } @@ -110,7 +116,7 @@ trait CompletableFuture[T] extends Future[T] { // Based on code from the actorom actor framework by Sergio Bossa [http://code.google.com/p/actorom/]. class DefaultCompletableFuture[T](timeout: Long) extends CompletableFuture[T] { - private val TIME_UNIT = TimeUnit.MILLISECONDS + import TimeUnit.{MILLISECONDS => TIME_UNIT} def this() = this(0) val timeoutInNanos = TIME_UNIT.toNanos(timeout) From ff81cfbf264f4b0d787e5ffad7d7bb81c3995f04 Mon Sep 17 00:00:00 2001 From: imn Date: Tue, 26 Oct 2010 19:58:14 +0200 Subject: [PATCH 31/39] refactoring the FSM part --- akka-actor/src/main/scala/actor/FSM.scala | 135 +++++++++++++----- .../test/scala/actor/actor/FSMActorSpec.scala | 51 +++++-- .../src/main/scala/DiningHakkersOnFsm.scala | 57 ++++---- 3 files changed, 167 insertions(+), 76 deletions(-) diff --git a/akka-actor/src/main/scala/actor/FSM.scala b/akka-actor/src/main/scala/actor/FSM.scala index 0bdc04fc48..c5eb00a6fd 100644 --- a/akka-actor/src/main/scala/actor/FSM.scala +++ b/akka-actor/src/main/scala/actor/FSM.scala @@ -4,58 +4,125 @@ package se.scalablesolutions.akka.actor -import se.scalablesolutions.akka.stm.Ref -import se.scalablesolutions.akka.stm.local._ - +import scala.collection.mutable import java.util.concurrent.{ScheduledFuture, TimeUnit} -trait FSM[S] { this: Actor => +trait FSM[S, D] { + this: Actor => type StateFunction = scala.PartialFunction[Event, State] - var currentState: State = initialState - var timeoutFuture: Option[ScheduledFuture[AnyRef]] = None + private var currentState: State = _ + private var timeoutFuture: Option[ScheduledFuture[AnyRef]] = None - def initialState: State + private val transitions = mutable.Map[S, StateFunction]() - def handleEvent: StateFunction = { - case event@Event(value, stateData) => - log.warning("No state for event with value %s - keeping current state %s at %s", value, stateData, self.id) - State(NextState, currentState.stateFunction, stateData, currentState.timeout) + private def register(name: S, function: StateFunction) { + if (transitions contains name) { + transitions(name) = transitions(name) orElse function + } else { + transitions(name) = function + } + } + + protected final def setInitialState(stateName: S, stateData: D, timeout: Option[Long] = None) = { + setState(State(stateName, stateData, timeout)) + } + + protected final def inState(stateName: S)(stateFunction: StateFunction) = { + register(stateName, stateFunction) + } + + protected final def goto(nextStateName: S): State = { + State(nextStateName, currentState.stateData) + } + + protected final def stay(): State = { + goto(currentState.stateName) + } + + protected final def reply(replyValue: Any): State = { + self.sender.foreach(_ ! replyValue) + stay() + } + + /** + * Stop + */ + protected final def stop(): State = { + stop(Normal) + } + + protected final def stop(reason: Reason): State = { + stop(reason, currentState.stateData) + } + + protected final def stop(reason: Reason, stateData: D): State = { + log.info("Stopped because of reason: %s", reason) + terminate(reason, currentState.stateName, stateData) + self.stop + State(currentState.stateName, stateData) + } + + def terminate(reason: Reason, stateName: S, stateData: D) = () + + def whenUnhandled(stateFunction: StateFunction) = { + handleEvent = stateFunction + } + + private var handleEvent: StateFunction = { + case Event(value, stateData) => + log.warning("Event %s not handled in state %s - keeping current state with data %s", value, currentState.stateName, stateData) + currentState } override final protected def receive: Receive = { + case StateTimeout if (self.dispatcher.mailboxSize(self) > 0) => () + // state timeout when new message in queue, skip this timeout case value => { timeoutFuture = timeoutFuture.flatMap {ref => ref.cancel(true); None} - val event = Event(value, currentState.stateData) - val newState = (currentState.stateFunction orElse handleEvent).apply(event) - - currentState = newState - - newState match { - case State(Reply, _, _, _, Some(replyValue)) => self.sender.foreach(_ ! replyValue) - case _ => () // ignore for now - } - - newState.timeout.foreach { - timeout => - timeoutFuture = Some(Scheduler.scheduleOnce(self, StateTimeout, timeout, TimeUnit.MILLISECONDS)) + val nextState = (transitions(currentState.stateName) orElse handleEvent).apply(event) + if (self.isRunning) { + setState(nextState) } } } - case class State(stateEvent: StateEvent, - stateFunction: StateFunction, - stateData: S, - timeout: Option[Int] = None, - replyValue: Option[Any] = None) + private def setState(nextState: State) = { + if (!transitions.contains(nextState.stateName)) { + stop(Failure("Next state %s not available".format(nextState.stateName))) + } else { + currentState = nextState + currentState.timeout.foreach {t => timeoutFuture = Some(Scheduler.scheduleOnce(self, StateTimeout, t, TimeUnit.MILLISECONDS))} + } + } - case class Event(event: Any, stateData: S) + case class Event(event: Any, stateData: D) - sealed trait StateEvent - object NextState extends StateEvent - object Reply extends StateEvent + case class State(stateName: S, stateData: D, timeout: Option[Long] = None) { + def until(timeout: Long): State = { + copy(timeout = Some(timeout)) + } - object StateTimeout + def then(nextStateName: S): State = { + copy(stateName = nextStateName) + } + + def replying(replyValue:Any): State = { + self.sender.foreach(_ ! replyValue) + this + } + + def using(nextStateDate: D): State = { + copy(stateData = nextStateDate) + } + } + + sealed trait Reason + case object Normal extends Reason + case object Shutdown extends Reason + case class Failure(cause: Any) extends Reason + + case object StateTimeout } diff --git a/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala b/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala index e4515bd3da..496d9e9e01 100644 --- a/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala +++ b/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala @@ -13,34 +13,44 @@ import java.util.concurrent.TimeUnit object FSMActorSpec { - class Lock(code: String, - timeout: Int, - unlockedLatch: StandardLatch, - lockedLatch: StandardLatch) extends Actor with FSM[CodeState] { + val unlockedLatch = new StandardLatch + val lockedLatch = new StandardLatch + val unhandledLatch = new StandardLatch - def initialState = State(NextState, locked, CodeState("", code)) + class Lock(code: String, timeout: Int) extends Actor with FSM[String, CodeState] { - def locked: StateFunction = { + inState("locked") { case Event(digit: Char, CodeState(soFar, code)) => { soFar + digit match { case incomplete if incomplete.length < code.length => - State(NextState, locked, CodeState(incomplete, code)) + stay using CodeState(incomplete, code) case codeTry if (codeTry == code) => { doUnlock - State(NextState, open, CodeState("", code), Some(timeout)) + goto("open") using CodeState("", code) until timeout } case wrong => { log.error("Wrong code %s", wrong) - State(NextState, locked, CodeState("", code)) + stay using CodeState("", code) } } } + case Event("hello", _) => stay replying "world" } - def open: StateFunction = { + inState("open") { case Event(StateTimeout, stateData) => { doLock - State(NextState, locked, stateData) + goto("locked") + } + } + + setInitialState("locked", CodeState("", code)) + + whenUnhandled { + case Event(_, stateData) => { + log.info("Unhandled") + unhandledLatch.open + stay } } @@ -63,11 +73,9 @@ class FSMActorSpec extends JUnitSuite { @Test def unlockTheLock = { - val unlockedLatch = new StandardLatch - val lockedLatch = new StandardLatch // lock that locked after being open for 1 sec - val lock = Actor.actorOf(new Lock("33221", 1000, unlockedLatch, lockedLatch)).start + val lock = Actor.actorOf(new Lock("33221", 1000)).start lock ! '3' lock ! '3' @@ -77,6 +85,21 @@ class FSMActorSpec extends JUnitSuite { assert(unlockedLatch.tryAwait(1, TimeUnit.SECONDS)) assert(lockedLatch.tryAwait(2, TimeUnit.SECONDS)) + + lock ! "not_handled" + assert(unhandledLatch.tryAwait(2, TimeUnit.SECONDS)) + + val answerLatch = new StandardLatch + object Go + val tester = Actor.actorOf(new Actor { + protected def receive = { + case Go => lock ! "hello" + case "world" => answerLatch.open + + } + }).start + tester ! Go + assert(answerLatch.tryAwait(2, TimeUnit.SECONDS)) } } 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 8348de2134..9ab27d4fbb 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala @@ -20,27 +20,27 @@ case class TakenBy(hakker: Option[ActorRef]) /* * A chopstick is an actor, it can be taken, and put back */ -class Chopstick(name: String) extends Actor with FSM[TakenBy] { +class Chopstick(name: String) extends Actor with FSM[String, TakenBy] { self.id = name - // A chopstick begins its existence as available and taken by no one - def initialState = State(NextState, available, TakenBy(None)) - // When a chopstick is available, it can be taken by a some hakker - def available: StateFunction = { + inState("available") { case Event(Take, _) => - State(Reply, taken, TakenBy(self.sender), replyValue = Some(Taken(self))) + goto("taken") using TakenBy(self.sender) replying Taken(self) } // 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 - def taken: StateFunction = { + inState("taken") { case Event(Take, currentState) => - State(Reply, taken, currentState, replyValue = Some(Busy(self))) + stay replying Busy(self) case Event(Put, TakenBy(hakker)) if self.sender == hakker => - State(NextState, available, TakenBy(None)) + goto("available") using TakenBy(None) } + + // A chopstick begins its existence as available and taken by no one + setInitialState("available", TakenBy(None)) } /** @@ -57,13 +57,10 @@ case class TakenChopsticks(left: Option[ActorRef], right: Option[ActorRef]) /* * A fsm hakker is an awesome dude or dudette who either thinks about hacking or has to eat ;-) */ -class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor with FSM[TakenChopsticks] { +class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor with FSM[String, TakenChopsticks] { self.id = name - //All hakkers start waiting - def initialState = State(NextState, waiting, TakenChopsticks(None, None)) - - def waiting: StateFunction = { + inState("waiting") { case Event(Think, _) => log.info("%s starts to think", name) startThinking(5000) @@ -71,30 +68,30 @@ 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 - def thinking: StateFunction = { - case Event(StateTimeout, current) => + inState("thinking") { + case Event(StateTimeout, _) => left ! Take right ! Take - State(NextState, hungry, current) + goto("hungry") } // When a hakker is hungry it tries to pick up its chopsticks and eat // 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 - def hungry: StateFunction = { + inState("hungry") { case Event(Taken(`left`), _) => - State(NextState, waitForOtherChopstick, TakenChopsticks(Some(left), None)) + goto("waitForOtherChopstick") using TakenChopsticks(Some(left), None) case Event(Taken(`right`), _) => - State(NextState, waitForOtherChopstick, TakenChopsticks(None, Some(right))) - case Event(Busy(_), current) => - State(NextState, firstChopstickDenied, current) + goto("waitForOtherChopstick") using TakenChopsticks(None, Some(right)) + case Event(Busy(_), _) => + goto("firstChopstickDenied") } // 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 :-) - def waitForOtherChopstick: StateFunction = { + inState("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)) => @@ -105,13 +102,13 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit private def startEating(left: ActorRef, right: ActorRef): State = { log.info("%s has picked up %s and %s, and starts to eat", name, left.id, right.id) - State(NextState, eating, TakenChopsticks(Some(left), Some(right)), timeout = Some(5000)) + goto("eating") using TakenChopsticks(Some(left), Some(right)) until 5000 } // 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 - def firstChopstickDenied: StateFunction = { + inState("firstChopstickDenied") { case Event(Taken(secondChopstick), _) => secondChopstick ! Put startThinking(10) @@ -121,7 +118,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 - def eating: StateFunction = { + inState("eating") { case Event(StateTimeout, _) => log.info("%s puts down his chopsticks and starts to think", name) left ! Put @@ -130,15 +127,19 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit } private def startThinking(period: Int): State = { - State(NextState, thinking, TakenChopsticks(None, None), timeout = Some(period)) + goto("thinking") using TakenChopsticks(None, None) until period } + + //All hakkers start waiting + setInitialState("waiting", TakenChopsticks(None, None)) } /* * Alright, here's our test-harness */ object DiningHakkersOnFSM { - def run { + def main(args: Array[String]) { + // Create 5 chopsticks val chopsticks = for (i <- 1 to 5) yield actorOf(new Chopstick("Chopstick " + i)).start // Create 5 awesome fsm hakkers and assign them their left and right chopstick From 90642f830352cc7d3b81ba9cb0a0c1d0683e2745 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bon=C3=A9r?= Date: Tue, 26 Oct 2010 23:43:16 +0200 Subject: [PATCH 32/39] Changed the script to spit out a full akka.conf file with the secure cookie --- ...h => generate_config_with_secure_cookie.sh} | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) rename scripts/{generate_secure_cookie.sh => generate_config_with_secure_cookie.sh} (62%) diff --git a/scripts/generate_secure_cookie.sh b/scripts/generate_config_with_secure_cookie.sh similarity index 62% rename from scripts/generate_secure_cookie.sh rename to scripts/generate_config_with_secure_cookie.sh index bd3813793e..bd7a83eda2 100755 --- a/scripts/generate_secure_cookie.sh +++ b/scripts/generate_config_with_secure_cookie.sh @@ -23,5 +23,19 @@ bytes.foreach { b => sb.append(hex.charAt((n & 0xF) >> 4)).append(hex.charAt(n & 0xF)) } -println("Cryptographically secure cookie:") -println(sb.toString) +print(""" +# This config imports the Akka reference configuration. +include "akka-reference.conf" + +# In this file you can override any option defined in the 'akka-reference.conf' file. +# Copy in all or parts of the 'akka-reference.conf' file and modify as you please. + +akka { + remote { + secure-cookie = """") +print(sb.toString) +print("""" + } +} +""") + From 83ab962bdac839f97a78cb2831ecf700e262b163 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bon=C3=A9r?= Date: Tue, 26 Oct 2010 23:44:13 +0200 Subject: [PATCH 33/39] converted tabs to spaces --- akka-actor/src/main/scala/util/Helpers.scala | 8 +- .../main/scala/CouchDBStorageBackend.scala | 190 +++++++++--------- .../HbaseStorageSpecTestIntegration.scala | 12 +- config/microkernel-server.xml | 42 ++-- project/build/AkkaProject.scala | 8 +- 5 files changed, 130 insertions(+), 130 deletions(-) diff --git a/akka-actor/src/main/scala/util/Helpers.scala b/akka-actor/src/main/scala/util/Helpers.scala index b9fa5d10b6..bc468fff3c 100644 --- a/akka-actor/src/main/scala/util/Helpers.scala +++ b/akka-actor/src/main/scala/util/Helpers.scala @@ -72,10 +72,10 @@ object Helpers extends Logging { * * scala> res0() * java.lang.RuntimeException: Lets see what happens here... - * at ResultOrError.apply(Helper.scala:11) - * at .(:6) - * at .() - * at Re... + * at ResultOrError.apply(Helper.scala:11) + * at .(:6) + * at .() + * at Re... *
*/ class ResultOrError[R](result: R){ 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 3be2657540..dcda8e8d3d 100644 --- a/akka-persistence/akka-persistence-couchdb/src/main/scala/CouchDBStorageBackend.scala +++ b/akka-persistence/akka-persistence-couchdb/src/main/scala/CouchDBStorageBackend.scala @@ -23,21 +23,21 @@ private [akka] object CouchDBStorageBackend extends RefStorageBackend[Array[Byte]] with Logging { - - import dispatch.json._ - - implicit object widgetWrites extends Writes[Map[String,Any]] { - def writes(o: Map[String,Any]): JsValue = JsValue(o) - } + + import dispatch.json._ + + implicit object widgetWrites extends Writes[Map[String,Any]] { + def writes(o: Map[String,Any]): JsValue = JsValue(o) + } - lazy val URL = config. + lazy val URL = config. getString("akka.storage.couchdb.url"). getOrElse(throw new IllegalArgumentException("'akka.storage.couchdb.url' not found in config")) def drop() = { val client = new HttpClient() - val delete = new DeleteMethod(URL) - client.executeMethod(delete) + val delete = new DeleteMethod(URL) + client.executeMethod(delete) } def create() = { @@ -45,60 +45,60 @@ private [akka] object CouchDBStorageBackend extends val put = new PutMethod(URL) put.setRequestEntity(new StringRequestEntity("", null, "utf-8")) put.setRequestHeader("Content-Type", "application/json") - client.executeMethod(put) + client.executeMethod(put) put.getResponseBodyAsString } - private def storeMap(name: String, postfix: String, entries: List[(Array[Byte], Array[Byte])]) ={ - var m = entries.map(e=>(new String(e._1) -> new String(e._2))).toMap + ("_id" -> (name + postfix)) - val dataJson = JsonSerialization.tojson(m) - postData(URL, dataJson.toString) - } - - private def storeMap(name: String, postfix: String, entries: Map[String, Any]) ={ + private def storeMap(name: String, postfix: String, entries: List[(Array[Byte], Array[Byte])]) ={ + var m = entries.map(e=>(new String(e._1) -> new String(e._2))).toMap + ("_id" -> (name + postfix)) + val dataJson = JsonSerialization.tojson(m) + postData(URL, dataJson.toString) + } + + private def storeMap(name: String, postfix: String, entries: Map[String, Any]) ={ postData(URL, JsonSerialization.tojson(entries + ("_id" -> (name + postfix))).toString) - } + } private def getResponseForNameAsMap(name: String, postfix: String): Option[Map[String, Any]] = { getResponse(URL + name + postfix).flatMap(JSON.parseFull(_)).asInstanceOf[Option[Map[String, Any]]] } - def insertMapStorageEntriesFor(name: String, entries: List[(Array[Byte], Array[Byte])]) ={ - val newDoc = getResponseForNameAsMap(name, "_map").getOrElse(Map[String, Any]()) ++ - entries.map(e => (new String(e._1) -> new String(e._2))).toMap + def insertMapStorageEntriesFor(name: String, entries: List[(Array[Byte], Array[Byte])]) ={ + val newDoc = getResponseForNameAsMap(name, "_map").getOrElse(Map[String, Any]()) ++ + entries.map(e => (new String(e._1) -> new String(e._2))).toMap storeMap(name, "_map", newDoc) - } - + } + def insertMapStorageEntryFor(name: String, key: Array[Byte], value: Array[Byte])={ - insertMapStorageEntriesFor(name, List((key, value))) - } - - + insertMapStorageEntriesFor(name, List((key, value))) + } + + def removeMapStorageFor(name: String) { lazy val url = URL + name + "_map" - findDocRev(name + "_map").foreach(deleteData(url, _)) - } - + findDocRev(name + "_map").foreach(deleteData(url, _)) + } + def removeMapStorageFor(name: String, key: Array[Byte]): Unit = { lazy val sKey = new String(key) // if we can't find the map for name, then we don't need to delete it. - getResponseForNameAsMap(name, "_map").foreach(doc => storeMap(name, "_map", doc - sKey)) - } - + getResponseForNameAsMap(name, "_map").foreach(doc => storeMap(name, "_map", doc - sKey)) + } + def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = { lazy val sKey = new String(key) getResponseForNameAsMap(name, "_map").flatMap(_.get(sKey)).asInstanceOf[Option[String]].map(_.getBytes) - } + } - def getMapStorageSizeFor(name: String): Int = getMapStorageFor(name).size - + def getMapStorageSizeFor(name: String): Int = getMapStorageFor(name).size + def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = { - val m = getResponseForNameAsMap(name, "_map").map(_ - ("_id", "_rev")).getOrElse(Map[String, Any]()) - m.toList.map(e => (e._1.getBytes, e._2.asInstanceOf[String].getBytes)) - } + val m = getResponseForNameAsMap(name, "_map").map(_ - ("_id", "_rev")).getOrElse(Map[String, Any]()) + m.toList.map(e => (e._1.getBytes, e._2.asInstanceOf[String].getBytes)) + } - def getMapStorageRangeFor(name: String, start: Option[Array[Byte]], finish: Option[Array[Byte]], count: Int): List[(Array[Byte], Array[Byte])] = { + def getMapStorageRangeFor(name: String, start: Option[Array[Byte]], finish: Option[Array[Byte]], count: Int): List[(Array[Byte], Array[Byte])] = { val m = getResponseForNameAsMap(name, "_map").map(_ - ("_id", "_rev")).getOrElse(Map[String, Any]()) val keys = m.keys.toList.sortWith(_ < _) @@ -112,7 +112,7 @@ private [akka] object CouchDBStorageBackend extends // slice from keys: both ends inclusive val ks = keys.slice(keys.indexOf(s), scala.math.min(keys.indexOf(s) + c, keys.indexOf(f) + 1)) ks.map(k => (k.getBytes, m(k).asInstanceOf[String].getBytes)) - } + } def insertVectorStorageEntryFor(name: String, element: Array[Byte]) = { insertVectorStorageEntriesFor(name, List(element)) @@ -133,16 +133,16 @@ private [akka] object CouchDBStorageBackend extends } def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] ={ - val v = getResponseForNameAsMap(name, "_vector").flatMap(_.get("vector")).getOrElse(List[String]()).asInstanceOf[List[String]] - if (v.indices.contains(index)) - v(index).getBytes - else - Array[Byte]() - } - + val v = getResponseForNameAsMap(name, "_vector").flatMap(_.get("vector")).getOrElse(List[String]()).asInstanceOf[List[String]] + if (v.indices.contains(index)) + v(index).getBytes + else + Array[Byte]() + } + def getVectorStorageSizeFor(name: String): Int ={ - getResponseForNameAsMap(name, "_vector").flatMap(_.get("vector")).map(_.asInstanceOf[List[String]].size).getOrElse(0) - } + getResponseForNameAsMap(name, "_vector").flatMap(_.get("vector")).map(_.asInstanceOf[List[String]].size).getOrElse(0) + } def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): List[Array[Byte]] = { val v = getResponseForNameAsMap(name, "_vector").flatMap(_.get("vector")).asInstanceOf[Option[List[String]]].getOrElse(List[String]()) @@ -151,60 +151,60 @@ private [akka] object CouchDBStorageBackend extends val c = if (count == 0) v.length else count v.slice(s, scala.math.min(s + c, f)).map(_.getBytes) } - + def insertRefStorageFor(name: String, element: Array[Byte]) ={ - val newDoc = getResponseForNameAsMap(name, "_ref").getOrElse(Map[String, Any]()) + ("ref" -> new String(element)) - storeMap(name, "_ref", newDoc) - } - - def getRefStorageFor(name: String): Option[Array[Byte]] ={ - getResponseForNameAsMap(name, "_ref").flatMap(_.get("ref")).map(_.asInstanceOf[String].getBytes) - } + val newDoc = getResponseForNameAsMap(name, "_ref").getOrElse(Map[String, Any]()) + ("ref" -> new String(element)) + storeMap(name, "_ref", newDoc) + } + + def getRefStorageFor(name: String): Option[Array[Byte]] ={ + getResponseForNameAsMap(name, "_ref").flatMap(_.get("ref")).map(_.asInstanceOf[String].getBytes) + } - private def findDocRev(name: String) = { - getResponse(URL + name).flatMap(JSON.parseFull(_)).asInstanceOf[Option[Map[String, Any]]] - .flatMap(_.get("_rev")).asInstanceOf[Option[String]] - } + private def findDocRev(name: String) = { + getResponse(URL + name).flatMap(JSON.parseFull(_)).asInstanceOf[Option[Map[String, Any]]] + .flatMap(_.get("_rev")).asInstanceOf[Option[String]] + } - private def deleteData(url:String, rev:String): Option[String] = { - val client = new HttpClient() - val delete = new DeleteMethod(url) - delete.setRequestHeader("If-Match", rev) - client.executeMethod(delete) - - val response = delete.getResponseBodyAsString() - if (response != null) - Some(response) - else - None - } + private def deleteData(url:String, rev:String): Option[String] = { + val client = new HttpClient() + val delete = new DeleteMethod(url) + delete.setRequestHeader("If-Match", rev) + client.executeMethod(delete) + + val response = delete.getResponseBodyAsString() + if (response != null) + Some(response) + else + None + } - private def postData(url: String, data: String): Option[String] = { - val client = new HttpClient() - val post = new PostMethod(url) - post.setRequestEntity(new StringRequestEntity(data, null, "utf-8")) - post.setRequestHeader("Content-Type", "application/json") - client.executeMethod(post) + private def postData(url: String, data: String): Option[String] = { + val client = new HttpClient() + val post = new PostMethod(url) + post.setRequestEntity(new StringRequestEntity(data, null, "utf-8")) + post.setRequestHeader("Content-Type", "application/json") + client.executeMethod(post) val response = post.getResponseBodyAsString if (response != null) - Some(response) - else - None - } - - private def getResponse(url: String): Option[String] = { - val client = new HttpClient() - val method = new GetMethod(url) + Some(response) + else + None + } + + private def getResponse(url: String): Option[String] = { + val client = new HttpClient() + val method = new GetMethod(url) - method.getParams().setParameter(HttpMethodParams.RETRY_HANDLER, + method.getParams().setParameter(HttpMethodParams.RETRY_HANDLER, new DefaultHttpMethodRetryHandler(3, false)) - client.executeMethod(method) + client.executeMethod(method) val response = method.getResponseBodyAsString - if (method.getStatusCode == 200 && response != null) - Some(response) - else - None - } + if (method.getStatusCode == 200 && response != null) + Some(response) + else + None + } } 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 4d118850f0..02f3c04172 100644 --- a/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseStorageSpecTestIntegration.scala +++ b/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseStorageSpecTestIntegration.scala @@ -82,12 +82,12 @@ BeforeAndAfterEach { ("guido van rossum", "python"), ("james strachan", "groovy")) val rl = List( - ("james gosling", "java"), - ("james strachan", "groovy"), - ("larry wall", "perl"), - ("martin odersky", "scala"), - ("ola bini", "ioke"), ("rich hickey", "clojure"), - ("slava pestov", "factor")) + ("james gosling", "java"), + ("james strachan", "groovy"), + ("larry wall", "perl"), + ("martin odersky", "scala"), + ("ola bini", "ioke"), ("rich hickey", "clojure"), + ("slava pestov", "factor")) insertMapStorageEntriesFor("t1", l.map { case (k, v) => (k.getBytes, v.getBytes) }) getMapStorageSizeFor("t1") should equal(l.size) getMapStorageRangeFor("t1", None, None, 100).map { case (k, v) => (new String(k), new String(v)) } should equal(l.sortWith(_._1 < _._1)) diff --git a/config/microkernel-server.xml b/config/microkernel-server.xml index d7b8087428..71160a6e54 100644 --- a/config/microkernel-server.xml +++ b/config/microkernel-server.xml @@ -38,29 +38,29 @@ 2 false 8443 - 20000 - 5000 + 20000 + 5000 + 30000 + 2 + 100 + /etc/keystore + PASSWORD + KEYPASSWORD + /etc/keystore + TRUSTPASSWORD + + + + --> @@ -73,10 +73,10 @@ / - se.scalablesolutions.akka.comet.AkkaServlet - /* - - + se.scalablesolutions.akka.comet.AkkaServlet + /* + + @@ -94,4 +94,4 @@ true 1000 - + diff --git a/project/build/AkkaProject.scala b/project/build/AkkaProject.scala index 6add9542ee..440c838d5c 100644 --- a/project/build/AkkaProject.scala +++ b/project/build/AkkaProject.scala @@ -139,7 +139,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { object Dependencies { // Compile - lazy val commonsHttpClient = "commons-httpclient" % "commons-httpclient" % "3.1" % "compile" + lazy val commonsHttpClient = "commons-httpclient" % "commons-httpclient" % "3.1" % "compile" lazy val annotation = "javax.annotation" % "jsr250-api" % "1.0" % "compile" @@ -606,7 +606,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { - + @@ -655,8 +655,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { } class AkkaCouchDBProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) { - val couch = Dependencies.commonsHttpClient - val spec = Dependencies.specs + val couch = Dependencies.commonsHttpClient + val spec = Dependencies.specs override def testOptions = createTestFilter( _.endsWith("Test")) } From 40ddac666c2010325cb8e6ac0fd5706995d6e4fb Mon Sep 17 00:00:00 2001 From: imn Date: Wed, 27 Oct 2010 11:41:35 +0200 Subject: [PATCH 34/39] use nice case objects for the states :-) --- .../test/scala/actor/actor/FSMActorSpec.scala | 16 +++-- .../src/main/scala/DiningHakkersOnFsm.scala | 62 ++++++++++++------- 2 files changed, 50 insertions(+), 28 deletions(-) diff --git a/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala b/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala index 496d9e9e01..8646dd5561 100644 --- a/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala +++ b/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala @@ -17,16 +17,20 @@ object FSMActorSpec { val lockedLatch = new StandardLatch val unhandledLatch = new StandardLatch - class Lock(code: String, timeout: Int) extends Actor with FSM[String, CodeState] { + sealed trait LockState + case object Locked extends LockState + case object Open extends LockState - inState("locked") { + class Lock(code: String, timeout: Int) extends Actor with FSM[LockState, CodeState] { + + inState(Locked) { case Event(digit: Char, CodeState(soFar, code)) => { soFar + digit match { case incomplete if incomplete.length < code.length => stay using CodeState(incomplete, code) case codeTry if (codeTry == code) => { doUnlock - goto("open") using CodeState("", code) until timeout + goto(Open) using CodeState("", code) until timeout } case wrong => { log.error("Wrong code %s", wrong) @@ -37,14 +41,14 @@ object FSMActorSpec { case Event("hello", _) => stay replying "world" } - inState("open") { + inState(Open) { case Event(StateTimeout, stateData) => { doLock - goto("locked") + goto(Locked) } } - setInitialState("locked", CodeState("", code)) + setInitialState(Locked, CodeState("", code)) whenUnhandled { case Event(_, stateData) => { 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 9ab27d4fbb..ecb4d82ba0 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala @@ -12,6 +12,13 @@ object Put extends ChopstickMessage case class Taken(chopstick: ActorRef) extends ChopstickMessage case class Busy(chopstick: ActorRef) extends ChopstickMessage +/** + * Some states the chopstick can be in + */ +sealed trait ChopstickState +case object Available extends ChopstickState +case object Taken extends ChopstickState + /** * Some state container for the chopstick */ @@ -20,27 +27,27 @@ case class TakenBy(hakker: Option[ActorRef]) /* * A chopstick is an actor, it can be taken, and put back */ -class Chopstick(name: String) extends Actor with FSM[String, TakenBy] { +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") { + inState(Available) { case Event(Take, _) => - goto("taken") using TakenBy(self.sender) replying Taken(self) + goto(Taken) using TakenBy(self.sender) replying Taken(self) } // 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") { + inState(Taken) { case Event(Take, currentState) => stay replying Busy(self) case Event(Put, TakenBy(hakker)) if self.sender == hakker => - goto("available") using TakenBy(None) + goto(Available) using TakenBy(None) } // A chopstick begins its existence as available and taken by no one - setInitialState("available", TakenBy(None)) + setInitialState(Available, TakenBy(None)) } /** @@ -49,6 +56,17 @@ class Chopstick(name: String) extends Actor with FSM[String, TakenBy] { sealed trait FSMHakkerMessage object Think extends FSMHakkerMessage +/** + * Some fsm hakker states + */ +sealed trait FSMHakkerState +case object Waiting extends FSMHakkerState +case object Thinking extends FSMHakkerState +case object Hungry extends FSMHakkerState +case object WaitForOtherChopstick extends FSMHakkerState +case object FirstChopstickDenied extends FSMHakkerState +case object Eating extends FSMHakkerState + /** * Some state container to keep track of which chopsticks we have */ @@ -57,10 +75,10 @@ case class TakenChopsticks(left: Option[ActorRef], right: Option[ActorRef]) /* * A fsm hakker is an awesome dude or dudette who either thinks about hacking or has to eat ;-) */ -class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor with FSM[String, TakenChopsticks] { +class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor with FSM[FSMHakkerState, TakenChopsticks] { self.id = name - inState("waiting") { + inState(Waiting) { case Event(Think, _) => log.info("%s starts to think", name) startThinking(5000) @@ -68,30 +86,30 @@ 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") { + inState(Thinking) { case Event(StateTimeout, _) => left ! Take right ! Take - goto("hungry") + goto(Hungry) } // When a hakker is hungry it tries to pick up its chopsticks and eat // 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") { + inState(Hungry) { case Event(Taken(`left`), _) => - goto("waitForOtherChopstick") using TakenChopsticks(Some(left), None) + goto(WaitForOtherChopstick) using TakenChopsticks(Some(left), None) case Event(Taken(`right`), _) => - goto("waitForOtherChopstick") using TakenChopsticks(None, Some(right)) + goto(WaitForOtherChopstick) using TakenChopsticks(None, Some(right)) case Event(Busy(_), _) => - goto("firstChopstickDenied") + goto(FirstChopstickDenied) } // 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") { + inState(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)) => @@ -102,13 +120,13 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit private def startEating(left: ActorRef, right: ActorRef): State = { log.info("%s has picked up %s and %s, and starts to eat", name, left.id, right.id) - goto("eating") using TakenChopsticks(Some(left), Some(right)) until 5000 + goto(Eating) using TakenChopsticks(Some(left), Some(right)) until 5000 } // 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") { + inState(FirstChopstickDenied) { case Event(Taken(secondChopstick), _) => secondChopstick ! Put startThinking(10) @@ -118,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") { + inState(Eating) { case Event(StateTimeout, _) => log.info("%s puts down his chopsticks and starts to think", name) left ! Put @@ -127,19 +145,19 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit } private def startThinking(period: Int): State = { - goto("thinking") using TakenChopsticks(None, None) until period + goto(Thinking) using TakenChopsticks(None, None) until period } //All hakkers start waiting - setInitialState("waiting", TakenChopsticks(None, None)) + setInitialState(Waiting, TakenChopsticks(None, None)) } /* * Alright, here's our test-harness */ -object DiningHakkersOnFSM { - def main(args: Array[String]) { +object DiningHakkersOnFsm { + def run = { // Create 5 chopsticks val chopsticks = for (i <- 1 to 5) yield actorOf(new Chopstick("Chopstick " + i)).start // Create 5 awesome fsm hakkers and assign them their left and right chopstick From 658b073ace5c60f681045d2c48a94fa331555e82 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bon=C3=A9r?= Date: Wed, 27 Oct 2010 13:22:38 +0200 Subject: [PATCH 35/39] Improved secure cookie generation script --- scripts/generate_config_with_secure_cookie.sh | 45 ++++++++++++++----- 1 file changed, 33 insertions(+), 12 deletions(-) diff --git a/scripts/generate_config_with_secure_cookie.sh b/scripts/generate_config_with_secure_cookie.sh index bd7a83eda2..899ec22025 100755 --- a/scripts/generate_config_with_secure_cookie.sh +++ b/scripts/generate_config_with_secure_cookie.sh @@ -7,20 +7,41 @@ exec scala "$0" "$@" */ import java.security.{MessageDigest, SecureRandom} -lazy val random = SecureRandom.getInstance("SHA1PRNG") +/** + * @author Jonas Bonér + */ +object Crypt { + val hex = "0123456789ABCDEF" + val lineSeparator = System.getProperty("line.separator") -val buffer = Array.fill(32)(0.byteValue) -random.nextBytes(buffer) + lazy val random = SecureRandom.getInstance("SHA1PRNG") -val digest = MessageDigest.getInstance("SHA1") -digest.update(buffer) -val bytes = digest.digest + def md5(text: String): String = md5(unifyLineSeparator(text).getBytes("ASCII")) -val sb = new StringBuilder -val hex = "0123456789ABCDEF" -bytes.foreach { b => - val n = b.asInstanceOf[Int] - sb.append(hex.charAt((n & 0xF) >> 4)).append(hex.charAt(n & 0xF)) + def md5(bytes: Array[Byte]): String = digest(bytes, MessageDigest.getInstance("MD5")) + + def sha1(text: String): String = sha1(unifyLineSeparator(text).getBytes("ASCII")) + + def sha1(bytes: Array[Byte]): String = digest(bytes, MessageDigest.getInstance("SHA1")) + + def generateSecureCookie: String = { + val bytes = Array.fill(32)(0.byteValue) + random.nextBytes(bytes) + sha1(bytes) + } + + def digest(bytes: Array[Byte], md: MessageDigest): String = { + md.update(bytes) + hexify(md.digest) + } + + def hexify(bytes: Array[Byte]): String = { + val builder = new StringBuilder + bytes.foreach { byte => builder.append(hex.charAt((byte & 0xF) >> 4)).append(hex.charAt(byte & 0xF)) } + builder.toString + } + + private def unifyLineSeparator(text: String): String = text.replaceAll(lineSeparator, "\n") } print(""" @@ -33,7 +54,7 @@ include "akka-reference.conf" akka { remote { secure-cookie = """") -print(sb.toString) +print(Crypt.generateSecureCookie) print("""" } } From 440d36aed9dc1c5f3ef0b1a390007dd999bbd4d8 Mon Sep 17 00:00:00 2001 From: imn Date: Wed, 27 Oct 2010 15:45:30 +0200 Subject: [PATCH 36/39] polishing up code --- akka-actor/src/main/scala/actor/FSM.scala | 89 ++++++++++--------- .../test/scala/actor/actor/FSMActorSpec.scala | 18 +++- 2 files changed, 61 insertions(+), 46 deletions(-) diff --git a/akka-actor/src/main/scala/actor/FSM.scala b/akka-actor/src/main/scala/actor/FSM.scala index c5eb00a6fd..eac861d358 100644 --- a/akka-actor/src/main/scala/actor/FSM.scala +++ b/akka-actor/src/main/scala/actor/FSM.scala @@ -12,27 +12,15 @@ trait FSM[S, D] { type StateFunction = scala.PartialFunction[Event, State] - private var currentState: State = _ - private var timeoutFuture: Option[ScheduledFuture[AnyRef]] = None - - private val transitions = mutable.Map[S, StateFunction]() - - private def register(name: S, function: StateFunction) { - if (transitions contains name) { - transitions(name) = transitions(name) orElse function - } else { - transitions(name) = function - } + /** DSL */ + protected final def inState(stateName: S)(stateFunction: StateFunction) = { + register(stateName, stateFunction) } protected final def setInitialState(stateName: S, stateData: D, timeout: Option[Long] = None) = { setState(State(stateName, stateData, timeout)) } - protected final def inState(stateName: S)(stateFunction: StateFunction) = { - register(stateName, stateFunction) - } - protected final def goto(nextStateName: S): State = { State(nextStateName, currentState.stateData) } @@ -41,14 +29,6 @@ trait FSM[S, D] { goto(currentState.stateName) } - protected final def reply(replyValue: Any): State = { - self.sender.foreach(_ ! replyValue) - stay() - } - - /** - * Stop - */ protected final def stop(): State = { stop(Normal) } @@ -58,59 +38,82 @@ trait FSM[S, D] { } protected final def stop(reason: Reason, stateData: D): State = { - log.info("Stopped because of reason: %s", reason) - terminate(reason, currentState.stateName, stateData) - self.stop - State(currentState.stateName, stateData) + self ! Stop(reason, stateData) + stay } - def terminate(reason: Reason, stateName: S, stateData: D) = () - def whenUnhandled(stateFunction: StateFunction) = { handleEvent = stateFunction } + def onTermination(terminationHandler: PartialFunction[Reason, Unit]) = { + terminateEvent = terminationHandler + } + + /** FSM State data and default handlers */ + private var currentState: State = _ + private var timeoutFuture: Option[ScheduledFuture[AnyRef]] = None + + private val transitions = mutable.Map[S, StateFunction]() + private def register(name: S, function: StateFunction) { + if (transitions contains name) { + transitions(name) = transitions(name) orElse function + } else { + transitions(name) = function + } + } + private var handleEvent: StateFunction = { case Event(value, stateData) => - log.warning("Event %s not handled in state %s - keeping current state with data %s", value, currentState.stateName, stateData) - currentState + log.warning("Event %s not handled in state %s, staying at current state", value, currentState.stateName) + stay + } + + private var terminateEvent: PartialFunction[Reason, Unit] = { + case failure@Failure(_) => log.error("Stopping because of a %s", failure) + case reason => log.info("Stopping because of reason: %s", reason) } override final protected def receive: Receive = { - case StateTimeout if (self.dispatcher.mailboxSize(self) > 0) => () + case Stop(reason, stateData) => + terminateEvent.apply(reason) + self.stop + case StateTimeout if (self.dispatcher.mailboxSize(self) > 0) => + log.trace("Ignoring StateTimeout - ") // state timeout when new message in queue, skip this timeout case value => { timeoutFuture = timeoutFuture.flatMap {ref => ref.cancel(true); None} val event = Event(value, currentState.stateData) val nextState = (transitions(currentState.stateName) orElse handleEvent).apply(event) - if (self.isRunning) { - setState(nextState) - } + setState(nextState) } } private def setState(nextState: State) = { if (!transitions.contains(nextState.stateName)) { - stop(Failure("Next state %s not available".format(nextState.stateName))) + stop(Failure("Next state %s does not exist".format(nextState.stateName))) } else { currentState = nextState - currentState.timeout.foreach {t => timeoutFuture = Some(Scheduler.scheduleOnce(self, StateTimeout, t, TimeUnit.MILLISECONDS))} + currentState.timeout.foreach { + t => + timeoutFuture = Some(Scheduler.scheduleOnce(self, StateTimeout, t, TimeUnit.MILLISECONDS)) + } } } case class Event(event: Any, stateData: D) case class State(stateName: S, stateData: D, timeout: Option[Long] = None) { + def until(timeout: Long): State = { copy(timeout = Some(timeout)) } - def then(nextStateName: S): State = { - copy(stateName = nextStateName) - } - def replying(replyValue:Any): State = { - self.sender.foreach(_ ! replyValue) + self.sender match { + case Some(sender) => sender ! replyValue + case None => log.error("Unable to send reply value %s, no sender reference to reply to", replyValue) + } this } @@ -125,4 +128,6 @@ trait FSM[S, D] { case class Failure(cause: Any) extends Reason case object StateTimeout + + private case class Stop(reason: Reason, stateData: D) } diff --git a/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala b/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala index 8646dd5561..dc6893c820 100644 --- a/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala +++ b/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala @@ -16,6 +16,7 @@ object FSMActorSpec { val unlockedLatch = new StandardLatch val lockedLatch = new StandardLatch val unhandledLatch = new StandardLatch + val terminatedLatch = new StandardLatch sealed trait LockState case object Locked extends LockState @@ -39,6 +40,7 @@ object FSMActorSpec { } } case Event("hello", _) => stay replying "world" + case Event("bye", _) => stop(Shutdown) } inState(Open) { @@ -58,6 +60,10 @@ object FSMActorSpec { } } + onTermination { + case reason => terminatedLatch.open + } + private def doLock() { log.info("Locked") lockedLatch.open @@ -94,16 +100,20 @@ class FSMActorSpec extends JUnitSuite { assert(unhandledLatch.tryAwait(2, TimeUnit.SECONDS)) val answerLatch = new StandardLatch - object Go + object Hello + object Bye val tester = Actor.actorOf(new Actor { protected def receive = { - case Go => lock ! "hello" + case Hello => lock ! "hello" case "world" => answerLatch.open - + case Bye => lock ! "bye" } }).start - tester ! Go + tester ! Hello assert(answerLatch.tryAwait(2, TimeUnit.SECONDS)) + + tester ! Bye + assert(terminatedLatch.tryAwait(2, TimeUnit.SECONDS)) } } From 620e2342a4fa8dabd5d34425ddb1a326ae5edbc9 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 28 Oct 2010 14:27:48 +0200 Subject: [PATCH 37/39] Bumping Jackson version to 1.4.3 --- project/build/AkkaProject.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/project/build/AkkaProject.scala b/project/build/AkkaProject.scala index 456da23960..b579f391f5 100644 --- a/project/build/AkkaProject.scala +++ b/project/build/AkkaProject.scala @@ -122,7 +122,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { lazy val CASSANDRA_VERSION = "0.6.1" lazy val DISPATCH_VERSION = "0.7.4" lazy val HAWT_DISPATCH_VERSION = "1.0" - lazy val JACKSON_VERSION = "1.2.1" + lazy val JACKSON_VERSION = "1.4.3" lazy val JERSEY_VERSION = "1.3" lazy val MULTIVERSE_VERSION = "0.6.1" lazy val SCALATEST_VERSION = "1.2" @@ -186,7 +186,6 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { lazy val jackson = "org.codehaus.jackson" % "jackson-mapper-asl" % JACKSON_VERSION % "compile" lazy val jackson_core = "org.codehaus.jackson" % "jackson-core-asl" % JACKSON_VERSION % "compile" - lazy val jackson_core_asl = "org.codehaus.jackson" % "jackson-core-asl" % JACKSON_VERSION % "compile" lazy val jersey = "com.sun.jersey" % "jersey-core" % JERSEY_VERSION % "compile" lazy val jersey_json = "com.sun.jersey" % "jersey-json" % JERSEY_VERSION % "compile" @@ -496,7 +495,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) { val jetty_util = Dependencies.jetty_util val jetty_xml = Dependencies.jetty_xml val jetty_servlet = Dependencies.jetty_servlet - val jackson_core_asl = Dependencies.jackson_core_asl + val jackson_core = Dependencies.jackson_core val jersey = Dependencies.jersey val jersey_contrib = Dependencies.jersey_contrib val jersey_json = Dependencies.jersey_json From c83804cf6f0bb65b3ae7b4119abeb4fe8d90ab67 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 28 Oct 2010 14:55:20 +0200 Subject: [PATCH 38/39] Optimization, 2 less allocs and 1 less field in actorref --- akka-actor/src/main/scala/actor/ActorRef.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/actor/ActorRef.scala b/akka-actor/src/main/scala/actor/ActorRef.scala index 41d0fcce1c..b65fca33c1 100644 --- a/akka-actor/src/main/scala/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/actor/ActorRef.scala @@ -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 From eef6350c45bff095f0eaf4346fbbc5714c9d66c3 Mon Sep 17 00:00:00 2001 From: momania Date: Thu, 28 Oct 2010 16:52:50 +0200 Subject: [PATCH 39/39] More sugar on the syntax --- akka-actor/src/main/scala/actor/FSM.scala | 19 +++++++++++++++--- .../test/scala/actor/actor/FSMActorSpec.scala | 13 +++++++++--- .../src/main/scala/DiningHakkersOnFsm.scala | 20 +++++++++---------- 3 files changed, 36 insertions(+), 16 deletions(-) diff --git a/akka-actor/src/main/scala/actor/FSM.scala b/akka-actor/src/main/scala/actor/FSM.scala index eac861d358..07ea1fc047 100644 --- a/akka-actor/src/main/scala/actor/FSM.scala +++ b/akka-actor/src/main/scala/actor/FSM.scala @@ -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/test/scala/actor/actor/FSMActorSpec.scala b/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala index dc6893c820..692c4cf807 100644 --- a/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala +++ b/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala @@ -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-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala index ecb4d82ba0..19ae5e02fb 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala @@ -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)) } /*