diff --git a/akka-actor/src/main/scala/actor/Actor.scala b/akka-actor/src/main/scala/actor/Actor.scala index c939e8b62a..57b3dae38e 100644 --- a/akka-actor/src/main/scala/actor/Actor.scala +++ b/akka-actor/src/main/scala/actor/Actor.scala @@ -6,7 +6,7 @@ package se.scalablesolutions.akka.actor import se.scalablesolutions.akka.dispatch._ import se.scalablesolutions.akka.config.Config._ -import se.scalablesolutions.akka.config.ScalaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import se.scalablesolutions.akka.util.Helpers.{narrow, narrowSilently} import se.scalablesolutions.akka.util.{Logging, Duration} import se.scalablesolutions.akka.AkkaException diff --git a/akka-actor/src/main/scala/actor/ActorRef.scala b/akka-actor/src/main/scala/actor/ActorRef.scala index c49777a978..1127da8a2e 100644 --- a/akka-actor/src/main/scala/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/actor/ActorRef.scala @@ -6,8 +6,8 @@ package se.scalablesolutions.akka.actor import se.scalablesolutions.akka.dispatch._ import se.scalablesolutions.akka.config.Config._ -import se.scalablesolutions.akka.config.ScalaConfig._ -import se.scalablesolutions.akka.config.{NoFaultHandlingStrategy, AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy} +import se.scalablesolutions.akka.config.{NoFaultHandlingStrategy, AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy, + LifeCycle, Temporary, Permanent, UndefinedLifeCycle} import se.scalablesolutions.akka.stm.global._ import se.scalablesolutions.akka.stm.TransactionManagement._ import se.scalablesolutions.akka.stm.{ TransactionManagement, TransactionSetAbortedException } diff --git a/akka-actor/src/main/scala/actor/Supervisor.scala b/akka-actor/src/main/scala/actor/Supervisor.scala index ba559e6945..bed8848b21 100644 --- a/akka-actor/src/main/scala/actor/Supervisor.scala +++ b/akka-actor/src/main/scala/actor/Supervisor.scala @@ -4,8 +4,7 @@ package se.scalablesolutions.akka.actor -import se.scalablesolutions.akka.config.ScalaConfig._ -import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy} +import se.scalablesolutions.akka.config.Supervision._ import se.scalablesolutions.akka.AkkaException import se.scalablesolutions.akka.util._ import ReflectiveAccess._ @@ -13,6 +12,7 @@ import Actor._ import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} import java.net.InetSocketAddress +import se.scalablesolutions.akka.config. {Supervision, AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy} class SupervisorException private[akka](message: String) extends AkkaException(message) @@ -83,8 +83,8 @@ object SupervisorFactory { config match { case SupervisorConfig(RestartStrategy(scheme, maxNrOfRetries, timeRange, trapExceptions), _) => scheme match { - case AllForOne => AllForOneStrategy(trapExceptions,maxNrOfRetries, timeRange) - case OneForOne => OneForOneStrategy(trapExceptions,maxNrOfRetries, timeRange) + case a:AllForOne => AllForOneStrategy(trapExceptions.toList,maxNrOfRetries, timeRange) + case o:OneForOne => OneForOneStrategy(trapExceptions.toList,maxNrOfRetries, timeRange) } } } diff --git a/akka-actor/src/main/scala/actor/UntypedActor.scala b/akka-actor/src/main/scala/actor/UntypedActor.scala index cf16b7f064..fa53344c5f 100644 --- a/akka-actor/src/main/scala/actor/UntypedActor.scala +++ b/akka-actor/src/main/scala/actor/UntypedActor.scala @@ -7,7 +7,7 @@ package se.scalablesolutions.akka.actor import se.scalablesolutions.akka.dispatch._ import se.scalablesolutions.akka.stm.global._ import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy} -import se.scalablesolutions.akka.config.ScalaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import java.net.InetSocketAddress diff --git a/akka-actor/src/main/scala/config/Configurator.scala b/akka-actor/src/main/scala/config/Configurator.scala index ba7e1f35f2..eabb57ab51 100644 --- a/akka-actor/src/main/scala/config/Configurator.scala +++ b/akka-actor/src/main/scala/config/Configurator.scala @@ -4,12 +4,12 @@ package se.scalablesolutions.akka.config -import ScalaConfig.{RestartStrategy, Component} +import se.scalablesolutions.akka.config.Supervision. {SuperviseTypedActor, RestartStrategy} private[akka] trait TypedActorConfiguratorBase { def getExternalDependency[T](clazz: Class[T]): T - def configure(restartStrategy: RestartStrategy, components: List[Component]): TypedActorConfiguratorBase + def configure(restartStrategy: RestartStrategy, components: List[SuperviseTypedActor]): TypedActorConfiguratorBase def inject: TypedActorConfiguratorBase diff --git a/akka-actor/src/main/scala/config/SupervisionConfig.scala b/akka-actor/src/main/scala/config/SupervisionConfig.scala index c74820818b..1f30472b05 100644 --- a/akka-actor/src/main/scala/config/SupervisionConfig.scala +++ b/akka-actor/src/main/scala/config/SupervisionConfig.scala @@ -47,17 +47,24 @@ case object NoFaultHandlingStrategy extends FaultHandlingStrategy { def trapExit: List[Class[_ <: Throwable]] = Nil } +sealed abstract class LifeCycle + +case object Permanent extends LifeCycle +case object Temporary extends LifeCycle +case object UndefinedLifeCycle extends LifeCycle + +case class RemoteAddress(val hostname: String, val port: Int) + /** * Configuration classes - not to be used as messages. * * @author Jonas Bonér */ -object ScalaConfig { +object Supervision { sealed abstract class ConfigElement abstract class Server extends ConfigElement abstract class FailOverScheme extends ConfigElement - abstract class LifeCycle extends ConfigElement case class SupervisorConfig(restartStrategy: RestartStrategy, worker: List[Server]) extends Server { //Java API @@ -65,7 +72,7 @@ object ScalaConfig { } class Supervise(val actorRef: ActorRef, val lifeCycle: LifeCycle, _remoteAddress: RemoteAddress) extends Server { - val remoteAddress: Option[RemoteAddress] = if (_remoteAddress eq null) None else Some(_remoteAddress) + val remoteAddress: Option[RemoteAddress] = Option(_remoteAddress) } object Supervise { @@ -74,22 +81,27 @@ object ScalaConfig { def unapply(supervise: Supervise) = Some((supervise.actorRef, supervise.lifeCycle, supervise.remoteAddress)) } - case class RestartStrategy( - scheme: FailOverScheme, - maxNrOfRetries: Int, - withinTimeRange: Int, - trapExceptions: List[Class[_ <: Throwable]]) extends ConfigElement + case class RestartStrategy(scheme: FailOverScheme, maxNrOfRetries: Int, withinTimeRange: Int, trapExceptions: Array[Class[_ <: Throwable]]) extends ConfigElement - case object AllForOne extends FailOverScheme - case object OneForOne extends FailOverScheme + object RestartStrategy { + def apply(scheme: FailOverScheme, maxNrOfRetries: Int, withinTimeRange: Int, trapExceptions: List[Class[_ <: Throwable]]) = + new RestartStrategy(scheme,maxNrOfRetries,withinTimeRange,trapExceptions.toArray) + } - case object Permanent extends LifeCycle - case object Temporary extends LifeCycle - case object UndefinedLifeCycle extends LifeCycle + //Java API + class AllForOne extends FailOverScheme + class OneForOne extends FailOverScheme - case class RemoteAddress(val hostname: String, val port: Int) extends ConfigElement + //Java API (& Scala if you fancy) + def permanent() = se.scalablesolutions.akka.config.Permanent + def temporary() = se.scalablesolutions.akka.config.Temporary + def undefinedLifeCycle = se.scalablesolutions.akka.config.UndefinedLifeCycle - class Component(_intf: Class[_], + //Scala API + object AllForOne extends AllForOne { def apply() = this } + object OneForOne extends OneForOne { def apply() = this } + + case class SuperviseTypedActor(_intf: Class[_], val target: Class[_], val lifeCycle: LifeCycle, val timeout: Long, @@ -97,165 +109,53 @@ object ScalaConfig { _dispatcher: MessageDispatcher, // optional _remoteAddress: RemoteAddress // optional ) extends Server { - val intf: Option[Class[_]] = if (_intf eq null) None else Some(_intf) - val dispatcher: Option[MessageDispatcher] = if (_dispatcher eq null) None else Some(_dispatcher) - val remoteAddress: Option[RemoteAddress] = if (_remoteAddress eq null) None else Some(_remoteAddress) - } - object Component { - def apply(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long) = - new Component(intf, target, lifeCycle, timeout, false, null, null) - - def apply(target: Class[_], lifeCycle: LifeCycle, timeout: Long) = - new Component(null, target, lifeCycle, timeout, false, null, null) - - def apply(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, dispatcher: MessageDispatcher) = - new Component(intf, target, lifeCycle, timeout, false, dispatcher, null) - - def apply(target: Class[_], lifeCycle: LifeCycle, timeout: Long, dispatcher: MessageDispatcher) = - new Component(null, target, lifeCycle, timeout, false, dispatcher, null) - - def apply(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, remoteAddress: RemoteAddress) = - new Component(intf, target, lifeCycle, timeout, false, null, remoteAddress) - - def apply(target: Class[_], lifeCycle: LifeCycle, timeout: Long, remoteAddress: RemoteAddress) = - new Component(null, target, lifeCycle, timeout, false, null, remoteAddress) - - def apply(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, dispatcher: MessageDispatcher, remoteAddress: RemoteAddress) = - new Component(intf, target, lifeCycle, timeout, false, dispatcher, remoteAddress) - - def apply(target: Class[_], lifeCycle: LifeCycle, timeout: Long, dispatcher: MessageDispatcher, remoteAddress: RemoteAddress) = - new Component(null, target, lifeCycle, timeout, false, dispatcher, remoteAddress) - - def apply(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean) = - new Component(intf, target, lifeCycle, timeout, transactionRequired, null, null) - - def apply(target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean) = - new Component(null, target, lifeCycle, timeout, transactionRequired, null, null) - - def apply(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean, dispatcher: MessageDispatcher) = - new Component(intf, target, lifeCycle, timeout, transactionRequired, dispatcher, null) - - def apply(target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean, dispatcher: MessageDispatcher) = - new Component(null, target, lifeCycle, timeout, transactionRequired, dispatcher, null) - - def apply(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean, remoteAddress: RemoteAddress) = - new Component(intf, target, lifeCycle, timeout, transactionRequired, null, remoteAddress) - - def apply(target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean, remoteAddress: RemoteAddress) = - new Component(null, target, lifeCycle, timeout, transactionRequired, null, remoteAddress) - - def apply(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean, dispatcher: MessageDispatcher, remoteAddress: RemoteAddress) = - new Component(intf, target, lifeCycle, timeout, transactionRequired, dispatcher, remoteAddress) - - def apply(target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean, dispatcher: MessageDispatcher, remoteAddress: RemoteAddress) = - new Component(null, target, lifeCycle, timeout, transactionRequired, dispatcher, remoteAddress) - } -} - -/** - * @author Jonas Bonér - */ -object JavaConfig { - import scala.reflect.BeanProperty - - sealed abstract class ConfigElement - - class RestartStrategy( - @BeanProperty val scheme: FailOverScheme, - @BeanProperty val maxNrOfRetries: Int, - @BeanProperty val withinTimeRange: Int, - @BeanProperty val trapExceptions: Array[Class[_ <: Throwable]]) extends ConfigElement { - def transform = se.scalablesolutions.akka.config.ScalaConfig.RestartStrategy( - scheme.transform, maxNrOfRetries, withinTimeRange, trapExceptions.toList) - } - - abstract class LifeCycle extends ConfigElement { - def transform: se.scalablesolutions.akka.config.ScalaConfig.LifeCycle - } - - class Permanent extends LifeCycle { - override def transform = se.scalablesolutions.akka.config.ScalaConfig.Permanent - } - - class Temporary extends LifeCycle { - override def transform = se.scalablesolutions.akka.config.ScalaConfig.Temporary - } - - class UndefinedLifeCycle extends LifeCycle { - override def transform = se.scalablesolutions.akka.config.ScalaConfig.UndefinedLifeCycle - } - - abstract class FailOverScheme extends ConfigElement { - def transform: se.scalablesolutions.akka.config.ScalaConfig.FailOverScheme - } - class AllForOne extends FailOverScheme { - override def transform = se.scalablesolutions.akka.config.ScalaConfig.AllForOne - } - class OneForOne extends FailOverScheme { - override def transform = se.scalablesolutions.akka.config.ScalaConfig.OneForOne - } - - class RemoteAddress(@BeanProperty val hostname: String, @BeanProperty val port: Int) - - abstract class Server extends ConfigElement - class Component(@BeanProperty val intf: Class[_], - @BeanProperty val target: Class[_], - @BeanProperty val lifeCycle: LifeCycle, - @BeanProperty val timeout: Long, - @BeanProperty val transactionRequired: Boolean, // optional - @BeanProperty val dispatcher: MessageDispatcher, // optional - @BeanProperty val remoteAddress: RemoteAddress // optional - ) extends Server { - - def this(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long) = - this(intf, target, lifeCycle, timeout, false, null, null) + val intf: Option[Class[_]] = Option(_intf) + val dispatcher: Option[MessageDispatcher] = Option(_dispatcher) + val remoteAddress: Option[RemoteAddress] = Option(_remoteAddress) def this(target: Class[_], lifeCycle: LifeCycle, timeout: Long) = - this(null, target, lifeCycle, timeout, false, null, null) + this(null: Class[_], target, lifeCycle, timeout, false, null.asInstanceOf[MessageDispatcher], null: RemoteAddress) - def this(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, remoteAddress: RemoteAddress) = - this(intf, target, lifeCycle, timeout, false, null, remoteAddress) - - def this(target: Class[_], lifeCycle: LifeCycle, timeout: Long, remoteAddress: RemoteAddress) = - this(null, target, lifeCycle, timeout, false, null, remoteAddress) + def this(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long) = + this(intf, target, lifeCycle, timeout, false, null.asInstanceOf[MessageDispatcher], null: RemoteAddress) def this(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, dispatcher: MessageDispatcher) = this(intf, target, lifeCycle, timeout, false, dispatcher, null) def this(target: Class[_], lifeCycle: LifeCycle, timeout: Long, dispatcher: MessageDispatcher) = - this(null, target, lifeCycle, timeout, false, dispatcher, null) + this(null: Class[_], target, lifeCycle, timeout, false, dispatcher, null:RemoteAddress) + + def this(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, remoteAddress: RemoteAddress) = + this(intf, target, lifeCycle, timeout, false, null: MessageDispatcher, remoteAddress) + + def this(target: Class[_], lifeCycle: LifeCycle, timeout: Long, remoteAddress: RemoteAddress) = + this(null: Class[_], target, lifeCycle, timeout, false, null, remoteAddress) + + def this(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, dispatcher: MessageDispatcher, remoteAddress: RemoteAddress) = + this(intf, target, lifeCycle, timeout, false, dispatcher, remoteAddress) def this(target: Class[_], lifeCycle: LifeCycle, timeout: Long, dispatcher: MessageDispatcher, remoteAddress: RemoteAddress) = - this(null, target, lifeCycle, timeout, false, dispatcher, remoteAddress) + this(null: Class[_], target, lifeCycle, timeout, false, dispatcher, remoteAddress) def this(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean) = - this(intf, target, lifeCycle, timeout, transactionRequired, null, null) + this(intf, target, lifeCycle, timeout, transactionRequired, null: MessageDispatcher, null: RemoteAddress) def this(target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean) = - this(null, target, lifeCycle, timeout, transactionRequired, null, null) - - def this(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean, remoteAddress: RemoteAddress) = - this(intf, target, lifeCycle, timeout, transactionRequired, null, remoteAddress) - - def this(target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean, remoteAddress: RemoteAddress) = - this(null, target, lifeCycle, timeout, transactionRequired, null, remoteAddress) + this(null: Class[_], target, lifeCycle, timeout, transactionRequired, null: MessageDispatcher, null: RemoteAddress) def this(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean, dispatcher: MessageDispatcher) = - this(intf, target, lifeCycle, timeout, transactionRequired, dispatcher, null) + this(intf, target, lifeCycle, timeout, transactionRequired, dispatcher, null: RemoteAddress) def this(target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean, dispatcher: MessageDispatcher) = - this(null, target, lifeCycle, timeout, transactionRequired, dispatcher, null) + this(null: Class[_], target, lifeCycle, timeout, transactionRequired, dispatcher, null: RemoteAddress) + + def this(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean, remoteAddress: RemoteAddress) = + this(intf, target, lifeCycle, timeout, transactionRequired, null: MessageDispatcher, remoteAddress) + + def this(target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean, remoteAddress: RemoteAddress) = + this(null: Class[_], target, lifeCycle, timeout, transactionRequired, null: MessageDispatcher, remoteAddress) def this(target: Class[_], lifeCycle: LifeCycle, timeout: Long, transactionRequired: Boolean, dispatcher: MessageDispatcher, remoteAddress: RemoteAddress) = - this(null, target, lifeCycle, timeout, transactionRequired, dispatcher, remoteAddress) - - def transform = - se.scalablesolutions.akka.config.ScalaConfig.Component( - intf, target, lifeCycle.transform, timeout, transactionRequired, dispatcher, - if (remoteAddress ne null) se.scalablesolutions.akka.config.ScalaConfig.RemoteAddress(remoteAddress.hostname, remoteAddress.port) else null) - - def newSupervised(actorRef: ActorRef) = - se.scalablesolutions.akka.config.ScalaConfig.Supervise(actorRef, lifeCycle.transform) + this(null: Class[_], target, lifeCycle, timeout, transactionRequired, dispatcher, remoteAddress) } - -} +} \ No newline at end of file diff --git a/akka-actor/src/test/java/se/scalablesolutions/akka/config/SupervisionConfig.java b/akka-actor/src/test/java/se/scalablesolutions/akka/config/SupervisionConfig.java new file mode 100644 index 0000000000..d8de22eb59 --- /dev/null +++ b/akka-actor/src/test/java/se/scalablesolutions/akka/config/SupervisionConfig.java @@ -0,0 +1,20 @@ +package se.scalablesolutions.akka.config; + +import se.scalablesolutions.akka.actor.ActorRef; + +import java.util.ArrayList; +import java.util.List; + +import static se.scalablesolutions.akka.config.Supervision.*; + +public class SupervisionConfig { + /*Just some sample code to demonstrate the declarative supervision configuration for Java */ + public SupervisorConfig createSupervisorConfig(List toSupervise) { + ArrayList targets = new ArrayList(toSupervise.size()); + for(ActorRef ref : toSupervise) { + targets.add(new Supervise(ref, permanent(), new RemoteAddress("localhost",9999))); + } + + return new SupervisorConfig(new RestartStrategy(new OneForOne(),50,1000,new Class[]{ Exception.class}), targets.toArray(new Server[0])); + } +} diff --git a/akka-actor/src/test/scala/actor/actor/ActorFireForgetRequestReplySpec.scala b/akka-actor/src/test/scala/actor/actor/ActorFireForgetRequestReplySpec.scala index 7741b79cea..f04d8016d2 100644 --- a/akka-actor/src/test/scala/actor/actor/ActorFireForgetRequestReplySpec.scala +++ b/akka-actor/src/test/scala/actor/actor/ActorFireForgetRequestReplySpec.scala @@ -1,7 +1,7 @@ package se.scalablesolutions.akka.actor import java.util.concurrent.{TimeUnit, CyclicBarrier, TimeoutException} -import se.scalablesolutions.akka.config.ScalaConfig._ +import se.scalablesolutions.akka.config._ import org.scalatest.junit.JUnitSuite import org.junit.Test diff --git a/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala b/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala index 887785f568..6b105061a7 100644 --- a/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala +++ b/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala @@ -12,7 +12,7 @@ import org.junit.Test import Actor._ import se.scalablesolutions.akka.config.OneForOneStrategy import java.util.concurrent.{TimeUnit, CountDownLatch} -import se.scalablesolutions.akka.config.ScalaConfig.{Permanent, LifeCycle} +import se.scalablesolutions.akka.config.{Permanent, LifeCycle} import org.multiverse.api.latches.StandardLatch class RestartStrategySpec extends JUnitSuite { diff --git a/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala b/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala index 2805a8675d..933d9643f7 100644 --- a/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala +++ b/akka-actor/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala @@ -6,8 +6,9 @@ package se.scalablesolutions.akka.actor import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers import se.scalablesolutions.akka.dispatch.Dispatchers -import se.scalablesolutions.akka.config.ScalaConfig.{RestartStrategy, SupervisorConfig, LifeCycle, Permanent, OneForOne, Supervise} +import se.scalablesolutions.akka.config.Supervision.{RestartStrategy, SupervisorConfig, OneForOne, Supervise} import java.util.concurrent.CountDownLatch +import se.scalablesolutions.akka.config.Permanent class SupervisorMiscSpec extends WordSpec with MustMatchers { "A Supervisor" should { @@ -57,7 +58,7 @@ class SupervisorMiscSpec extends WordSpec with MustMatchers { val sup = Supervisor( SupervisorConfig( - RestartStrategy(OneForOne, 3, 5000, List(classOf[Exception])), + RestartStrategy(new OneForOne, 3, 5000, List(classOf[Exception])), Supervise(actor1, Permanent) :: Supervise(actor2, Permanent) :: Supervise(actor3, Permanent) :: diff --git a/akka-actor/src/test/scala/actor/supervisor/SupervisorSpec.scala b/akka-actor/src/test/scala/actor/supervisor/SupervisorSpec.scala index d7390a0d43..ee32ca7f3a 100644 --- a/akka-actor/src/test/scala/actor/supervisor/SupervisorSpec.scala +++ b/akka-actor/src/test/scala/actor/supervisor/SupervisorSpec.scala @@ -4,14 +4,14 @@ package se.scalablesolutions.akka.actor -import se.scalablesolutions.akka.config.ScalaConfig._ -import se.scalablesolutions.akka.config.OneForOneStrategy +import se.scalablesolutions.akka.config.Supervision._ import se.scalablesolutions.akka.{OneWay, Die, Ping} import Actor._ import org.scalatest.junit.JUnitSuite import org.junit.Test import java.util.concurrent.{TimeUnit, LinkedBlockingQueue} +import se.scalablesolutions.akka.config.{Temporary, Permanent, OneForOneStrategy} object SupervisorSpec { var messageLog = new LinkedBlockingQueue[String] @@ -78,7 +78,7 @@ object SupervisorSpec { class TemporaryActor extends Actor { import self._ - lifeCycle = Temporary + lifeCycle = se.scalablesolutions.akka.config.Temporary def receive = { case Ping => messageLog.put("ping") diff --git a/akka-actor/src/test/scala/misc/SchedulerSpec.scala b/akka-actor/src/test/scala/misc/SchedulerSpec.scala index 2c7d43896c..9f0e127e56 100644 --- a/akka-actor/src/test/scala/misc/SchedulerSpec.scala +++ b/akka-actor/src/test/scala/misc/SchedulerSpec.scala @@ -3,9 +3,10 @@ package se.scalablesolutions.akka.actor import org.scalatest.junit.JUnitSuite import Actor._ import java.util.concurrent.{CountDownLatch, TimeUnit} -import se.scalablesolutions.akka.config.ScalaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import org.multiverse.api.latches.StandardLatch import org.junit.Test +import se.scalablesolutions.akka.config.Permanent class SchedulerSpec extends JUnitSuite { @@ -98,7 +99,7 @@ class SchedulerSpec extends JUnitSuite { val pingLatch = new CountDownLatch(6) val actor = actorOf(new Actor { - self.lifeCycle = Permanent + self.lifeCycle = se.scalablesolutions.akka.config.Permanent def receive = { case Ping => pingLatch.countDown @@ -107,6 +108,7 @@ class SchedulerSpec extends JUnitSuite { override def postRestart(reason: Throwable) = restartLatch.open }) + Supervisor( SupervisorConfig( RestartStrategy(AllForOne, 3, 1000, 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 16ec8db389..acb61f02e3 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 @@ -8,8 +8,7 @@ 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.ScalaConfig.{Permanent} -import se.scalablesolutions.akka.config.OneForOneStrategy +import se.scalablesolutions.akka.config.{ Permanent, OneForOneStrategy } import se.scalablesolutions.akka.actor.{Exit, Actor} private[amqp] class FaultTolerantConnectionActor(connectionParameters: ConnectionParameters) extends Actor { diff --git a/akka-http/src/test/scala/SecuritySpec.scala b/akka-http/src/test/scala/SecuritySpec.scala index 6a3cf4f803..5931df69d0 100644 --- a/akka-http/src/test/scala/SecuritySpec.scala +++ b/akka-http/src/test/scala/SecuritySpec.scala @@ -4,7 +4,7 @@ package se.scalablesolutions.akka.security -import se.scalablesolutions.akka.config.ScalaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import se.scalablesolutions.akka.actor.Actor._ import org.scalatest.Suite 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 14eba7d4e3..db7d96876a 100644 --- a/akka-persistence/akka-persistence-common/src/test/scala/Ticket343Test.scala +++ b/akka-persistence/akka-persistence-common/src/test/scala/Ticket343Test.scala @@ -11,10 +11,9 @@ import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith import se.scalablesolutions.akka.actor.{Actor, ActorRef} -import se.scalablesolutions.akka.config.OneForOneStrategy +import se.scalablesolutions.akka.config.{OneForOneStrategy, Permanent} import Actor._ import se.scalablesolutions.akka.stm.global._ -import se.scalablesolutions.akka.config.ScalaConfig._ import se.scalablesolutions.akka.util.Logging import StorageObj._ 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 930a3b25a7..9c1c91d01e 100644 --- a/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseTicket343SpecTestIntegration.scala +++ b/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseTicket343SpecTestIntegration.scala @@ -7,10 +7,9 @@ import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith import se.scalablesolutions.akka.actor.{Actor, ActorRef} -import se.scalablesolutions.akka.config.OneForOneStrategy +import se.scalablesolutions.akka.config.{OneForOneStrategy,Permanent} import Actor._ import se.scalablesolutions.akka.stm.global._ -import se.scalablesolutions.akka.config.ScalaConfig._ import se.scalablesolutions.akka.util.Logging import HbaseStorageBackend._ 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 a614fbc78d..c34f237566 100644 --- a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala +++ b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala @@ -7,10 +7,9 @@ import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith import se.scalablesolutions.akka.actor.{Actor, ActorRef} -import se.scalablesolutions.akka.config.OneForOneStrategy +import se.scalablesolutions.akka.config.{OneForOneStrategy,Permanent} import Actor._ import se.scalablesolutions.akka.stm.global._ -import se.scalablesolutions.akka.config.ScalaConfig._ import se.scalablesolutions.akka.util.Logging import MongoStorageBackend._ 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 1bd2c34d86..a51d4150b1 100644 --- a/akka-persistence/akka-persistence-redis/src/test/scala/RedisInconsistentSizeBugTest.scala +++ b/akka-persistence/akka-persistence-redis/src/test/scala/RedisInconsistentSizeBugTest.scala @@ -5,11 +5,10 @@ import sbinary.Operations._ import sbinary.DefaultProtocol._ import se.scalablesolutions.akka.actor.{Actor, ActorRef} -import se.scalablesolutions.akka.config.OneForOneStrategy +import se.scalablesolutions.akka.config.{OneForOneStrategy, Permanent} import Actor._ import se.scalablesolutions.akka.persistence.common.PersistentVector import se.scalablesolutions.akka.stm.global._ -import se.scalablesolutions.akka.config.ScalaConfig._ import se.scalablesolutions.akka.util.Logging import java.util.{Calendar, Date} 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 f46aa9f224..46f233ecf9 100644 --- a/akka-persistence/akka-persistence-redis/src/test/scala/RedisTicket343Spec.scala +++ b/akka-persistence/akka-persistence-redis/src/test/scala/RedisTicket343Spec.scala @@ -7,11 +7,10 @@ import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith import se.scalablesolutions.akka.actor.{Actor} -import se.scalablesolutions.akka.config.OneForOneStrategy +import se.scalablesolutions.akka.config.{OneForOneStrategy,Permanent} import Actor._ import se.scalablesolutions.akka.persistence.common.PersistentVector import se.scalablesolutions.akka.stm.global._ -import se.scalablesolutions.akka.config.ScalaConfig._ import se.scalablesolutions.akka.util.Logging import RedisStorageBackend._ diff --git a/akka-remote/src/main/scala/remote/Cluster.scala b/akka-remote/src/main/scala/remote/Cluster.scala index c668228291..71354e8e77 100644 --- a/akka-remote/src/main/scala/remote/Cluster.scala +++ b/akka-remote/src/main/scala/remote/Cluster.scala @@ -5,11 +5,12 @@ package se.scalablesolutions.akka.remote import se.scalablesolutions.akka.config.Config.config -import se.scalablesolutions.akka.config.ScalaConfig._ +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 scala.collection.immutable.{Map, HashMap} +import se.scalablesolutions.akka.config. {Permanent, RemoteAddress} /** * Interface for interacting with the Cluster Membership API. diff --git a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala index 6ebc624e62..2343608607 100644 --- a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala +++ b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala @@ -11,8 +11,7 @@ import se.scalablesolutions.akka.dispatch.MessageInvocation import se.scalablesolutions.akka.remote.{RemoteServer, MessageSerializer} import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.{ActorType => ActorTypeProtocol, _} import ActorTypeProtocol._ -import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy} -import se.scalablesolutions.akka.config.ScalaConfig._ +import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy, Permanent, Temporary, UndefinedLifeCycle} import se.scalablesolutions.akka.actor.{uuidFrom,newUuid} import se.scalablesolutions.akka.actor._ diff --git a/akka-remote/src/test/scala/remote/RemoteSupervisorSpec.scala b/akka-remote/src/test/scala/remote/RemoteSupervisorSpec.scala index 40f0d27640..e815231509 100644 --- a/akka-remote/src/test/scala/remote/RemoteSupervisorSpec.scala +++ b/akka-remote/src/test/scala/remote/RemoteSupervisorSpec.scala @@ -6,13 +6,14 @@ package se.scalablesolutions.akka.actor.remote import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, BlockingQueue} import se.scalablesolutions.akka.serialization.BinaryString -import se.scalablesolutions.akka.config.ScalaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import se.scalablesolutions.akka.remote.{RemoteServer, RemoteClient} import se.scalablesolutions.akka.OneWay import org.scalatest.junit.JUnitSuite import org.junit.{Test, Before, After} import se.scalablesolutions.akka.actor.{SupervisorFactory, Supervisor, ActorRef, Actor} import Actor._ +import se.scalablesolutions.akka.config.Permanent object Log { val messageLog: BlockingQueue[String] = new LinkedBlockingQueue[String] diff --git a/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala b/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala index 5a3a5bc2c4..9b5f407a27 100644 --- a/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala +++ b/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala @@ -8,15 +8,13 @@ import org.scalatest.matchers.ShouldMatchers 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.JavaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import se.scalablesolutions.akka.actor._ import se.scalablesolutions.akka.remote.{RemoteServer, RemoteClient} import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, BlockingQueue} import org.scalatest.{BeforeAndAfterEach, Spec, Assertions, BeforeAndAfterAll} +import se.scalablesolutions.akka.config. {Permanent, Config, TypedActorConfigurator, RemoteAddress} object RemoteTypedActorSpec { val HOSTNAME = "localhost" @@ -50,18 +48,18 @@ class RemoteTypedActorSpec extends server.start("localhost", 9995) Config.config conf.configure( - new RestartStrategy(new AllForOne, 3, 5000, List(classOf[Exception]).toArray), + new RestartStrategy(AllForOne(), 3, 5000, List(classOf[Exception]).toArray), List( - new Component( + new SuperviseTypedActor( classOf[RemoteTypedActorOne], classOf[RemoteTypedActorOneImpl], - new Permanent, + Permanent, 10000, new RemoteAddress("localhost", 9995)), - new Component( + new SuperviseTypedActor( classOf[RemoteTypedActorTwo], classOf[RemoteTypedActorTwoImpl], - new Permanent, + Permanent, 10000, new RemoteAddress("localhost", 9995)) ).toArray).supervise 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 fc6afc856a..3892583026 100644 --- a/akka-samples/akka-sample-camel/src/main/scala/Boot.scala +++ b/akka-samples/akka-sample-camel/src/main/scala/Boot.scala @@ -9,7 +9,7 @@ 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.ScalaConfig._ +import se.scalablesolutions.akka.config.Supervision._ /** * @author Martin Krasser 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 b65f833763..d2d09fc22e 100644 --- a/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala +++ b/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala @@ -11,8 +11,7 @@ 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.ScalaConfig._ -import se.scalablesolutions.akka.config.OneForOneStrategy +import se.scalablesolutions.akka.config.{OneForOneStrategy,Permanent} import se.scalablesolutions.akka.util.Logging import Actor._ 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 4702eead02..4775e0e7a8 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 @@ -5,23 +5,23 @@ package sample.rest.java; import se.scalablesolutions.akka.config.TypedActorConfigurator; -import static se.scalablesolutions.akka.config.JavaConfig.*; +import static se.scalablesolutions.akka.config.Supervision.*; public class Boot { public final static TypedActorConfigurator configurator = new TypedActorConfigurator(); static { configurator.configure( new RestartStrategy(new OneForOne(), 3, 5000, new Class[]{Exception.class}), - new Component[] { - new Component( + new SuperviseTypedActor[] { + new SuperviseTypedActor( SimpleService.class, SimpleServiceImpl.class, - new Permanent(), + permanent(), 1000), - new Component( + new SuperviseTypedActor( PersistentSimpleService.class, PersistentSimpleServiceImpl.class, - new Permanent(), + permanent(), 1000) }).supervise(); } 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 fb8bd7c381..31547e85a7 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 @@ -8,7 +8,7 @@ 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.ScalaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import se.scalablesolutions.akka.util.Logging import se.scalablesolutions.akka.comet.AkkaClusterBroadcastFilter import scala.xml.NodeSeq @@ -21,6 +21,7 @@ import org.atmosphere.annotation.{Broadcast, Suspend,Cluster} import org.atmosphere.util.XSSHtmlFilter import org.atmosphere.cpr.{Broadcaster, BroadcastFilter} import org.atmosphere.jersey.Broadcastable +import se.scalablesolutions.akka.config.Permanent class Boot { val factory = SupervisorFactory( 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 3f2b76a359..5af3a7ff69 100644 --- a/akka-samples/akka-sample-security/src/main/scala/SimpleService.scala +++ b/akka-samples/akka-sample-security/src/main/scala/SimpleService.scala @@ -6,11 +6,12 @@ package sample.security import se.scalablesolutions.akka.actor.{SupervisorFactory, Transactor, Actor} import se.scalablesolutions.akka.actor.Actor._ -import se.scalablesolutions.akka.config.ScalaConfig._ +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 se.scalablesolutions.akka.config.Permanent class Boot { val factory = SupervisorFactory( diff --git a/akka-spring/src/main/scala/DispatcherFactoryBean.scala b/akka-spring/src/main/scala/DispatcherFactoryBean.scala index 34a3a012ea..893b44e24d 100644 --- a/akka-spring/src/main/scala/DispatcherFactoryBean.scala +++ b/akka-spring/src/main/scala/DispatcherFactoryBean.scala @@ -4,7 +4,7 @@ package se.scalablesolutions.akka.spring import org.springframework.beans.factory.config.AbstractFactoryBean -import se.scalablesolutions.akka.config.JavaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import AkkaSpringConfigurationTags._ import reflect.BeanProperty import se.scalablesolutions.akka.actor.ActorRef diff --git a/akka-spring/src/main/scala/SupervisionBeanDefinitionParser.scala b/akka-spring/src/main/scala/SupervisionBeanDefinitionParser.scala index 164018f588..cb797ba29a 100644 --- a/akka-spring/src/main/scala/SupervisionBeanDefinitionParser.scala +++ b/akka-spring/src/main/scala/SupervisionBeanDefinitionParser.scala @@ -6,7 +6,7 @@ package se.scalablesolutions.akka.spring import se.scalablesolutions.akka.util.Logging import org.springframework.beans.factory.support.BeanDefinitionBuilder import org.springframework.beans.factory.xml.{ParserContext, AbstractSingleBeanDefinitionParser} -import se.scalablesolutions.akka.config.JavaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import AkkaSpringConfigurationTags._ @@ -47,7 +47,7 @@ class SupervisionBeanDefinitionParser extends AbstractSingleBeanDefinitionParser } private[akka] def parseRestartStrategy(element: Element, builder: BeanDefinitionBuilder) { - val failover = if (mandatory(element, FAILOVER) == "AllForOne") new AllForOne() else new OneForOne() + val failover = if (mandatory(element, FAILOVER) == "AllForOne") AllForOne else OneForOne val timeRange = mandatory(element, TIME_RANGE).toInt val retries = mandatory(element, RETRIES).toInt val trapExitsElement = mandatoryElement(element, TRAP_EXISTS_TAG) @@ -71,7 +71,7 @@ class SupervisionBeanDefinitionParser extends AbstractSingleBeanDefinitionParser private def parseTrapExits(element: Element): Array[Class[_ <: Throwable]] = { import StringReflect._ val trapExits = DomUtils.getChildElementsByTagName(element, TRAP_EXIT_TAG).toArray.toList.asInstanceOf[List[Element]] - trapExits.map(DomUtils.getTextValue(_).toClass).toArray + trapExits.map(DomUtils.getTextValue(_).toClass.asInstanceOf[Class[_ <: Throwable]]).toArray } /* diff --git a/akka-spring/src/main/scala/SupervisionFactoryBean.scala b/akka-spring/src/main/scala/SupervisionFactoryBean.scala index c6d1e7ddc0..b11117ee35 100644 --- a/akka-spring/src/main/scala/SupervisionFactoryBean.scala +++ b/akka-spring/src/main/scala/SupervisionFactoryBean.scala @@ -4,13 +4,11 @@ package se.scalablesolutions.akka.spring import org.springframework.beans.factory.config.AbstractFactoryBean -import se.scalablesolutions.akka.config.TypedActorConfigurator -import se.scalablesolutions.akka.config.JavaConfig._ -import se.scalablesolutions.akka.config.ScalaConfig.{Supervise, Server, SupervisorConfig, RemoteAddress => SRemoteAddress} +import se.scalablesolutions.akka.config.Supervision._ import se.scalablesolutions.akka.actor.{Supervisor, SupervisorFactory, Actor} import AkkaSpringConfigurationTags._ import reflect.BeanProperty - +import se.scalablesolutions.akka.config.{Temporary, Permanent, TypedActorConfigurator, RemoteAddress} /** * Factory bean for supervisor configuration. @@ -46,7 +44,7 @@ class SupervisionFactoryBean extends AbstractFactoryBean[AnyRef] { private def createInstanceForUntypedActors() : Supervisor = { val factory = new SupervisorFactory( new SupervisorConfig( - restartStrategy.transform, + restartStrategy, supervised.map(createSupervise(_)))) factory.newInstance } @@ -54,24 +52,24 @@ class SupervisionFactoryBean extends AbstractFactoryBean[AnyRef] { /** * Create configuration for TypedActor */ - private[akka] def createComponent(props: ActorProperties): Component = { + private[akka] def createComponent(props: ActorProperties): SuperviseTypedActor = { import StringReflect._ - val lifeCycle = if (!props.lifecycle.isEmpty && props.lifecycle.equalsIgnoreCase(VAL_LIFECYCYLE_TEMPORARY)) new Temporary() else new Permanent() + val lifeCycle = if (!props.lifecycle.isEmpty && props.lifecycle.equalsIgnoreCase(VAL_LIFECYCYLE_TEMPORARY)) Temporary else Permanent val isRemote = (props.host ne null) && (!props.host.isEmpty) val withInterface = (props.interface ne null) && (!props.interface.isEmpty) if (isRemote) { //val remote = new RemoteAddress(props.host, props.port) val remote = new RemoteAddress(props.host, props.port.toInt) if (withInterface) { - new Component(props.interface.toClass, props.target.toClass, lifeCycle, props.timeout, props.transactional, remote) + new SuperviseTypedActor(props.interface.toClass, props.target.toClass, lifeCycle, props.timeout, props.transactional, remote) } else { - new Component(props.target.toClass, lifeCycle, props.timeout, props.transactional, remote) + new SuperviseTypedActor(props.target.toClass, lifeCycle, props.timeout, props.transactional, remote) } } else { if (withInterface) { - new Component(props.interface.toClass, props.target.toClass, lifeCycle, props.timeout, props.transactional) + new SuperviseTypedActor(props.interface.toClass, props.target.toClass, lifeCycle, props.timeout, props.transactional) } else { - new Component(props.target.toClass, lifeCycle, props.timeout, props.transactional) + new SuperviseTypedActor(props.target.toClass, lifeCycle, props.timeout, props.transactional) } } } @@ -81,7 +79,7 @@ class SupervisionFactoryBean extends AbstractFactoryBean[AnyRef] { */ private[akka] def createSupervise(props: ActorProperties): Server = { import StringReflect._ - val lifeCycle = if (!props.lifecycle.isEmpty && props.lifecycle.equalsIgnoreCase(VAL_LIFECYCYLE_TEMPORARY)) new Temporary() else new Permanent() + val lifeCycle = if (!props.lifecycle.isEmpty && props.lifecycle.equalsIgnoreCase(VAL_LIFECYCYLE_TEMPORARY)) Temporary else Permanent val isRemote = (props.host ne null) && (!props.host.isEmpty) val actorRef = Actor.actorOf(props.target.toClass) if (props.timeout > 0) { @@ -92,10 +90,10 @@ class SupervisionFactoryBean extends AbstractFactoryBean[AnyRef] { } val supervise = if (isRemote) { - val remote = new SRemoteAddress(props.host, props.port.toInt) - Supervise(actorRef, lifeCycle.transform, remote) + val remote = new RemoteAddress(props.host, props.port.toInt) + Supervise(actorRef, lifeCycle, remote) } else { - Supervise(actorRef, lifeCycle.transform) + Supervise(actorRef, lifeCycle) } supervise } diff --git a/akka-spring/src/test/scala/DispatcherFactoryBeanTest.scala b/akka-spring/src/test/scala/DispatcherFactoryBeanTest.scala index 3453fb5200..f4e9f640a4 100644 --- a/akka-spring/src/test/scala/DispatcherFactoryBeanTest.scala +++ b/akka-spring/src/test/scala/DispatcherFactoryBeanTest.scala @@ -7,7 +7,7 @@ import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.config.JavaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import se.scalablesolutions.akka.dispatch.MessageDispatcher @RunWith(classOf[JUnitRunner]) diff --git a/akka-spring/src/test/scala/SupervisionBeanDefinitionParserTest.scala b/akka-spring/src/test/scala/SupervisionBeanDefinitionParserTest.scala index 15734fc9fa..45effd28e5 100644 --- a/akka-spring/src/test/scala/SupervisionBeanDefinitionParserTest.scala +++ b/akka-spring/src/test/scala/SupervisionBeanDefinitionParserTest.scala @@ -9,10 +9,9 @@ import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith import ScalaDom._ -import se.scalablesolutions.akka.config.JavaConfig._ - import org.w3c.dom.Element import org.springframework.beans.factory.support.BeanDefinitionBuilder +import se.scalablesolutions.akka.config.Supervision. {RestartStrategy, AllForOne} /** * Test for SupervisionBeanDefinitionParser @@ -39,7 +38,7 @@ class SupervisionBeanDefinitionParserTest extends Spec with ShouldMatchers { val strategy = builder.getBeanDefinition.getPropertyValues.getPropertyValue("restartStrategy").getValue.asInstanceOf[RestartStrategy] assert(strategy ne null) assert(strategy.scheme match { - case x:AllForOne => true + case AllForOne => true case _ => false }) expect(3) { strategy.maxNrOfRetries } expect(1000) { strategy.withinTimeRange } diff --git a/akka-spring/src/test/scala/SupervisionFactoryBeanTest.scala b/akka-spring/src/test/scala/SupervisionFactoryBeanTest.scala index 79872b18d4..d85cfff942 100644 --- a/akka-spring/src/test/scala/SupervisionFactoryBeanTest.scala +++ b/akka-spring/src/test/scala/SupervisionFactoryBeanTest.scala @@ -7,7 +7,7 @@ import org.scalatest.Spec import org.scalatest.matchers.ShouldMatchers import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import se.scalablesolutions.akka.config.JavaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import se.scalablesolutions.akka.config.TypedActorConfigurator private[akka] class Foo @@ -15,7 +15,7 @@ private[akka] class Foo @RunWith(classOf[JUnitRunner]) class SupervisionFactoryBeanTest extends Spec with ShouldMatchers { - val restartStrategy = new RestartStrategy(new AllForOne(), 3, 1000, Array(classOf[Throwable])) + val restartStrategy = new RestartStrategy(AllForOne(), 3, 1000, Array(classOf[Throwable])) val typedActors = List(createTypedActorProperties("se.scalablesolutions.akka.spring.Foo", "1000")) private def createTypedActorProperties(target: String, timeout: String) : ActorProperties = { diff --git a/akka-typed-actor/src/main/scala/actor/TypedActor.scala b/akka-typed-actor/src/main/scala/actor/TypedActor.scala index 8b9cc2034a..12c3c4d9de 100644 --- a/akka-typed-actor/src/main/scala/actor/TypedActor.scala +++ b/akka-typed-actor/src/main/scala/actor/TypedActor.scala @@ -7,7 +7,7 @@ package se.scalablesolutions.akka.actor import Actor._ import se.scalablesolutions.akka.config.FaultHandlingStrategy import se.scalablesolutions.akka.dispatch.{MessageDispatcher, Future, CompletableFuture, Dispatchers} -import se.scalablesolutions.akka.config.ScalaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import se.scalablesolutions.akka.util._ import ReflectiveAccess._ diff --git a/akka-typed-actor/src/main/scala/config/TypedActorConfigurator.scala b/akka-typed-actor/src/main/scala/config/TypedActorConfigurator.scala index d639d21f5f..995640f979 100644 --- a/akka-typed-actor/src/main/scala/config/TypedActorConfigurator.scala +++ b/akka-typed-actor/src/main/scala/config/TypedActorConfigurator.scala @@ -4,7 +4,7 @@ package se.scalablesolutions.akka.config -import JavaConfig._ +import Supervision._ import java.util.{List => JList} import java.util.{ArrayList} @@ -43,10 +43,10 @@ class TypedActorConfigurator { */ def getInstance[T](clazz: Class[T]): T = INSTANCE.getInstance(clazz).head - def configure(restartStrategy: RestartStrategy, components: Array[Component]): TypedActorConfigurator = { + def configure(restartStrategy: RestartStrategy, components: Array[SuperviseTypedActor]): TypedActorConfigurator = { INSTANCE.configure( - restartStrategy.transform, - components.toList.asInstanceOf[scala.List[Component]].map(_.transform)) + restartStrategy, + components.toList.asInstanceOf[scala.List[SuperviseTypedActor]]) this } diff --git a/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala b/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala index 5ca249a3ec..4cd71ab060 100644 --- a/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala +++ b/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala @@ -5,7 +5,7 @@ package se.scalablesolutions.akka.config import se.scalablesolutions.akka.actor._ -import se.scalablesolutions.akka.config.ScalaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import se.scalablesolutions.akka.util._ import ReflectiveAccess._ @@ -28,11 +28,11 @@ private[akka] class TypedActorGuiceConfigurator extends TypedActorConfiguratorBa private var injector: Injector = _ private var supervisor: Option[Supervisor] = None private var restartStrategy: RestartStrategy = _ - private var components: List[Component] = _ + private var components: List[SuperviseTypedActor] = _ private var supervised: List[Supervise] = Nil private var bindings: List[DependencyBinding] = Nil - private var configRegistry = new HashMap[Class[_], Component] // TODO is configRegistry needed? - private var typedActorRegistry = new HashMap[Class[_], Tuple3[AnyRef, AnyRef, Component]] + private var configRegistry = new HashMap[Class[_], SuperviseTypedActor] // TODO is configRegistry needed? + private var typedActorRegistry = new HashMap[Class[_], Tuple3[AnyRef, AnyRef, SuperviseTypedActor]] private var modules = new java.util.ArrayList[Module] private var methodToUriRegistry = new HashMap[Method, String] @@ -68,10 +68,10 @@ private[akka] class TypedActorGuiceConfigurator extends TypedActorConfiguratorBa else c.target } - override def configure(restartStrategy: RestartStrategy, components: List[Component]): + override def configure(restartStrategy: RestartStrategy, components: List[SuperviseTypedActor]): TypedActorConfiguratorBase = synchronized { this.restartStrategy = restartStrategy - this.components = components.toArray.toList.asInstanceOf[List[Component]] + this.components = components.toArray.toList.asInstanceOf[List[SuperviseTypedActor]] bindings = for (component <- this.components) yield { newDelegatingProxy(component) // if (component.intf.isDefined) newDelegatingProxy(component) @@ -84,7 +84,7 @@ private[akka] class TypedActorGuiceConfigurator extends TypedActorConfiguratorBa } /* - private def newSubclassingProxy(component: Component): DependencyBinding = { + private def newSubclassingProxy(component: SuperviseTypedActor): DependencyBinding = { val targetClass = if (component.target.isInstanceOf[Class[_ <: TypedActor]]) component.target.asInstanceOf[Class[_ <: TypedActor]] else throw new IllegalArgumentException("TypedActor [" + component.target.getName + "] must be a subclass of TypedActor") @@ -101,7 +101,7 @@ private[akka] class TypedActorGuiceConfigurator extends TypedActorConfiguratorBa new DependencyBinding(targetClass, proxy) } */ - private def newDelegatingProxy(component: Component): DependencyBinding = { + private def newDelegatingProxy(component: SuperviseTypedActor): DependencyBinding = { component.target.getConstructor(Array[Class[_]](): _*).setAccessible(true) val interfaceClass = if (component.intf.isDefined) component.intf.get else throw new IllegalActorStateException("No interface for TypedActor specified") @@ -169,8 +169,8 @@ private[akka] class TypedActorGuiceConfigurator extends TypedActorConfiguratorBa def reset = synchronized { modules = new java.util.ArrayList[Module] - configRegistry = new HashMap[Class[_], Component] - typedActorRegistry = new HashMap[Class[_], Tuple3[AnyRef, AnyRef, Component]] + configRegistry = new HashMap[Class[_], SuperviseTypedActor] + typedActorRegistry = new HashMap[Class[_], Tuple3[AnyRef, AnyRef, SuperviseTypedActor]] methodToUriRegistry = new HashMap[Method, String] injector = null restartStrategy = null 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 ea5db11531..63c8856075 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 @@ -14,7 +14,7 @@ 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.JavaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import se.scalablesolutions.akka.actor._ @RunWith(classOf[JUnitRunner]) 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 8f80fbcd1b..87dd31fe0f 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 @@ -11,11 +11,9 @@ 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.JavaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import se.scalablesolutions.akka.actor._ +import se.scalablesolutions.akka.config. {Temporary, Config, TypedActorConfigurator} @RunWith(classOf[JUnitRunner]) class RestartTransactionalTypedActorSpec extends @@ -29,15 +27,15 @@ class RestartTransactionalTypedActorSpec extends def before { Config.config conf.configure( - new RestartStrategy(new AllForOne, 3, 5000, List(classOf[Exception]).toArray), + new RestartStrategy(AllForOne, 3, 5000, Array(classOf[Exception])), List( - new Component( + new SuperviseTypedActor( classOf[TransactionalTypedActor], - new Temporary, + Temporary, 10000), - new Component( + new SuperviseTypedActor( classOf[TypedActorFailer], - new Temporary, + Temporary, 10000) ).toArray).supervise } 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 814cd299d9..88eb5f28ef 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 @@ -14,11 +14,10 @@ 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.TypedActorConfigurator -import se.scalablesolutions.akka.config.JavaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import se.scalablesolutions.akka.dispatch._ import se.scalablesolutions.akka.dispatch.FutureTimeoutException +import se.scalablesolutions.akka.config. {Permanent, Config, TypedActorConfigurator} @RunWith(classOf[JUnitRunner]) class TypedActorGuiceConfiguratorSpec extends @@ -36,18 +35,18 @@ class TypedActorGuiceConfiguratorSpec extends conf.addExternalGuiceModule(new AbstractModule { def configure = bind(classOf[Ext]).to(classOf[ExtImpl]).in(Scopes.SINGLETON) }).configure( - new RestartStrategy(new AllForOne, 3, 5000, List(classOf[Exception]).toArray), + new RestartStrategy(AllForOne(), 3, 5000, Array(classOf[Exception])), List( - new Component( + new SuperviseTypedActor( classOf[Foo], classOf[FooImpl], - new Permanent, + Permanent, 1000, dispatcher), - new Component( + new SuperviseTypedActor( classOf[Bar], classOf[BarImpl], - new Permanent, + Permanent, 1000, dispatcher) ).toArray).inject.supervise 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 f2903adf03..f737fb25eb 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 @@ -7,10 +7,10 @@ import org.scalatest.matchers.ShouldMatchers import se.scalablesolutions.akka.actor.TypedActor._ -import se.scalablesolutions.akka.config.{OneForOneStrategy, TypedActorConfigurator} -import se.scalablesolutions.akka.config.JavaConfig._ +import se.scalablesolutions.akka.config.Supervision._ import java.util.concurrent.CountDownLatch +import se.scalablesolutions.akka.config. {OneForOneStrategy, TypedActorConfigurator} /** * @author Martin Krasser @@ -21,9 +21,9 @@ class TypedActorLifecycleSpec extends Spec with ShouldMatchers with BeforeAndAft var conf2: TypedActorConfigurator = _ override protected def beforeAll() = { - val strategy = new RestartStrategy(new AllForOne(), 3, 1000, Array(classOf[Exception])) - val comp3 = new Component(classOf[SamplePojo], classOf[SamplePojoImpl], new Permanent(), 1000) - val comp4 = new Component(classOf[SamplePojo], classOf[SamplePojoImpl], new Temporary(), 1000) + val strategy = new RestartStrategy(AllForOne(), 3, 1000, Array(classOf[Exception])) + val comp3 = new SuperviseTypedActor(classOf[SamplePojo], classOf[SamplePojoImpl], permanent(), 1000) + val comp4 = new SuperviseTypedActor(classOf[SamplePojo], classOf[SamplePojoImpl], temporary(), 1000) conf1 = new TypedActorConfigurator().configure(strategy, Array(comp3)).supervise conf2 = new TypedActorConfigurator().configure(strategy, Array(comp4)).supervise }