BREAKAGE: switching from se.scalablesolutions.akka to akka for all packages

This commit is contained in:
Viktor Klang 2010-10-26 12:49:25 +02:00
parent 29791593f0
commit 680ee7ddf1
393 changed files with 2100 additions and 2101 deletions

View file

@ -2,20 +2,20 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor package akka.actor
import se.scalablesolutions.akka.dispatch._ import akka.dispatch._
import se.scalablesolutions.akka.config.Config._ import akka.config.Config._
import se.scalablesolutions.akka.config.Supervision._ import akka.config.Supervision._
import se.scalablesolutions.akka.util.Helpers.{narrow, narrowSilently} import akka.util.Helpers.{narrow, narrowSilently}
import se.scalablesolutions.akka.AkkaException import akka.AkkaException
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import java.net.InetSocketAddress import java.net.InetSocketAddress
import scala.reflect.BeanProperty import scala.reflect.BeanProperty
import se.scalablesolutions.akka.util. {ReflectiveAccess, Logging, Duration} import akka.util. {ReflectiveAccess, Logging, Duration}
import se.scalablesolutions.akka.japi.Procedure import akka.japi.Procedure
/** /**
* Implements the Transactor abstraction. E.g. a transactional actor. * Implements the Transactor abstraction. E.g. a transactional actor.

View file

@ -2,16 +2,16 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor package akka.actor
import se.scalablesolutions.akka.dispatch._ import akka.dispatch._
import se.scalablesolutions.akka.config.Config._ import akka.config.Config._
import se.scalablesolutions.akka.config.Supervision._ import akka.config.Supervision._
import se.scalablesolutions.akka.stm.global._ import akka.stm.global._
import se.scalablesolutions.akka.stm.TransactionManagement._ import akka.stm.TransactionManagement._
import se.scalablesolutions.akka.stm.{ TransactionManagement, TransactionSetAbortedException } import akka.stm.{ TransactionManagement, TransactionSetAbortedException }
import se.scalablesolutions.akka.AkkaException import akka.AkkaException
import se.scalablesolutions.akka.util._ import akka.util._
import ReflectiveAccess._ import ReflectiveAccess._
import org.multiverse.api.ThreadLocalTransaction._ 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. * This means that all actors will share the same event-driven executor based dispatcher.
* <p/> * <p/>
* You can override it so it fits the specific use-case that the actor is used for. * You can override it so it fits the specific use-case that the actor is used for.
* See the <tt>se.scalablesolutions.akka.dispatch.Dispatchers</tt> class for the different * See the <tt>akka.dispatch.Dispatchers</tt> class for the different
* dispatchers available. * dispatchers available.
* <p/> * <p/>
* The default is also that all actors that are created and spawned from within this actor * The default is also that all actors that are created and spawned from within this actor

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor package akka.actor
import scala.collection.mutable.{ListBuffer, Map} import scala.collection.mutable.{ListBuffer, Map}
import scala.reflect.Manifest import scala.reflect.Manifest
@ -11,8 +11,8 @@ import java.util.concurrent.{ConcurrentSkipListSet, ConcurrentHashMap}
import java.util.{Set => JSet} import java.util.{Set => JSet}
import annotation.tailrec import annotation.tailrec
import se.scalablesolutions.akka.util.ReflectiveAccess._ import akka.util.ReflectiveAccess._
import se.scalablesolutions.akka.util.{ReadWriteGuard, Address, ListenerManagement} import akka.util.{ReadWriteGuard, Address, ListenerManagement}
import java.net.InetSocketAddress import java.net.InetSocketAddress
/** /**

View file

@ -2,11 +2,11 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor package akka.actor
import se.scalablesolutions.akka.stm.Ref import akka.stm.Ref
import se.scalablesolutions.akka.AkkaException import akka.AkkaException
import se.scalablesolutions.akka.japi.{ Function => JFunc, Procedure => JProc } import akka.japi.{ Function => JFunc, Procedure => JProc }
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.CountDownLatch import java.util.concurrent.CountDownLatch

View file

@ -2,15 +2,15 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor package akka.actor
import java.io.File import java.io.File
import java.net.{URL, URLClassLoader} import java.net.{URL, URLClassLoader}
import java.util.jar.JarFile import java.util.jar.JarFile
import java.util.Enumeration import java.util.Enumeration
import se.scalablesolutions.akka.util.{Bootable, Logging} import akka.util.{Bootable, Logging}
import se.scalablesolutions.akka.config.Config._ import akka.config.Config._
class AkkaDeployClassLoader(urls : List[URL], parent : ClassLoader) extends URLClassLoader(urls.toArray.asInstanceOf[Array[URL]],parent) class AkkaDeployClassLoader(urls : List[URL], parent : ClassLoader) extends URLClassLoader(urls.toArray.asInstanceOf[Array[URL]],parent)
{ {

View file

@ -2,10 +2,10 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor package akka.actor
import se.scalablesolutions.akka.stm.Ref import akka.stm.Ref
import se.scalablesolutions.akka.stm.local._ import akka.stm.local._
import java.util.concurrent.{ScheduledFuture, TimeUnit} import java.util.concurrent.{ScheduledFuture, TimeUnit}

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka package akka
import actor.{ScalaActorRef, ActorRef} import actor.{ScalaActorRef, ActorRef}

View file

@ -13,14 +13,14 @@
* Rework of David Pollak's ActorPing class in the Lift Project * Rework of David Pollak's ActorPing class in the Lift Project
* which is licensed under the Apache 2 License. * which is licensed under the Apache 2 License.
*/ */
package se.scalablesolutions.akka.actor package akka.actor
import scala.collection.JavaConversions import scala.collection.JavaConversions
import java.util.concurrent._ import java.util.concurrent._
import se.scalablesolutions.akka.util.Logging import akka.util.Logging
import se.scalablesolutions.akka.AkkaException import akka.AkkaException
object Scheduler extends Logging { object Scheduler extends Logging {
import Actor._ import Actor._

View file

@ -2,17 +2,17 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor package akka.actor
import se.scalablesolutions.akka.config.Supervision._ import akka.config.Supervision._
import se.scalablesolutions.akka.AkkaException import akka.AkkaException
import se.scalablesolutions.akka.util._ import akka.util._
import ReflectiveAccess._ import ReflectiveAccess._
import Actor._ import Actor._
import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap}
import java.net.InetSocketAddress import java.net.InetSocketAddress
import se.scalablesolutions.akka.config.Supervision._ import akka.config.Supervision._
class SupervisorException private[akka](message: String) extends AkkaException(message) class SupervisorException private[akka](message: String) extends AkkaException(message)

View file

@ -2,16 +2,16 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor package akka.actor
import se.scalablesolutions.akka.dispatch._ import akka.dispatch._
import se.scalablesolutions.akka.stm.global._ import akka.stm.global._
import se.scalablesolutions.akka.config.Supervision._ import akka.config.Supervision._
import java.net.InetSocketAddress import java.net.InetSocketAddress
import scala.reflect.BeanProperty 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. * Subclass this abstract class to create a MDB-style untyped actor.

View file

@ -2,12 +2,12 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.config package akka.config
import se.scalablesolutions.akka.AkkaException import akka.AkkaException
import se.scalablesolutions.akka.util.Logging import akka.util.Logging
import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException} import akka.actor.{ActorRef, IllegalActorStateException}
import se.scalablesolutions.akka.dispatch.CompletableFuture import akka.dispatch.CompletableFuture
import net.lag.configgy.{Config => CConfig, Configgy, ParseException} import net.lag.configgy.{Config => CConfig, Configgy, ParseException}

View file

@ -2,10 +2,10 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
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 com.google.inject.{AbstractModule}
import java.util.{List => JList, ArrayList} import java.util.{List => JList, ArrayList}
import scala.reflect.BeanProperty import scala.reflect.BeanProperty
@ -18,43 +18,43 @@ import scala.reflect.BeanProperty
sealed abstract class Configuration sealed abstract class Configuration
class RestartStrategy(@BeanProperty val scheme: FailOverScheme, @BeanProperty val maxNrOfRetries: Int, @BeanProperty val withinTimeRange: Int) extends 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 { 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 { abstract class Scope extends Configuration {
def transform: se.scalablesolutions.akka.kernel.Scope def transform: akka.kernel.Scope
} }
class Permanent extends Scope { class Permanent extends Scope {
override def transform = se.scalablesolutions.akka.kernel.Permanent override def transform = akka.kernel.Permanent
} }
class Transient extends Scope { class Transient extends Scope {
override def transform = se.scalablesolutions.akka.kernel.Transient override def transform = akka.kernel.Transient
} }
class Temporary extends Scope { class Temporary extends Scope {
override def transform = se.scalablesolutions.akka.kernel.Temporary override def transform = akka.kernel.Temporary
} }
abstract class FailOverScheme extends Configuration { abstract class FailOverScheme extends Configuration {
def transform: se.scalablesolutions.akka.kernel.FailOverScheme def transform: akka.kernel.FailOverScheme
} }
class AllForOne extends FailOverScheme { class AllForOne extends FailOverScheme {
override def transform = se.scalablesolutions.akka.kernel.AllForOne override def transform = akka.kernel.AllForOne
} }
class OneForOne extends FailOverScheme { class OneForOne extends FailOverScheme {
override def transform = se.scalablesolutions.akka.kernel.OneForOne override def transform = akka.kernel.OneForOne
} }
abstract class Server extends Configuration abstract class Server extends Configuration
//class kernelConfig(@BeanProperty val restartStrategy: RestartStrategy, @BeanProperty val servers: JList[Server]) extends Server { //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[_], class Component(@BeanProperty val intf: Class[_],
@BeanProperty val target: Class[_], @BeanProperty val target: Class[_],
@BeanProperty val lifeCycle: LifeCycle, @BeanProperty val lifeCycle: LifeCycle,
@BeanProperty val timeout: Int) extends Server { @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)
} }
*/ */

View file

@ -2,9 +2,9 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
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 { private[akka] trait TypedActorConfiguratorBase {
def getExternalDependency[T](clazz: Class[T]): T def getExternalDependency[T](clazz: Class[T]): T

View file

@ -2,10 +2,10 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.config package akka.config
import se.scalablesolutions.akka.actor.{ActorRef} import akka.actor.{ActorRef}
import se.scalablesolutions.akka.dispatch.MessageDispatcher import akka.dispatch.MessageDispatcher
case class RemoteAddress(val hostname: String, val port: Int) case class RemoteAddress(val hostname: String, val port: Int)

View file

@ -2,16 +2,16 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.dataflow package akka.dataflow
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.{ConcurrentLinkedQueue, LinkedBlockingQueue} import java.util.concurrent.{ConcurrentLinkedQueue, LinkedBlockingQueue}
import se.scalablesolutions.akka.actor.{Actor, ActorRef} import akka.actor.{Actor, ActorRef}
import se.scalablesolutions.akka.actor.Actor._ import akka.actor.Actor._
import se.scalablesolutions.akka.dispatch.CompletableFuture import akka.dispatch.CompletableFuture
import se.scalablesolutions.akka.AkkaException import akka.AkkaException
import se.scalablesolutions.akka.japi.{ Function, SideEffect } import akka.japi.{ Function, SideEffect }
/** /**
* Implements Oz-style dataflow (single assignment) variables. * Implements Oz-style dataflow (single assignment) variables.

View file

@ -2,12 +2,12 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.dispatch package akka.dispatch
import se.scalablesolutions.akka.actor.{Actor, ActorRef} import akka.actor.{Actor, ActorRef}
import se.scalablesolutions.akka.config.Config._ import akka.config.Config._
import se.scalablesolutions.akka.util.{Duration, Logging} import akka.util.{Duration, Logging}
import se.scalablesolutions.akka.actor.newUuid import akka.actor.newUuid
import net.lag.configgy.ConfigMap import net.lag.configgy.ConfigMap
@ -67,7 +67,7 @@ object Dispatchers extends Logging {
* <p/> * <p/>
* Can be beneficial to use the <code>HawtDispatcher.pin(self)</code> to "pin" an actor to a specific thread. * Can be beneficial to use the <code>HawtDispatcher.pin(self)</code> to "pin" an actor to a specific thread.
* <p/> * <p/>
* 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) def newHawtDispatcher(aggregate: Boolean) = new HawtDispatcher(aggregate)

View file

@ -2,13 +2,13 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.dispatch package akka.dispatch
import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException} import akka.actor.{ActorRef, IllegalActorStateException}
import se.scalablesolutions.akka.util.ReflectiveAccess.EnterpriseModule import akka.util.ReflectiveAccess.EnterpriseModule
import java.util.Queue import java.util.Queue
import se.scalablesolutions.akka.util.Switch import akka.util.Switch
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent. {ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue, LinkedBlockingQueue} import java.util.concurrent. {ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue, LinkedBlockingQueue}
@ -57,7 +57,7 @@ import java.util.concurrent. {ExecutorService, RejectedExecutionException, Concu
* <p/> * <p/>
* *
* But the preferred way of creating dispatchers is to use * 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 <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
* @param throughput positive integer indicates the dispatcher will only process so much messages at a time from the * @param throughput positive integer indicates the dispatcher will only process so much messages at a time from the

View file

@ -2,12 +2,12 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.dispatch package akka.dispatch
import jsr166x.{Deque, ConcurrentLinkedDeque, LinkedBlockingDeque} import jsr166x.{Deque, ConcurrentLinkedDeque, LinkedBlockingDeque}
import se.scalablesolutions.akka.actor.{Actor, ActorRef, IllegalActorStateException} import akka.actor.{Actor, ActorRef, IllegalActorStateException}
import se.scalablesolutions.akka.util.Switch import akka.util.Switch
import java.util.concurrent. {ExecutorService, CopyOnWriteArrayList} import java.util.concurrent. {ExecutorService, CopyOnWriteArrayList}
import java.util.concurrent.atomic.AtomicReference 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 ?! * TODO: it would be nice to be able to redistribute work even when no new messages are being dispatched, without impacting dispatching performance ?!
* <p/> * <p/>
* The preferred way of creating dispatchers is to use * 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 akka.dispatch.ExecutorBasedEventDrivenWorkStealingDispatcher
* @see se.scalablesolutions.akka.dispatch.Dispatchers * @see akka.dispatch.Dispatchers
* *
* @author Jan Van Besien * @author Jan Van Besien
*/ */

View file

@ -2,13 +2,13 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.dispatch package akka.dispatch
import se.scalablesolutions.akka.AkkaException import akka.AkkaException
import se.scalablesolutions.akka.actor.Actor.spawn import akka.actor.Actor.spawn
import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.locks.ReentrantLock
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import se.scalablesolutions.akka.routing.Dispatcher import akka.routing.Dispatcher
class FutureTimeoutException(message: String) extends AkkaException(message) class FutureTimeoutException(message: String) extends AkkaException(message)

View file

@ -2,9 +2,9 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
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.DispatchQueue
import org.fusesource.hawtdispatch.ScalaDispatch._ 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.atomic.{AtomicInteger, AtomicBoolean}
import java.util.concurrent.CountDownLatch 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. * Holds helper methods for working with actors that are using a HawtDispatcher as it's dispatcher.

View file

@ -2,16 +2,16 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.dispatch package akka.dispatch
import se.scalablesolutions.akka.actor.{Actor, ActorType, ActorRef, ActorInitializationException} import akka.actor.{Actor, ActorType, ActorRef, ActorInitializationException}
import se.scalablesolutions.akka.util.ReflectiveAccess.EnterpriseModule import akka.util.ReflectiveAccess.EnterpriseModule
import se.scalablesolutions.akka.AkkaException import akka.AkkaException
import java.util.{Queue, List} import java.util.{Queue, List}
import java.util.concurrent._ import java.util.concurrent._
import concurrent.forkjoin.LinkedTransferQueue import concurrent.forkjoin.LinkedTransferQueue
import se.scalablesolutions.akka.util._ import akka.util._
class MessageQueueAppendFailedException(message: String) extends AkkaException(message) class MessageQueueAppendFailedException(message: String) extends AkkaException(message)

View file

@ -2,14 +2,14 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.dispatch package akka.dispatch
import org.multiverse.commitbarriers.CountDownCommitBarrier import org.multiverse.commitbarriers.CountDownCommitBarrier
import java.util.concurrent._ import java.util.concurrent._
import atomic. {AtomicInteger, AtomicBoolean, AtomicReference, AtomicLong} import atomic. {AtomicInteger, AtomicBoolean, AtomicReference, AtomicLong}
import se.scalablesolutions.akka.util. {Switch, ReentrantGuard, Logging, HashCode} import akka.util. {Switch, ReentrantGuard, Logging, HashCode}
import se.scalablesolutions.akka.actor._ import akka.actor._
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>

View file

@ -2,15 +2,15 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.dispatch package akka.dispatch
import se.scalablesolutions.akka.actor.{Actor, ActorRef} import akka.actor.{Actor, ActorRef}
import se.scalablesolutions.akka.config.Config.config import akka.config.Config.config
import se.scalablesolutions.akka.util.Duration import akka.util.Duration
import java.util.Queue import java.util.Queue
import java.util.concurrent.{ConcurrentLinkedQueue, BlockingQueue, TimeUnit, LinkedBlockingQueue} import java.util.concurrent.{ConcurrentLinkedQueue, BlockingQueue, TimeUnit, LinkedBlockingQueue}
import se.scalablesolutions.akka.actor import akka.actor
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
/** /**

View file

@ -2,14 +2,14 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.dispatch package akka.dispatch
import java.util.Collection import java.util.Collection
import java.util.concurrent._ import java.util.concurrent._
import atomic.{AtomicLong, AtomicInteger} import atomic.{AtomicLong, AtomicInteger}
import ThreadPoolExecutor.CallerRunsPolicy import ThreadPoolExecutor.CallerRunsPolicy
import se.scalablesolutions.akka.util. {Duration, Logging} import akka.util. {Duration, Logging}
object ThreadPoolConfig { object ThreadPoolConfig {
type Bounds = Int type Bounds = Int

View file

@ -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). * A Function interface. Used to create first-class-functions is Java (sort of).

View file

@ -2,9 +2,9 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.routing package akka.routing
import se.scalablesolutions.akka.actor.ActorRef import akka.actor.ActorRef
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
/** /**

View file

@ -2,9 +2,9 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
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 java.util.concurrent.ConcurrentSkipListSet
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._

View file

@ -2,9 +2,9 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
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. * A Dispatcher is a trait whose purpose is to route incoming messages to actors.

View file

@ -2,10 +2,10 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.routing package akka.routing
import se.scalablesolutions.akka.actor.{Actor, ActorRef} import akka.actor.{Actor, ActorRef}
import se.scalablesolutions.akka.actor.Actor._ import akka.actor.Actor._
object Routing { object Routing {

View file

@ -2,9 +2,9 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
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 import org.multiverse.transactional.refs.BasicRef

View file

@ -2,18 +2,18 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.stm package akka.stm
import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicLong
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import scala.collection.mutable.HashMap import scala.collection.mutable.HashMap
import se.scalablesolutions.akka.util.ReflectiveAccess.JtaModule import akka.util.ReflectiveAccess.JtaModule
import se.scalablesolutions.akka.util.Logging import akka.util.Logging
import se.scalablesolutions.akka.config.Config._ import akka.config.Config._
import se.scalablesolutions.akka.AkkaException import akka.AkkaException
import org.multiverse.api.{Transaction => MultiverseTransaction} import org.multiverse.api.{Transaction => MultiverseTransaction}
import org.multiverse.api.lifecycle.{TransactionLifecycleListener, TransactionLifecycleEvent} import org.multiverse.api.lifecycle.{TransactionLifecycleListener, TransactionLifecycleEvent}
@ -28,10 +28,10 @@ class StmConfigurationException(message: String) extends AkkaException(message)
object Transaction { object Transaction {
val idFactory = new AtomicLong(-1L) 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 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 Global extends GlobalStm
object Util extends StmUtil object Util extends StmUtil

View file

@ -2,12 +2,12 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.stm package akka.stm
import java.lang.{Boolean => JBoolean} import java.lang.{Boolean => JBoolean}
import se.scalablesolutions.akka.config.Config._ import akka.config.Config._
import se.scalablesolutions.akka.util.Duration import akka.util.Duration
import org.multiverse.api.GlobalStmInstance.getGlobalStmInstance import org.multiverse.api.GlobalStmInstance.getGlobalStmInstance
import org.multiverse.stms.alpha.AlphaStm import org.multiverse.stms.alpha.AlphaStm

View file

@ -2,11 +2,11 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.stm package akka.stm
import java.lang.{Boolean => JBoolean} import java.lang.{Boolean => JBoolean}
import se.scalablesolutions.akka.util.Duration import akka.util.Duration
import org.multiverse.api.TraceLevel import org.multiverse.api.TraceLevel
import org.multiverse.api.{PropagationLevel => Propagation} import org.multiverse.api.{PropagationLevel => Propagation}

View file

@ -2,9 +2,9 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
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.{StmUtils => MultiverseStmUtils}
import org.multiverse.api.ThreadLocalTransaction._ import org.multiverse.api.ThreadLocalTransaction._
@ -18,7 +18,7 @@ class TransactionSetAbortedException(msg: String) extends AkkaException(msg)
* Internal helper methods and properties for transaction management. * Internal helper methods and properties for transaction management.
*/ */
object TransactionManagement extends TransactionManagement { object TransactionManagement extends TransactionManagement {
import se.scalablesolutions.akka.config.Config._ import akka.config.Config._
// FIXME move to stm.global.fair? // FIXME move to stm.global.fair?
val FAIR_TRANSACTIONS = config.getBool("akka.stm.fair", true) val FAIR_TRANSACTIONS = config.getBool("akka.stm.fair", true)
@ -126,16 +126,16 @@ trait StmUtil {
} }
trait StmCommon { trait StmCommon {
type TransactionConfig = se.scalablesolutions.akka.stm.TransactionConfig type TransactionConfig = akka.stm.TransactionConfig
val TransactionConfig = se.scalablesolutions.akka.stm.TransactionConfig val TransactionConfig = akka.stm.TransactionConfig
type TransactionFactory = se.scalablesolutions.akka.stm.TransactionFactory type TransactionFactory = akka.stm.TransactionFactory
val TransactionFactory = se.scalablesolutions.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] type Ref[T] = akka.stm.Ref[T]
val Ref = se.scalablesolutions.akka.stm.Ref val Ref = akka.stm.Ref
} }

View file

@ -2,11 +2,11 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.stm package akka.stm
import scala.collection.immutable.HashMap import scala.collection.immutable.HashMap
import se.scalablesolutions.akka.actor.{newUuid} import akka.actor.{newUuid}
import org.multiverse.api.ThreadLocalTransaction.getThreadLocalTransaction import org.multiverse.api.ThreadLocalTransaction.getThreadLocalTransaction

View file

@ -2,11 +2,11 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.stm package akka.stm
import scala.collection.immutable.Vector import scala.collection.immutable.Vector
import se.scalablesolutions.akka.actor.newUuid import akka.actor.newUuid
import org.multiverse.api.ThreadLocalTransaction.getThreadLocalTransaction import org.multiverse.api.ThreadLocalTransaction.getThreadLocalTransaction

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.stm.global package akka.stm.global
/** /**
* Java-friendly atomic blocks. * Java-friendly atomic blocks.
@ -10,8 +10,8 @@ package se.scalablesolutions.akka.stm.global
* Example usage (in Java): * Example usage (in Java):
* <p/> * <p/>
* <pre> * <pre>
* import se.scalablesolutions.akka.stm.*; * import akka.stm.*;
* import se.scalablesolutions.akka.stm.global.Atomic; * import akka.stm.global.Atomic;
* *
* final Ref<Integer> ref = new Ref<Integer>(0); * final Ref<Integer> ref = new Ref<Integer>(0);
* *

View file

@ -2,9 +2,9 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
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.api.{Transaction => MultiverseTransaction}
import org.multiverse.templates.TransactionalCallable import org.multiverse.templates.TransactionalCallable
@ -18,7 +18,7 @@ object GlobalStm extends Logging
* Example of atomic transaction management using the atomic block: * Example of atomic transaction management using the atomic block:
* <p/> * <p/>
* <pre> * <pre>
* import se.scalablesolutions.akka.stm.global._ * import akka.stm.global._
* *
* atomic { * atomic {
* // do something within a transaction * // do something within a transaction

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.stm package akka.stm
/** /**
* For easily importing global STM. * For easily importing global STM.

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.stm.local package akka.stm.local
/** /**
* Java-friendly atomic blocks. * Java-friendly atomic blocks.
@ -10,8 +10,8 @@ package se.scalablesolutions.akka.stm.local
* Example usage (in Java): * Example usage (in Java):
* <p/> * <p/>
* <pre> * <pre>
* import se.scalablesolutions.akka.stm.*; * import akka.stm.*;
* import se.scalablesolutions.akka.stm.local.Atomic; * import akka.stm.local.Atomic;
* *
* final Ref<Integer> ref = new Ref<Integer>(0); * final Ref<Integer> ref = new Ref<Integer>(0);
* *

View file

@ -2,9 +2,9 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
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.api.{Transaction => MultiverseTransaction}
import org.multiverse.templates.TransactionalCallable import org.multiverse.templates.TransactionalCallable
@ -19,7 +19,7 @@ object LocalStm extends Logging
* Example of atomic transaction management using the atomic block. * Example of atomic transaction management using the atomic block.
* <p/> * <p/>
* <pre> * <pre>
* import se.scalablesolutions.akka.stm.local._ * import akka.stm.local._
* *
* atomic { * atomic {
* // do something within a transaction * // do something within a transaction

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.stm package akka.stm
/** /**
* For easily importing local STM. * For easily importing local STM.

View file

@ -2,18 +2,18 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.stm package akka.stm
/** /**
* For importing the transactional datastructures, including the primitive refs * For importing the transactional datastructures, including the primitive refs
* and transactional data structures from Multiverse. * and transactional data structures from Multiverse.
*/ */
package object transactional { package object transactional {
type TransactionalMap[K,V] = se.scalablesolutions.akka.stm.TransactionalMap[K,V] type TransactionalMap[K,V] = akka.stm.TransactionalMap[K,V]
val TransactionalMap = se.scalablesolutions.akka.stm.TransactionalMap val TransactionalMap = akka.stm.TransactionalMap
type TransactionalVector[T] = se.scalablesolutions.akka.stm.TransactionalVector[T] type TransactionalVector[T] = akka.stm.TransactionalVector[T]
val TransactionalVector = se.scalablesolutions.akka.stm.TransactionalVector val TransactionalVector = akka.stm.TransactionalVector
type BooleanRef = org.multiverse.transactional.refs.BooleanRef type BooleanRef = org.multiverse.transactional.refs.BooleanRef
type ByteRef = org.multiverse.transactional.refs.ByteRef type ByteRef = org.multiverse.transactional.refs.ByteRef

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.util package akka.util
object Address { object Address {
def apply(hostname: String, port: Int) = new Address(hostname, port) def apply(hostname: String, port: Int) = new Address(hostname, port)

View file

@ -2,10 +2,10 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka package akka
import se.scalablesolutions.akka.util.Logging import akka.util.Logging
import se.scalablesolutions.akka.actor.newUuid import akka.actor.newUuid
import java.io.{StringWriter, PrintWriter} import java.io.{StringWriter, PrintWriter}
import java.net.{InetAddress, UnknownHostException} import java.net.{InetAddress, UnknownHostException}

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.util package akka.util
trait Bootable { trait Bootable {
def onLoad {} def onLoad {}

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.util package akka.util
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
@ -24,7 +24,7 @@ object Duration {
* <p/> * <p/>
* Examples of usage from Java: * Examples of usage from Java:
* <pre> * <pre>
* import se.scalablesolutions.akka.util.Duration; * import akka.util.Duration;
* import java.util.concurrent.TimeUnit; * import java.util.concurrent.TimeUnit;
* *
* Duration duration = new Duration(100, TimeUnit.MILLISECONDS); * Duration duration = new Duration(100, TimeUnit.MILLISECONDS);
@ -36,7 +36,7 @@ object Duration {
* <p/> * <p/>
* Examples of usage from Scala: * Examples of usage from Scala:
* <pre> * <pre>
* import se.scalablesolutions.akka.util.Duration * import akka.util.Duration
* import java.util.concurrent.TimeUnit * import java.util.concurrent.TimeUnit
* *
* val duration = Duration(100, TimeUnit.MILLISECONDS) * val duration = Duration(100, TimeUnit.MILLISECONDS)
@ -48,7 +48,7 @@ object Duration {
* <p/> * <p/>
* Implicits are also provided for Int and Long. Example usage: * Implicits are also provided for Int and Long. Example usage:
* <pre> * <pre>
* import se.scalablesolutions.akka.util.duration._ * import akka.util.duration._
* *
* val duration = 100.millis * val duration = 100.millis
* </pre> * </pre>

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.util package akka.util
import java.lang.reflect.{Array => JArray} import java.lang.reflect.{Array => JArray}
import java.lang.{Float => JFloat, Double => JDouble} import java.lang.{Float => JFloat, Double => JDouble}

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.util package akka.util
import java.security.MessageDigest import java.security.MessageDigest

View file

@ -2,11 +2,11 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.util package akka.util
import java.util.concurrent.ConcurrentSkipListSet 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. * A manager for listener actors. Intended for mixin by observables.

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.util package akka.util
import java.util.concurrent.locks.{ReentrantReadWriteLock, ReentrantLock} import java.util.concurrent.locks.{ReentrantReadWriteLock, ReentrantLock}
import java.util.concurrent.atomic. {AtomicBoolean} import java.util.concurrent.atomic. {AtomicBoolean}

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.util package akka.util
import org.slf4j.{Logger => SLFLogger,LoggerFactory => SLFLoggerFactory} import org.slf4j.{Logger => SLFLogger,LoggerFactory => SLFLoggerFactory}

View file

@ -2,13 +2,13 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.util package akka.util
import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException, ActorType, Uuid} import akka.actor.{ActorRef, IllegalActorStateException, ActorType, Uuid}
import se.scalablesolutions.akka.dispatch.{Future, CompletableFuture, MessageInvocation} import akka.dispatch.{Future, CompletableFuture, MessageInvocation}
import se.scalablesolutions.akka.config.{Config, ModuleNotAvailableException} import akka.config.{Config, ModuleNotAvailableException}
import se.scalablesolutions.akka.stm.Transaction import akka.stm.Transaction
import se.scalablesolutions.akka.AkkaException import akka.AkkaException
import java.net.InetSocketAddress 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") "Can't load the remoting module, make sure that akka-remote.jar is on the classpath")
val remoteClientObjectInstance: Option[RemoteClientObject] = val remoteClientObjectInstance: Option[RemoteClientObject] =
getObjectFor("se.scalablesolutions.akka.remote.RemoteClient$") getObjectFor("akka.remote.RemoteClient$")
def register(address: InetSocketAddress, uuid: Uuid) = { def register(address: InetSocketAddress, uuid: Uuid) = {
ensureRemotingEnabled ensureRemotingEnabled
@ -123,10 +123,10 @@ object ReflectiveAccess extends Logging {
} }
val remoteServerObjectInstance: Option[RemoteServerObject] = val remoteServerObjectInstance: Option[RemoteServerObject] =
getObjectFor("se.scalablesolutions.akka.remote.RemoteServer$") getObjectFor("akka.remote.RemoteServer$")
val remoteNodeObjectInstance: Option[RemoteNodeObject] = val remoteNodeObjectInstance: Option[RemoteNodeObject] =
getObjectFor("se.scalablesolutions.akka.remote.RemoteNode$") getObjectFor("akka.remote.RemoteNode$")
def registerActor(address: InetSocketAddress, uuid: Uuid, actorRef: ActorRef) = { def registerActor(address: InetSocketAddress, uuid: Uuid, actorRef: ActorRef) = {
ensureRemotingEnabled 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") "Can't load the typed actor module, make sure that akka-typed-actor.jar is on the classpath")
val typedActorObjectInstance: Option[TypedActorObject] = val typedActorObjectInstance: Option[TypedActorObject] =
getObjectFor("se.scalablesolutions.akka.actor.TypedActor$") getObjectFor("akka.actor.TypedActor$")
def resolveFutureIfMessageIsJoinPoint(message: Any, future: Future[_]): Boolean = { def resolveFutureIfMessageIsJoinPoint(message: Any, future: Future[_]): Boolean = {
ensureTypedActorEnabled 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") "Can't load the typed actor module, make sure that akka-jta.jar is on the classpath")
val transactionContainerObjectInstance: Option[TransactionContainerObject] = val transactionContainerObjectInstance: Option[TransactionContainerObject] =
getObjectFor("se.scalablesolutions.akka.jta.TransactionContainer$") getObjectFor("akka.jta.TransactionContainer$")
def createTransactionContainer: TransactionContainer = { def createTransactionContainer: TransactionContainer = {
ensureJtaEnabled ensureJtaEnabled
@ -217,21 +217,21 @@ object ReflectiveAccess extends Logging {
lazy val isEnterpriseEnabled = clusterObjectInstance.isDefined lazy val isEnterpriseEnabled = clusterObjectInstance.isDefined
val clusterObjectInstance: Option[AnyRef] = val clusterObjectInstance: Option[AnyRef] =
getObjectFor("se.scalablesolutions.akka.cluster.Cluster$") getObjectFor("akka.cluster.Cluster$")
val serializerClass: Option[Class[_]] = val serializerClass: Option[Class[_]] =
getClassFor("se.scalablesolutions.akka.serialization.Serializer") getClassFor("akka.serialization.Serializer")
def ensureEnterpriseEnabled = if (!isEnterpriseEnabled) throw new ModuleNotAvailableException( def ensureEnterpriseEnabled = if (!isEnterpriseEnabled) throw new ModuleNotAvailableException(
"Feature is only available in Akka Enterprise edition") "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 = { private def createMailbox(mailboxClassname: String, actorRef: ActorRef): Mailbox = {
ensureEnterpriseEnabled ensureEnterpriseEnabled

View file

@ -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.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import static se.scalablesolutions.akka.config.Supervision.*; import static akka.config.Supervision.*;
public class SupervisionConfig { public class SupervisionConfig {
/*Just some sample code to demonstrate the declarative supervision configuration for Java */ /*Just some sample code to demonstrate the declarative supervision configuration for Java */

View file

@ -1,4 +1,4 @@
package se.scalablesolutions.akka.japi; package akka.japi;
import org.junit.Test; import org.junit.Test;

View file

@ -1,4 +1,4 @@
package se.scalablesolutions.akka.stm; package akka.stm;
public class Address { public class Address {
private String location; private String location;

View file

@ -1,7 +1,7 @@
package se.scalablesolutions.akka.stm; package akka.stm;
import se.scalablesolutions.akka.stm.Ref; import akka.stm.Ref;
import se.scalablesolutions.akka.stm.local.Atomic; import akka.stm.local.Atomic;
public class CounterExample { public class CounterExample {
final static Ref<Integer> ref = new Ref<Integer>(0); final static Ref<Integer> ref = new Ref<Integer>(0);

View file

@ -1,11 +1,11 @@
package se.scalablesolutions.akka.stm; package akka.stm;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import org.junit.Test; import org.junit.Test;
import org.junit.Before; import org.junit.Before;
import se.scalablesolutions.akka.stm.*; import akka.stm.*;
import se.scalablesolutions.akka.stm.local.Atomic; import akka.stm.local.Atomic;
import org.multiverse.api.ThreadLocalTransaction; import org.multiverse.api.ThreadLocalTransaction;
import org.multiverse.api.TransactionConfiguration; import org.multiverse.api.TransactionConfiguration;

View file

@ -1,7 +1,7 @@
package se.scalablesolutions.akka.stm; package akka.stm;
import se.scalablesolutions.akka.stm.Ref; import akka.stm.Ref;
import se.scalablesolutions.akka.stm.local.Atomic; import akka.stm.local.Atomic;
public class RefExample { public class RefExample {
public static void main(String[] args) { public static void main(String[] args) {

View file

@ -1,7 +1,7 @@
package se.scalablesolutions.akka.stm; package akka.stm;
import se.scalablesolutions.akka.stm.Ref; import akka.stm.Ref;
import se.scalablesolutions.akka.stm.local.Atomic; import akka.stm.local.Atomic;
public class StmExamples { public class StmExamples {
public static void main(String[] args) { public static void main(String[] args) {

View file

@ -1,7 +1,7 @@
package se.scalablesolutions.akka.stm; package akka.stm;
import se.scalablesolutions.akka.stm.*; import akka.stm.*;
import se.scalablesolutions.akka.stm.local.Atomic; import akka.stm.local.Atomic;
import org.multiverse.api.ThreadLocalTransaction; import org.multiverse.api.ThreadLocalTransaction;
import org.multiverse.api.TransactionConfiguration; import org.multiverse.api.TransactionConfiguration;

View file

@ -1,7 +1,7 @@
package se.scalablesolutions.akka.stm; package akka.stm;
import se.scalablesolutions.akka.stm.*; import akka.stm.*;
import se.scalablesolutions.akka.stm.local.Atomic; import akka.stm.local.Atomic;
public class TransactionalMapExample { public class TransactionalMapExample {
public static void main(String[] args) { public static void main(String[] args) {

View file

@ -1,7 +1,7 @@
package se.scalablesolutions.akka.stm; package akka.stm;
import se.scalablesolutions.akka.stm.*; import akka.stm.*;
import se.scalablesolutions.akka.stm.local.Atomic; import akka.stm.local.Atomic;
public class TransactionalVectorExample { public class TransactionalVectorExample {
public static void main(String[] args) { public static void main(String[] args) {

View file

@ -1,4 +1,4 @@
package se.scalablesolutions.akka.stm; package akka.stm;
public class User { public class User {
private String name; private String name;

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka package akka
abstract class TestMessage abstract class TestMessage

View file

@ -1,11 +1,11 @@
package se.scalablesolutions.akka.actor package akka.actor
import java.util.concurrent.{TimeUnit, CyclicBarrier, TimeoutException} import java.util.concurrent.{TimeUnit, CyclicBarrier, TimeoutException}
import se.scalablesolutions.akka.config.Supervision._ import akka.config.Supervision._
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test
import se.scalablesolutions.akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
import Actor._ import Actor._
object ActorFireForgetRequestReplySpec { object ActorFireForgetRequestReplySpec {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor package akka.actor
import org.scalatest.Spec import org.scalatest.Spec
import org.scalatest.matchers.ShouldMatchers import org.scalatest.matchers.ShouldMatchers
@ -10,7 +10,7 @@ import org.scalatest.BeforeAndAfterAll
import org.scalatest.junit.JUnitRunner import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith import org.junit.runner.RunWith
import se.scalablesolutions.akka.actor._ import akka.actor._
import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.{CountDownLatch, TimeUnit}
object ActorRefSpec { object ActorRefSpec {

View file

@ -1,4 +1,4 @@
package se.scalablesolutions.akka.actor package akka.actor
import org.scalatest.Suite import org.scalatest.Suite
import org.scalatest.junit.JUnitRunner import org.scalatest.junit.JUnitRunner

View file

@ -3,9 +3,9 @@
contributed by Julien Gaugaz contributed by Julien Gaugaz
inspired by the version contributed by Yura Taras and modified by Isaac Gouy 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 { object Chameneos {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor package akka.actor
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test

View file

@ -1,4 +1,4 @@
package se.scalablesolutions.akka.actor package akka.actor
import java.util.concurrent.{TimeUnit, CountDownLatch} import java.util.concurrent.{TimeUnit, CountDownLatch}
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite

View file

@ -1,4 +1,4 @@
package se.scalablesolutions.akka.actor package akka.actor
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers

View file

@ -1,4 +1,4 @@
package se.scalablesolutions.akka.actor package akka.actor
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test

View file

@ -1,10 +1,10 @@
package se.scalablesolutions.akka.actor package akka.actor
import java.util.concurrent.{TimeUnit, CountDownLatch} import java.util.concurrent.{TimeUnit, CountDownLatch}
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test
import se.scalablesolutions.akka.stm.{Ref, TransactionalMap, TransactionalVector} import akka.stm.{Ref, TransactionalMap, TransactionalVector}
import Actor._ import Actor._
object TransactorSpec { object TransactorSpec {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor package akka.actor
import java.lang.Thread.sleep import java.lang.Thread.sleep
@ -11,7 +11,7 @@ import org.junit.Test
import Actor._ import Actor._
import java.util.concurrent.{TimeUnit, CountDownLatch} 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 import org.multiverse.api.latches.StandardLatch
class RestartStrategySpec extends JUnitSuite { class RestartStrategySpec extends JUnitSuite {

View file

@ -2,13 +2,13 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor package akka.actor
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test
import Actor._ import Actor._
import se.scalablesolutions.akka.config.Supervision.OneForOneStrategy import akka.config.Supervision.OneForOneStrategy
import java.util.concurrent.{TimeUnit, CountDownLatch} import java.util.concurrent.{TimeUnit, CountDownLatch}

View file

@ -1,12 +1,12 @@
/** /**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor package akka.actor
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import se.scalablesolutions.akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
import se.scalablesolutions.akka.config.Supervision.{SupervisorConfig, OneForOneStrategy, Supervise, Permanent} import akka.config.Supervision.{SupervisorConfig, OneForOneStrategy, Supervise, Permanent}
import java.util.concurrent.CountDownLatch import java.util.concurrent.CountDownLatch
class SupervisorMiscSpec extends WordSpec with MustMatchers { class SupervisorMiscSpec extends WordSpec with MustMatchers {

View file

@ -2,10 +2,10 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor package akka.actor
import se.scalablesolutions.akka.config.Supervision._ import akka.config.Supervision._
import se.scalablesolutions.akka.{OneWay, Die, Ping} import akka.{OneWay, Die, Ping}
import Actor._ import Actor._
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.dataflow package akka.dataflow
import org.scalatest.Spec import org.scalatest.Spec
import org.scalatest.Assertions import org.scalatest.Assertions
@ -11,11 +11,11 @@ import org.scalatest.BeforeAndAfterAll
import org.scalatest.junit.JUnitRunner import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith import org.junit.runner.RunWith
import se.scalablesolutions.akka.dispatch.DefaultCompletableFuture import akka.dispatch.DefaultCompletableFuture
import java.util.concurrent.{TimeUnit, CountDownLatch} import java.util.concurrent.{TimeUnit, CountDownLatch}
import annotation.tailrec import annotation.tailrec
import java.util.concurrent.atomic.{AtomicLong, AtomicReference, AtomicInteger} import java.util.concurrent.atomic.{AtomicLong, AtomicReference, AtomicInteger}
import se.scalablesolutions.akka.actor.ActorRegistry import akka.actor.ActorRegistry
@RunWith(classOf[JUnitRunner]) @RunWith(classOf[JUnitRunner])
class DataFlowTest extends Spec with ShouldMatchers with BeforeAndAfterAll { class DataFlowTest extends Spec with ShouldMatchers with BeforeAndAfterAll {

View file

@ -1,17 +1,17 @@
/** /**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor.dispatch package akka.actor.dispatch
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test
import org.scalatest.Assertions._ import org.scalatest.Assertions._
import se.scalablesolutions.akka.dispatch._ import akka.dispatch._
import se.scalablesolutions.akka.actor.{ActorRef, Actor} import akka.actor.{ActorRef, Actor}
import se.scalablesolutions.akka.actor.Actor._ import akka.actor.Actor._
import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicLong
import java.util.concurrent. {ConcurrentHashMap, CountDownLatch, TimeUnit} import java.util.concurrent. {ConcurrentHashMap, CountDownLatch, TimeUnit}
import se.scalablesolutions.akka.actor.dispatch.ActorModelSpec.MessageDispatcherInterceptor import akka.actor.dispatch.ActorModelSpec.MessageDispatcherInterceptor
object ActorModelSpec { object ActorModelSpec {

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor.dispatch package akka.actor.dispatch
import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.{CountDownLatch, TimeUnit}
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
@ -9,7 +9,7 @@ import org.junit.Test
import net.lag.configgy.Config import net.lag.configgy.Config
import scala.reflect.{Manifest} import scala.reflect.{Manifest}
import se.scalablesolutions.akka.dispatch._ import akka.dispatch._
object DispatchersSpec { object DispatchersSpec {
import Dispatchers._ import Dispatchers._

View file

@ -1,10 +1,10 @@
package se.scalablesolutions.akka.actor.dispatch package akka.actor.dispatch
import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.{CountDownLatch, TimeUnit}
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test
import se.scalablesolutions.akka.dispatch.{Dispatchers,ExecutorBasedEventDrivenDispatcher} import akka.dispatch.{Dispatchers,ExecutorBasedEventDrivenDispatcher}
import se.scalablesolutions.akka.actor.Actor import akka.actor.Actor
import Actor._ import Actor._
import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger}

View file

@ -1,10 +1,10 @@
package se.scalablesolutions.akka.actor.dispatch package akka.actor.dispatch
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import java.util.concurrent.CountDownLatch import java.util.concurrent.CountDownLatch
import se.scalablesolutions.akka.actor.Actor import akka.actor.Actor
import Actor._ import Actor._
/** /**

View file

@ -1,4 +1,4 @@
package se.scalablesolutions.akka.actor.dispatch package akka.actor.dispatch
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
@ -6,9 +6,9 @@ import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test
import java.util.concurrent.{TimeUnit, CountDownLatch} import java.util.concurrent.{TimeUnit, CountDownLatch}
import se.scalablesolutions.akka.actor.{IllegalActorStateException, Actor} import akka.actor.{IllegalActorStateException, Actor}
import Actor._ import Actor._
import se.scalablesolutions.akka.dispatch.{MessageQueue, Dispatchers} import akka.dispatch.{MessageQueue, Dispatchers}
object ExecutorBasedEventDrivenWorkStealingDispatcherSpec { object ExecutorBasedEventDrivenWorkStealingDispatcherSpec {
val delayableActorDispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher").build val delayableActorDispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher").build

View file

@ -1,8 +1,8 @@
package se.scalablesolutions.akka.actor package akka.actor
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test
import se.scalablesolutions.akka.dispatch.Futures import akka.dispatch.Futures
import Actor._ import Actor._
import org.multiverse.api.latches.StandardLatch import org.multiverse.api.latches.StandardLatch

View file

@ -1,12 +1,12 @@
package se.scalablesolutions.akka.actor.dispatch package akka.actor.dispatch
import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.{CountDownLatch, TimeUnit}
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test
import se.scalablesolutions.akka.dispatch.{HawtDispatcher, Dispatchers} import akka.dispatch.{HawtDispatcher, Dispatchers}
import se.scalablesolutions.akka.actor.Actor import akka.actor.Actor
import Actor._ import Actor._
object HawtDispatcherActorSpec { object HawtDispatcherActorSpec {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.actor.dispatch package akka.actor.dispatch
import scala.collection.mutable.ListBuffer import scala.collection.mutable.ListBuffer
@ -12,9 +12,9 @@ import java.io.IOException
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.nio.channels.{SocketChannel, SelectionKey, ServerSocketChannel} import java.nio.channels.{SocketChannel, SelectionKey, ServerSocketChannel}
import se.scalablesolutions.akka.actor._ import akka.actor._
import se.scalablesolutions.akka.actor.Actor._ import akka.actor.Actor._
import se.scalablesolutions.akka.dispatch.HawtDispatcher import akka.dispatch.HawtDispatcher
import org.fusesource.hawtdispatch.DispatchSource import org.fusesource.hawtdispatch.DispatchSource
import org.fusesource.hawtdispatch.ScalaDispatch._ import org.fusesource.hawtdispatch.ScalaDispatch._

View file

@ -1,12 +1,12 @@
package se.scalablesolutions.akka.actor.dispatch package akka.actor.dispatch
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test
import se.scalablesolutions.akka.actor.Actor import akka.actor.Actor
import se.scalablesolutions.akka.util.Duration import akka.util.Duration
import se.scalablesolutions.akka.dispatch._ import akka.dispatch._
import Actor._ import Actor._
import java.util.concurrent.{BlockingQueue, CountDownLatch, TimeUnit} import java.util.concurrent.{BlockingQueue, CountDownLatch, TimeUnit}

View file

@ -1,11 +1,11 @@
package se.scalablesolutions.akka.actor.dispatch package akka.actor.dispatch
import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.{CountDownLatch, TimeUnit}
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test
import se.scalablesolutions.akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
import se.scalablesolutions.akka.actor.Actor import akka.actor.Actor
import Actor._ import Actor._
object ThreadBasedActorSpec { object ThreadBasedActorSpec {

View file

@ -1,4 +1,4 @@
package se.scalablesolutions.akka.dispatch package akka.dispatch
import java.util.concurrent.CountDownLatch import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
@ -9,7 +9,7 @@ import java.util.concurrent.locks.ReentrantLock
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.{Test, Before} import org.junit.{Test, Before}
import se.scalablesolutions.akka.actor.Actor import akka.actor.Actor
import Actor._ import Actor._
// FIXME use this test when we have removed the MessageInvoker classes // FIXME use this test when we have removed the MessageInvoker classes

View file

@ -1,4 +1,4 @@
package se.scalablesolutions.akka.japi package akka.japi
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite

View file

@ -1,4 +1,4 @@
package se.scalablesolutions.akka.actor package akka.actor
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test

View file

@ -1,9 +1,9 @@
package se.scalablesolutions.akka.actor package akka.actor
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import Actor._ import Actor._
import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.{CountDownLatch, TimeUnit}
import se.scalablesolutions.akka.config.Supervision._ import akka.config.Supervision._
import org.multiverse.api.latches.StandardLatch import org.multiverse.api.latches.StandardLatch
import org.junit.Test import org.junit.Test

View file

@ -1,8 +1,8 @@
package se.scalablesolutions.akka.actor.routing package akka.actor.routing
import se.scalablesolutions.akka.actor.Actor import akka.actor.Actor
import se.scalablesolutions.akka.actor.Actor._ import akka.actor.Actor._
import se.scalablesolutions.akka.util.Logging import akka.util.Logging
import org.scalatest.Suite import org.scalatest.Suite
import org.junit.runner.RunWith import org.junit.runner.RunWith
@ -12,7 +12,7 @@ import org.junit.Test
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.{CountDownLatch, TimeUnit}
import se.scalablesolutions.akka.routing._ import akka.routing._
@RunWith(classOf[JUnitRunner]) @RunWith(classOf[JUnitRunner])
class RoutingSpec extends junit.framework.TestCase with Suite with MustMatchers with Logging { 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 = { @Test def testIsDefinedAt = {
import se.scalablesolutions.akka.actor.ActorRef import akka.actor.ActorRef
val (testMsg1,testMsg2,testMsg3,testMsg4) = ("test1","test2","test3","test4") val (testMsg1,testMsg2,testMsg3,testMsg4) = ("test1","test2","test3","test4")

View file

@ -1,5 +1,5 @@
package se.scalablesolutions.akka.stm package akka.stm
import org.scalatest.junit.JUnitWrapperSuite 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)

View file

@ -1,11 +1,11 @@
package se.scalablesolutions.akka.stm package akka.stm
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
class RefSpec extends WordSpec with MustMatchers { class RefSpec extends WordSpec with MustMatchers {
import se.scalablesolutions.akka.stm.local._ import akka.stm.local._
"A Ref" should { "A Ref" should {

View file

@ -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 Actor._
import org.multiverse.api.exceptions.ReadonlyException import org.multiverse.api.exceptions.ReadonlyException
@ -12,7 +12,7 @@ class StmSpec extends WordSpec with MustMatchers {
"Local STM" should { "Local STM" should {
import se.scalablesolutions.akka.stm.local._ import akka.stm.local._
"be able to do multiple consecutive atomic {..} statements" in { "be able to do multiple consecutive atomic {..} statements" in {
val ref = Ref(0) val ref = Ref(0)
@ -178,7 +178,7 @@ object GlobalTransactionVectorTestActor {
class GlobalTransactionVectorTestActor extends Actor { class GlobalTransactionVectorTestActor extends Actor {
import GlobalTransactionVectorTestActor._ import GlobalTransactionVectorTestActor._
import se.scalablesolutions.akka.stm.global._ import akka.stm.global._
private val vector: TransactionalVector[Int] = atomic { TransactionalVector(1) } private val vector: TransactionalVector[Int] = atomic { TransactionalVector(1) }

View file

@ -1,4 +1,4 @@
package se.scalablesolutions.akka.actor.ticket package akka.actor.ticket
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers

View file

@ -1,17 +1,17 @@
package se.scalablesolutions.akka.amqp; package akka.amqp;
import org.multiverse.api.latches.StandardLatch; import org.multiverse.api.latches.StandardLatch;
import scala.Option; import scala.Option;
import se.scalablesolutions.akka.actor.ActorRef; import akka.actor.ActorRef;
import se.scalablesolutions.akka.actor.ActorRegistry; import akka.actor.ActorRegistry;
import se.scalablesolutions.akka.actor.UntypedActor; import akka.actor.UntypedActor;
import se.scalablesolutions.akka.actor.UntypedActorFactory; import akka.actor.UntypedActorFactory;
import se.scalablesolutions.akka.amqp.rpc.RPC; import akka.amqp.rpc.RPC;
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol; import akka.remote.protocol.RemoteProtocol;
import se.scalablesolutions.akka.japi.Function; import akka.japi.Function;
import se.scalablesolutions.akka.japi.Procedure; import akka.japi.Procedure;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;

View file

@ -2,22 +2,22 @@
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se> * Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/ */
package se.scalablesolutions.akka.amqp package akka.amqp
import se.scalablesolutions.akka.actor.{Actor, ActorRef} import akka.actor.{Actor, ActorRef}
import se.scalablesolutions.akka.actor.Actor._ import akka.actor.Actor._
import se.scalablesolutions.akka.config.Supervision.OneForOneStrategy import akka.config.Supervision.OneForOneStrategy
import com.rabbitmq.client.{ReturnListener, ShutdownListener, ConnectionFactory} import com.rabbitmq.client.{ReturnListener, ShutdownListener, ConnectionFactory}
import ConnectionFactory._ import ConnectionFactory._
import com.rabbitmq.client.AMQP.BasicProperties import com.rabbitmq.client.AMQP.BasicProperties
import java.lang.{String, IllegalArgumentException} import java.lang.{String, IllegalArgumentException}
import reflect.Manifest import reflect.Manifest
import se.scalablesolutions.akka.japi.Procedure import akka.japi.Procedure
/** /**
* AMQP Actor API. Implements Connection, Producer and Consumer materialized as Actors. * AMQP Actor API. Implements Connection, Producer and Consumer materialized as Actors.
* *
* @see se.scalablesolutions.akka.amqp.ExampleSession * @see akka.amqp.ExampleSession
* *
* @author Irmo Manie * @author Irmo Manie
*/ */

Some files were not shown because too many files have changed in this diff Show more