Merge with master

This commit is contained in:
Viktor Klang 2012-09-19 00:07:45 +02:00
commit d2c24f3034
79 changed files with 367 additions and 286 deletions

View file

@ -4,7 +4,6 @@
package akka.actor package akka.actor
import language.postfixOps import language.postfixOps
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
import akka.util.Timeout import akka.util.Timeout
import scala.concurrent.{ Await, Future, Promise } import scala.concurrent.{ Await, Future, Promise }
@ -21,6 +20,7 @@ import akka.serialization.JavaSerializer
import akka.actor.TypedActor._ import akka.actor.TypedActor._
import java.lang.IllegalStateException import java.lang.IllegalStateException
import java.util.concurrent.{ TimeoutException, TimeUnit, CountDownLatch } import java.util.concurrent.{ TimeoutException, TimeUnit, CountDownLatch }
import scala.concurrent.util.FiniteDuration
object TypedActorSpec { object TypedActorSpec {
@ -203,10 +203,10 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config)
def newFooBar: Foo = newFooBar(Duration(2, "s")) def newFooBar: Foo = newFooBar(Duration(2, "s"))
def newFooBar(d: Duration): Foo = def newFooBar(d: FiniteDuration): Foo =
TypedActor(system).typedActorOf(TypedProps[Bar](classOf[Foo], classOf[Bar]).withTimeout(Timeout(d))) TypedActor(system).typedActorOf(TypedProps[Bar](classOf[Foo], classOf[Bar]).withTimeout(Timeout(d)))
def newFooBar(dispatcher: String, d: Duration): Foo = def newFooBar(dispatcher: String, d: FiniteDuration): Foo =
TypedActor(system).typedActorOf(TypedProps[Bar](classOf[Foo], classOf[Bar]).withTimeout(Timeout(d)).withDispatcher(dispatcher)) TypedActor(system).typedActorOf(TypedProps[Bar](classOf[Foo], classOf[Bar]).withTimeout(Timeout(d)).withDispatcher(dispatcher))
def newStacked(): Stacked = def newStacked(): Stacked =

View file

@ -68,16 +68,6 @@ class AskSpec extends AkkaSpec {
}.getMessage must be === expectedMsg }.getMessage must be === expectedMsg
} }
"return broken promises on infinite timeout" in {
implicit val timeout = Timeout.never
val echo = system.actorOf(Props(new Actor { def receive = { case x sender ! x } }))
val f = echo ? "foo"
val expectedMsg = "Timeouts to `ask` must be finite. Question not sent to [%s]" format echo
intercept[IllegalArgumentException] {
Await.result(f, remaining)
}.getMessage must be === expectedMsg
}
} }
} }

View file

@ -4,7 +4,6 @@
package akka.routing package akka.routing
import language.postfixOps import language.postfixOps
import akka.actor.Actor import akka.actor.Actor
import akka.testkit._ import akka.testkit._
import akka.actor.Props import akka.actor.Props
@ -15,6 +14,7 @@ import java.util.concurrent.atomic.AtomicInteger
import akka.pattern.ask import akka.pattern.ask
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
import java.util.concurrent.TimeoutException import java.util.concurrent.TimeoutException
import scala.concurrent.util.FiniteDuration
object ResizerSpec { object ResizerSpec {
@ -174,8 +174,8 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
val router = system.actorOf(Props(new Actor { val router = system.actorOf(Props(new Actor {
def receive = { def receive = {
case d: Duration Thread.sleep(d.dilated.toMillis); sender ! "done" case d: FiniteDuration Thread.sleep(d.dilated.toMillis); sender ! "done"
case "echo" sender ! "reply" case "echo" sender ! "reply"
} }
}).withRouter(RoundRobinRouter(resizer = Some(resizer)))) }).withRouter(RoundRobinRouter(resizer = Some(resizer))))
@ -190,7 +190,7 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
routees(router) must be(3) routees(router) must be(3)
def loop(loops: Int, d: Duration) = { def loop(loops: Int, d: FiniteDuration) = {
for (m 0 until loops) router ! d for (m 0 until loops) router ! d
for (m 0 until loops) expectMsg(d * 3, "done") for (m 0 until loops) expectMsg(d * 3, "done")
} }

View file

@ -4,7 +4,6 @@
package akka.actor package akka.actor
import language.existentials import language.existentials
import akka.japi.{ Creator, Option JOption } import akka.japi.{ Creator, Option JOption }
import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy } import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy }
import akka.util.Timeout import akka.util.Timeout
@ -20,6 +19,7 @@ import scala.reflect.ClassTag
import akka.serialization.{ JavaSerializer, SerializationExtension } import akka.serialization.{ JavaSerializer, SerializationExtension }
import java.io.ObjectStreamException import java.io.ObjectStreamException
import scala.util.{ Try, Success, Failure } import scala.util.{ Try, Success, Failure }
import scala.concurrent.util.FiniteDuration
/** /**
* A TypedActorFactory is something that can created TypedActor instances. * A TypedActorFactory is something that can created TypedActor instances.
@ -421,7 +421,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
/** /**
* INTERNAL USE ONLY * INTERNAL USE ONLY
*/ */
private[akka] case class SerializedTypedActorInvocationHandler(val actor: ActorRef, val timeout: Duration) { private[akka] case class SerializedTypedActorInvocationHandler(val actor: ActorRef, val timeout: FiniteDuration) {
@throws(classOf[ObjectStreamException]) private def readResolve(): AnyRef = JavaSerializer.currentSystem.value match { @throws(classOf[ObjectStreamException]) private def readResolve(): AnyRef = JavaSerializer.currentSystem.value match {
case null throw new IllegalStateException("SerializedTypedActorInvocationHandler.readResolve requires that JavaSerializer.currentSystem.value is set to a non-null value") case null throw new IllegalStateException("SerializedTypedActorInvocationHandler.readResolve requires that JavaSerializer.currentSystem.value is set to a non-null value")
case some toTypedActorInvocationHandler(some) case some toTypedActorInvocationHandler(some)

View file

@ -169,7 +169,7 @@ trait Inbox { this: ActorDSL.type ⇒
* this method within an actor!</b> * this method within an actor!</b>
*/ */
def receive(timeout: FiniteDuration = defaultTimeout): Any = { def receive(timeout: FiniteDuration = defaultTimeout): Any = {
implicit val t = Timeout(timeout + extraTime) implicit val t = Timeout((timeout + extraTime).asInstanceOf[FiniteDuration])
Await.result(receiver ? Get(Deadline.now + timeout), Duration.Inf) Await.result(receiver ? Get(Deadline.now + timeout), Duration.Inf)
} }
@ -186,7 +186,7 @@ trait Inbox { this: ActorDSL.type ⇒
* this method within an actor!</b> * this method within an actor!</b>
*/ */
def select[T](timeout: FiniteDuration = defaultTimeout)(predicate: PartialFunction[Any, T]): T = { def select[T](timeout: FiniteDuration = defaultTimeout)(predicate: PartialFunction[Any, T]): T = {
implicit val t = Timeout(timeout + extraTime) implicit val t = Timeout((timeout + extraTime).asInstanceOf[FiniteDuration])
predicate(Await.result(receiver ? Select(Deadline.now + timeout, predicate), Duration.Inf)) predicate(Await.result(receiver ? Select(Deadline.now + timeout, predicate), Duration.Inf))
} }

View file

@ -78,8 +78,7 @@ trait AskSupport {
actorRef.tell(message) actorRef.tell(message)
Future.failed[Any](new AskTimeoutException("Recipient[%s] had already been terminated." format actorRef)) Future.failed[Any](new AskTimeoutException("Recipient[%s] had already been terminated." format actorRef))
case ref: InternalActorRef case ref: InternalActorRef
if (!timeout.duration.isFinite) Future.failed[Any](new IllegalArgumentException("Timeouts to `ask` must be finite. Question not sent to [%s]" format actorRef)) if (timeout.duration.length <= 0) Future.failed[Any](new IllegalArgumentException("Timeout length for an `ask` must be greater or equal to 1. Question not sent to [%s]" format actorRef))
else if (timeout.duration.length <= 0) Future.failed[Any](new IllegalArgumentException("Timeout length for an `ask` must be greater or equal to 1. Question not sent to [%s]" format actorRef))
else { else {
val provider = ref.provider val provider = ref.provider
val a = PromiseActorRef(provider, timeout) val a = PromiseActorRef(provider, timeout)

View file

@ -10,7 +10,7 @@ import akka.util.Unsafe
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import java.util.concurrent.{ Callable, CopyOnWriteArrayList } import java.util.concurrent.{ Callable, CopyOnWriteArrayList }
import scala.concurrent.{ ExecutionContext, Future, Promise, Await } import scala.concurrent.{ ExecutionContext, Future, Promise, Await }
import scala.concurrent.util.{ Duration, Deadline } import scala.concurrent.util.{ FiniteDuration, Deadline }
import scala.concurrent.util.duration._ import scala.concurrent.util.duration._
import scala.util.control.NonFatal import scala.util.control.NonFatal
import scala.util.Success import scala.util.Success
@ -38,8 +38,8 @@ object CircuitBreaker {
* @param callTimeout [[scala.concurrent.util.Duration]] of time after which to consider a call a failure * @param callTimeout [[scala.concurrent.util.Duration]] of time after which to consider a call a failure
* @param resetTimeout [[scala.concurrent.util.Duration]] of time after which to attempt to close the circuit * @param resetTimeout [[scala.concurrent.util.Duration]] of time after which to attempt to close the circuit
*/ */
def apply(scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration): CircuitBreaker = def apply(scheduler: Scheduler, maxFailures: Int, callTimeout: FiniteDuration, resetTimeout: FiniteDuration): CircuitBreaker =
new CircuitBreaker(scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration)(syncExecutionContext) new CircuitBreaker(scheduler, maxFailures, callTimeout, resetTimeout)(syncExecutionContext)
/** /**
* Callbacks run in caller's thread when using withSyncCircuitBreaker, and in same ExecutionContext as the passed * Callbacks run in caller's thread when using withSyncCircuitBreaker, and in same ExecutionContext as the passed
@ -52,8 +52,8 @@ object CircuitBreaker {
* @param callTimeout [[scala.concurrent.util.Duration]] of time after which to consider a call a failure * @param callTimeout [[scala.concurrent.util.Duration]] of time after which to consider a call a failure
* @param resetTimeout [[scala.concurrent.util.Duration]] of time after which to attempt to close the circuit * @param resetTimeout [[scala.concurrent.util.Duration]] of time after which to attempt to close the circuit
*/ */
def create(scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration): CircuitBreaker = def create(scheduler: Scheduler, maxFailures: Int, callTimeout: FiniteDuration, resetTimeout: FiniteDuration): CircuitBreaker =
apply(scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration) apply(scheduler, maxFailures, callTimeout, resetTimeout)
} }
/** /**
@ -76,9 +76,9 @@ object CircuitBreaker {
* @param resetTimeout [[scala.concurrent.util.Duration]] of time after which to attempt to close the circuit * @param resetTimeout [[scala.concurrent.util.Duration]] of time after which to attempt to close the circuit
* @param executor [[scala.concurrent.ExecutionContext]] used for execution of state transition listeners * @param executor [[scala.concurrent.ExecutionContext]] used for execution of state transition listeners
*/ */
class CircuitBreaker(scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration)(implicit executor: ExecutionContext) extends AbstractCircuitBreaker { class CircuitBreaker(scheduler: Scheduler, maxFailures: Int, callTimeout: FiniteDuration, resetTimeout: FiniteDuration)(implicit executor: ExecutionContext) extends AbstractCircuitBreaker {
def this(executor: ExecutionContext, scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration) = { def this(executor: ExecutionContext, scheduler: Scheduler, maxFailures: Int, callTimeout: FiniteDuration, resetTimeout: FiniteDuration) = {
this(scheduler, maxFailures, callTimeout, resetTimeout)(executor) this(scheduler, maxFailures, callTimeout, resetTimeout)(executor)
} }
@ -409,7 +409,7 @@ class CircuitBreaker(scheduler: Scheduler, maxFailures: Int, callTimeout: Durati
* @return Future containing result of protected call * @return Future containing result of protected call
*/ */
override def invoke[T](body: Future[T]): Future[T] = override def invoke[T](body: Future[T]): Future[T] =
if (compareAndSet(true, false)) callThrough(body) else Promise.failed[T](new CircuitBreakerOpenException(Duration.Zero)).future if (compareAndSet(true, false)) callThrough(body) else Promise.failed[T](new CircuitBreakerOpenException(0.seconds)).future
/** /**
* Reset breaker on successful call. * Reset breaker on successful call.
@ -453,7 +453,7 @@ class CircuitBreaker(scheduler: Scheduler, maxFailures: Int, callTimeout: Durati
* @return Future containing result of protected call * @return Future containing result of protected call
*/ */
override def invoke[T](body: Future[T]): Future[T] = override def invoke[T](body: Future[T]): Future[T] =
Promise.failed[T](new CircuitBreakerOpenException(remainingTimeout().timeLeft)).future Promise.failed[T](new CircuitBreakerOpenException(remainingTimeout().timeLeft.asInstanceOf[FiniteDuration])).future
/** /**
* Calculate remaining timeout to inform the caller in case a backoff algorithm is useful * Calculate remaining timeout to inform the caller in case a backoff algorithm is useful
@ -510,6 +510,6 @@ class CircuitBreaker(scheduler: Scheduler, maxFailures: Int, callTimeout: Durati
* @param message Defaults to "Circuit Breaker is open; calls are failing fast" * @param message Defaults to "Circuit Breaker is open; calls are failing fast"
*/ */
class CircuitBreakerOpenException( class CircuitBreakerOpenException(
val remainingDuration: Duration, val remainingDuration: FiniteDuration,
message: String = "Circuit Breaker is open; calls are failing fast") message: String = "Circuit Breaker is open; calls are failing fast")
extends AkkaException(message) with NoStackTrace extends AkkaException(message) with NoStackTrace

View file

@ -10,6 +10,7 @@ import akka.dispatch.{ Unwatch, Watch }
import scala.concurrent.Future import scala.concurrent.Future
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
import scala.util.Success import scala.util.Success
import scala.concurrent.util.FiniteDuration
trait GracefulStopSupport { trait GracefulStopSupport {
/** /**
@ -36,7 +37,7 @@ trait GracefulStopSupport {
* If the target actor isn't terminated within the timeout the [[scala.concurrent.Future]] * If the target actor isn't terminated within the timeout the [[scala.concurrent.Future]]
* is completed with failure [[akka.pattern.AskTimeoutException]]. * is completed with failure [[akka.pattern.AskTimeoutException]].
*/ */
def gracefulStop(target: ActorRef, timeout: Duration)(implicit system: ActorSystem): Future[Boolean] = { def gracefulStop(target: ActorRef, timeout: FiniteDuration)(implicit system: ActorSystem): Future[Boolean] = {
if (target.isTerminated) Future successful true if (target.isTerminated) Future successful true
else system match { else system match {
case e: ExtendedActorSystem case e: ExtendedActorSystem

View file

@ -6,6 +6,7 @@ package akka.pattern
import akka.actor.Scheduler import akka.actor.Scheduler
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext
import java.util.concurrent.Callable import java.util.concurrent.Callable
import scala.concurrent.util.FiniteDuration
object Patterns { object Patterns {
import akka.actor.{ ActorRef, ActorSystem } import akka.actor.{ ActorRef, ActorSystem }
@ -103,7 +104,7 @@ object Patterns {
* If the target actor isn't terminated within the timeout the [[scala.concurrent.Future]] * If the target actor isn't terminated within the timeout the [[scala.concurrent.Future]]
* is completed with failure [[akka.pattern.AskTimeoutException]]. * is completed with failure [[akka.pattern.AskTimeoutException]].
*/ */
def gracefulStop(target: ActorRef, timeout: Duration, system: ActorSystem): Future[java.lang.Boolean] = def gracefulStop(target: ActorRef, timeout: FiniteDuration, system: ActorSystem): Future[java.lang.Boolean] =
scalaGracefulStop(target, timeout)(system).asInstanceOf[Future[java.lang.Boolean]] scalaGracefulStop(target, timeout)(system).asInstanceOf[Future[java.lang.Boolean]]
/** /**

View file

@ -5,7 +5,6 @@ package akka.routing
import language.implicitConversions import language.implicitConversions
import language.postfixOps import language.postfixOps
import akka.actor._ import akka.actor._
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
import scala.concurrent.util.duration._ import scala.concurrent.util.duration._
@ -19,6 +18,7 @@ import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
import scala.annotation.tailrec import scala.annotation.tailrec
import concurrent.ExecutionContext import concurrent.ExecutionContext
import scala.concurrent.util.FiniteDuration
/** /**
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to * A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to
@ -1098,13 +1098,13 @@ object ScatterGatherFirstCompletedRouter {
/** /**
* Creates a new ScatterGatherFirstCompletedRouter, routing to the specified routees, timing out after the specified Duration * Creates a new ScatterGatherFirstCompletedRouter, routing to the specified routees, timing out after the specified Duration
*/ */
def apply(routees: Iterable[ActorRef], within: Duration): ScatterGatherFirstCompletedRouter = def apply(routees: Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter =
new ScatterGatherFirstCompletedRouter(routees = routees map (_.path.toString), within = within) new ScatterGatherFirstCompletedRouter(routees = routees map (_.path.toString), within = within)
/** /**
* Java API to create router with the supplied 'routees' actors. * Java API to create router with the supplied 'routees' actors.
*/ */
def create(routees: java.lang.Iterable[ActorRef], within: Duration): ScatterGatherFirstCompletedRouter = { def create(routees: java.lang.Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter = {
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
apply(routees.asScala, within) apply(routees.asScala, within)
} }
@ -1153,7 +1153,7 @@ object ScatterGatherFirstCompletedRouter {
* using `actorFor` in [[akka.actor.ActorRefProvider]] * using `actorFor` in [[akka.actor.ActorRefProvider]]
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, within: Duration, case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, within: FiniteDuration,
override val resizer: Option[Resizer] = None, override val resizer: Option[Resizer] = None,
val routerDispatcher: String = Dispatchers.DefaultDispatcherId, val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
@ -1166,7 +1166,7 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It
* Constructor that sets nrOfInstances to be created. * Constructor that sets nrOfInstances to be created.
* Java API * Java API
*/ */
def this(nr: Int, w: Duration) = this(nrOfInstances = nr, within = w) def this(nr: Int, w: FiniteDuration) = this(nrOfInstances = nr, within = w)
/** /**
* Constructor that sets the routees to be used. * Constructor that sets the routees to be used.
@ -1174,14 +1174,14 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It
* @param routeePaths string representation of the actor paths of the routees that will be looked up * @param routeePaths string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]] * using `actorFor` in [[akka.actor.ActorRefProvider]]
*/ */
def this(routeePaths: java.lang.Iterable[String], w: Duration) = def this(routeePaths: java.lang.Iterable[String], w: FiniteDuration) =
this(routees = iterableAsScalaIterable(routeePaths), within = w) this(routees = iterableAsScalaIterable(routeePaths), within = w)
/** /**
* Constructor that sets the resizer to be used. * Constructor that sets the resizer to be used.
* Java API * Java API
*/ */
def this(resizer: Resizer, w: Duration) = this(resizer = Some(resizer), within = w) def this(resizer: Resizer, w: FiniteDuration) = this(resizer = Some(resizer), within = w)
/** /**
* Java API for setting routerDispatcher * Java API for setting routerDispatcher
@ -1211,7 +1211,7 @@ trait ScatterGatherFirstCompletedLike { this: RouterConfig ⇒
def routees: Iterable[String] def routees: Iterable[String]
def within: Duration def within: FiniteDuration
def createRoute(routeeProvider: RouteeProvider): Route = { def createRoute(routeeProvider: RouteeProvider): Route = {
if (resizer.isEmpty) { if (resizer.isEmpty) {

View file

@ -8,10 +8,10 @@ import language.implicitConversions
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import java.lang.{ Double JDouble } import java.lang.{ Double JDouble }
import scala.concurrent.util.Duration import scala.concurrent.util.{ Duration, FiniteDuration }
@SerialVersionUID(1L) @SerialVersionUID(1L)
case class Timeout(duration: Duration) { case class Timeout(duration: FiniteDuration) {
def this(timeout: Long) = this(Duration(timeout, TimeUnit.MILLISECONDS)) def this(timeout: Long) = this(Duration(timeout, TimeUnit.MILLISECONDS))
def this(length: Long, unit: TimeUnit) = this(Duration(length, unit)) def this(length: Long, unit: TimeUnit) = this(Duration(length, unit))
} }
@ -26,17 +26,10 @@ object Timeout {
*/ */
val zero: Timeout = new Timeout(Duration.Zero) val zero: Timeout = new Timeout(Duration.Zero)
/**
* A Timeout with infinite duration. Will never timeout. Use extreme caution with this
* as it may cause memory leaks, blocked threads, or may not even be supported by
* the receiver, which would result in an exception.
*/
val never: Timeout = new Timeout(Duration.Inf)
def apply(timeout: Long): Timeout = new Timeout(timeout) def apply(timeout: Long): Timeout = new Timeout(timeout)
def apply(length: Long, unit: TimeUnit): Timeout = new Timeout(length, unit) def apply(length: Long, unit: TimeUnit): Timeout = new Timeout(length, unit)
implicit def durationToTimeout(duration: Duration): Timeout = new Timeout(duration) implicit def durationToTimeout(duration: FiniteDuration): Timeout = new Timeout(duration)
implicit def intToTimeout(timeout: Int): Timeout = new Timeout(timeout) implicit def intToTimeout(timeout: Int): Timeout = new Timeout(timeout)
implicit def longToTimeout(timeout: Long): Timeout = new Timeout(timeout) implicit def longToTimeout(timeout: Long): Timeout = new Timeout(timeout)
} }

View file

@ -10,7 +10,7 @@ import akka.pattern.ask
import akka.util.Timeout import akka.util.Timeout
import scala.concurrent.stm._ import scala.concurrent.stm._
import scala.concurrent.{ ExecutionContext, Future, Promise, Await } import scala.concurrent.{ ExecutionContext, Future, Promise, Await }
import scala.concurrent.util.Duration import scala.concurrent.util.{ FiniteDuration, Duration }
/** /**
* Used internally to send functions. * Used internally to send functions.
@ -240,7 +240,7 @@ class Agent[T](initialValue: T, refFactory: ActorRefFactory, system: ActorSystem
* Dispatch a function to update the internal state, and return a Future where that new state can be obtained * Dispatch a function to update the internal state, and return a Future where that new state can be obtained
* within the given timeout * within the given timeout
*/ */
def alter(f: JFunc[T, T], timeout: Duration): Future[T] = alter(x f(x))(timeout) def alter(f: JFunc[T, T], timeout: FiniteDuration): Future[T] = alter(x f(x))(timeout)
/** /**
* Java API: * Java API:
@ -259,7 +259,7 @@ class Agent[T](initialValue: T, refFactory: ActorRefFactory, system: ActorSystem
* or blocking operations. Dispatches using either `alterOff` or `alter` will * or blocking operations. Dispatches using either `alterOff` or `alter` will
* still be executed in order. * still be executed in order.
*/ */
def alterOff(f: JFunc[T, T], timeout: Duration, ec: ExecutionContext): Unit = alterOff(x f(x))(Timeout(timeout), ec) def alterOff(f: JFunc[T, T], timeout: FiniteDuration, ec: ExecutionContext): Unit = alterOff(x f(x))(Timeout(timeout), ec)
/** /**
* Java API: * Java API:

View file

@ -10,6 +10,7 @@ import akka.actor.{ ActorSystem, Props, ActorRef }
import akka.pattern._ import akka.pattern._
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
import concurrent.{ ExecutionContext, Future } import concurrent.{ ExecutionContext, Future }
import scala.concurrent.util.FiniteDuration
/** /**
* Activation trait that can be used to wait on activation or de-activation of Camel endpoints. * Activation trait that can be used to wait on activation or de-activation of Camel endpoints.
@ -27,7 +28,7 @@ trait Activation {
* @param endpoint the endpoint to be activated * @param endpoint the endpoint to be activated
* @param timeout the timeout for the Future * @param timeout the timeout for the Future
*/ */
def activationFutureFor(endpoint: ActorRef)(implicit timeout: Duration, executor: ExecutionContext): Future[ActorRef] = def activationFutureFor(endpoint: ActorRef)(implicit timeout: FiniteDuration, executor: ExecutionContext): Future[ActorRef] =
(activationTracker.ask(AwaitActivation(endpoint))(Timeout(timeout))).map[ActorRef]({ (activationTracker.ask(AwaitActivation(endpoint))(Timeout(timeout))).map[ActorRef]({
case EndpointActivated(`endpoint`) endpoint case EndpointActivated(`endpoint`) endpoint
case EndpointFailedToActivate(`endpoint`, cause) throw cause case EndpointFailedToActivate(`endpoint`, cause) throw cause
@ -40,7 +41,7 @@ trait Activation {
* @param endpoint the endpoint to be deactivated * @param endpoint the endpoint to be deactivated
* @param timeout the timeout of the Future * @param timeout the timeout of the Future
*/ */
def deactivationFutureFor(endpoint: ActorRef)(implicit timeout: Duration, executor: ExecutionContext): Future[ActorRef] = def deactivationFutureFor(endpoint: ActorRef)(implicit timeout: FiniteDuration, executor: ExecutionContext): Future[ActorRef] =
(activationTracker.ask(AwaitDeActivation(endpoint))(Timeout(timeout))).map[ActorRef]({ (activationTracker.ask(AwaitDeActivation(endpoint))(Timeout(timeout))).map[ActorRef]({
case EndpointDeActivated(`endpoint`) endpoint case EndpointDeActivated(`endpoint`) endpoint
case EndpointFailedToDeActivate(`endpoint`, cause) throw cause case EndpointFailedToDeActivate(`endpoint`, cause) throw cause

View file

@ -5,12 +5,12 @@
package akka.camel package akka.camel
import language.postfixOps import language.postfixOps
import internal.component.DurationTypeConverter import internal.component.DurationTypeConverter
import org.apache.camel.model.{ RouteDefinition, ProcessorDefinition } import org.apache.camel.model.{ RouteDefinition, ProcessorDefinition }
import akka.actor._ import akka.actor._
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
import scala.concurrent.util.duration._ import scala.concurrent.util.duration._
import scala.concurrent.util.FiniteDuration
/** /**
* Mixed in by Actor implementations that consume message from Camel endpoints. * Mixed in by Actor implementations that consume message from Camel endpoints.
@ -41,7 +41,7 @@ trait ConsumerConfig { this: CamelSupport ⇒
/** /**
* How long the actor should wait for activation before it fails. * How long the actor should wait for activation before it fails.
*/ */
def activationTimeout: Duration = camel.settings.activationTimeout def activationTimeout: FiniteDuration = camel.settings.activationTimeout
/** /**
* When endpoint is out-capable (can produce responses) replyTimeout is the maximum time * When endpoint is out-capable (can produce responses) replyTimeout is the maximum time

View file

@ -7,9 +7,7 @@ package akka.camel.internal
import akka.camel._ import akka.camel._
import component.CamelPath import component.CamelPath
import java.io.InputStream import java.io.InputStream
import org.apache.camel.builder.RouteBuilder import org.apache.camel.builder.RouteBuilder
import akka.actor._ import akka.actor._
import collection.mutable import collection.mutable
import org.apache.camel.model.RouteDefinition import org.apache.camel.model.RouteDefinition
@ -17,6 +15,7 @@ import org.apache.camel.CamelContext
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
import concurrent.Await import concurrent.Await
import akka.util.Timeout import akka.util.Timeout
import scala.concurrent.util.FiniteDuration
/** /**
* For internal use only. * For internal use only.
@ -38,7 +37,7 @@ private[camel] trait ConsumerRegistry { this: Activation ⇒
* @param activationTimeout the timeout for activation * @param activationTimeout the timeout for activation
* @return the actorRef to the consumer * @return the actorRef to the consumer
*/ */
private[camel] def registerConsumer(endpointUri: String, consumer: Consumer, activationTimeout: Duration) = { private[camel] def registerConsumer(endpointUri: String, consumer: Consumer, activationTimeout: FiniteDuration) = {
idempotentRegistry ! RegisterConsumer(endpointUri, consumer.self, consumer) idempotentRegistry ! RegisterConsumer(endpointUri, consumer.self, consumer)
Await.result(activationFutureFor(consumer.self)(activationTimeout, consumer.context.dispatcher), activationTimeout) Await.result(activationFutureFor(consumer.self)(activationTimeout, consumer.context.dispatcher), activationTimeout)
} }

View file

@ -8,8 +8,8 @@ import akka.event.Logging
import akka.camel.{ CamelSettings, Camel } import akka.camel.{ CamelSettings, Camel }
import scala.util.control.NonFatal import scala.util.control.NonFatal
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
import org.apache.camel.{ ProducerTemplate, CamelContext } import org.apache.camel.{ ProducerTemplate, CamelContext }
import scala.concurrent.util.FiniteDuration
/** /**
* For internal use only. * For internal use only.
@ -32,7 +32,7 @@ private[camel] class DefaultCamel(val system: ActorSystem) extends Camel {
ctx.setName(system.name) ctx.setName(system.name)
ctx.setStreamCaching(true) ctx.setStreamCaching(true)
ctx.addComponent("akka", new ActorComponent(this, system)) ctx.addComponent("akka", new ActorComponent(this, system))
ctx.getTypeConverterRegistry.addTypeConverter(classOf[Duration], classOf[String], DurationTypeConverter) ctx.getTypeConverterRegistry.addTypeConverter(classOf[FiniteDuration], classOf[String], DurationTypeConverter)
ctx ctx
} }

View file

@ -5,15 +5,11 @@
package akka.camel.internal.component package akka.camel.internal.component
import language.postfixOps import language.postfixOps
import java.util.{ Map JMap } import java.util.{ Map JMap }
import org.apache.camel._ import org.apache.camel._
import org.apache.camel.impl.{ DefaultProducer, DefaultEndpoint, DefaultComponent } import org.apache.camel.impl.{ DefaultProducer, DefaultEndpoint, DefaultComponent }
import akka.actor._ import akka.actor._
import akka.pattern._ import akka.pattern._
import scala.reflect.BeanProperty import scala.reflect.BeanProperty
import scala.concurrent.util.duration._ import scala.concurrent.util.duration._
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
@ -25,6 +21,7 @@ import akka.camel.internal.CamelExchangeAdapter
import akka.camel.{ ActorNotRegisteredException, Camel, Ack, FailureResult, CamelMessage } import akka.camel.{ ActorNotRegisteredException, Camel, Ack, FailureResult, CamelMessage }
import support.TypeConverterSupport import support.TypeConverterSupport
import scala.util.{ Failure, Success, Try } import scala.util.{ Failure, Success, Try }
import scala.concurrent.util.FiniteDuration
/** /**
* For internal use only. * For internal use only.
@ -98,7 +95,7 @@ private[camel] trait ActorEndpointConfig {
def path: ActorEndpointPath def path: ActorEndpointPath
def camel: Camel def camel: Camel
@BeanProperty var replyTimeout: Duration = camel.settings.replyTimeout @BeanProperty var replyTimeout: FiniteDuration = camel.settings.replyTimeout
@BeanProperty var autoAck: Boolean = camel.settings.autoAck @BeanProperty var autoAck: Boolean = camel.settings.autoAck
} }

View file

@ -4,20 +4,22 @@
package akka.camel; package akka.camel;
import akka.actor.ActorRef; import static org.junit.Assert.assertEquals;
import akka.actor.ActorSystem;
import akka.actor.Props; import java.util.concurrent.TimeUnit;
import akka.testkit.JavaTestKit;
import org.junit.AfterClass;
import org.junit.Test;
import scala.concurrent.Await; import scala.concurrent.Await;
import scala.concurrent.ExecutionContext; import scala.concurrent.ExecutionContext;
import scala.concurrent.util.Duration; import scala.concurrent.util.Duration;
import org.junit.AfterClass; import scala.concurrent.util.FiniteDuration;
import org.junit.Test; import akka.actor.ActorRef;
import java.util.concurrent.TimeUnit; import akka.actor.ActorSystem;
import akka.actor.Props;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
import akka.testkit.JavaTestKit.EventFilter; import akka.testkit.JavaTestKit;
import static org.junit.Assert.assertEquals;
/** /**
@ -37,7 +39,7 @@ public class ConsumerJavaTestBase {
new JavaTestKit(system) {{ new JavaTestKit(system) {{
String result = new EventFilter<String>(Exception.class) { String result = new EventFilter<String>(Exception.class) {
protected String run() { protected String run() {
Duration timeout = Duration.create(1, TimeUnit.SECONDS); FiniteDuration timeout = Duration.create(1, TimeUnit.SECONDS);
Camel camel = CamelExtension.get(system); Camel camel = CamelExtension.get(system);
ExecutionContext executionContext = system.dispatcher(); ExecutionContext executionContext = system.dispatcher();
try { try {

View file

@ -7,6 +7,7 @@ import akka.camel.javaapi.UntypedProducerActor;
import scala.concurrent.Await; import scala.concurrent.Await;
import scala.concurrent.ExecutionContext; import scala.concurrent.ExecutionContext;
import scala.concurrent.util.Duration; import scala.concurrent.util.Duration;
import scala.concurrent.util.FiniteDuration;
import org.apache.camel.CamelExecutionException; import org.apache.camel.CamelExecutionException;
import org.apache.camel.Exchange; import org.apache.camel.Exchange;
import org.apache.camel.Predicate; import org.apache.camel.Predicate;
@ -59,7 +60,7 @@ public class CustomRouteTestBase {
@Test @Test
public void testCustomConsumerRoute() throws Exception { public void testCustomConsumerRoute() throws Exception {
MockEndpoint mockEndpoint = camel.context().getEndpoint("mock:mockConsumer", MockEndpoint.class); MockEndpoint mockEndpoint = camel.context().getEndpoint("mock:mockConsumer", MockEndpoint.class);
Duration timeout = Duration.create(10, TimeUnit.SECONDS); FiniteDuration timeout = Duration.create(10, TimeUnit.SECONDS);
ExecutionContext executionContext = system.dispatcher(); ExecutionContext executionContext = system.dispatcher();
ActorRef consumer = Await.result( ActorRef consumer = Await.result(
camel.activationFutureFor(system.actorOf(new Props(TestConsumer.class), "testConsumer"), timeout, executionContext), camel.activationFutureFor(system.actorOf(new Props(TestConsumer.class), "testConsumer"), timeout, executionContext),
@ -73,7 +74,7 @@ public class CustomRouteTestBase {
@Test @Test
public void testCustomAckConsumerRoute() throws Exception { public void testCustomAckConsumerRoute() throws Exception {
MockEndpoint mockEndpoint = camel.context().getEndpoint("mock:mockAck", MockEndpoint.class); MockEndpoint mockEndpoint = camel.context().getEndpoint("mock:mockAck", MockEndpoint.class);
Duration timeout = Duration.create(10, TimeUnit.SECONDS); FiniteDuration timeout = Duration.create(10, TimeUnit.SECONDS);
ExecutionContext executionContext = system.dispatcher(); ExecutionContext executionContext = system.dispatcher();
ActorRef consumer = Await.result( ActorRef consumer = Await.result(
camel.activationFutureFor( camel.activationFutureFor(
@ -91,7 +92,7 @@ public class CustomRouteTestBase {
public void testCustomAckConsumerRouteFromUri() throws Exception { public void testCustomAckConsumerRouteFromUri() throws Exception {
MockEndpoint mockEndpoint = camel.context().getEndpoint("mock:mockAckUri", MockEndpoint.class); MockEndpoint mockEndpoint = camel.context().getEndpoint("mock:mockAckUri", MockEndpoint.class);
ExecutionContext executionContext = system.dispatcher(); ExecutionContext executionContext = system.dispatcher();
Duration timeout = Duration.create(10, TimeUnit.SECONDS); FiniteDuration timeout = Duration.create(10, TimeUnit.SECONDS);
ActorRef consumer = Await.result( ActorRef consumer = Await.result(
camel.activationFutureFor(system.actorOf(new Props( new UntypedActorFactory(){ public Actor create() { return new TestAckConsumer("direct:testConsumerAckFromUri","mock:mockAckUri"); } }), "testConsumerAckUri"), camel.activationFutureFor(system.actorOf(new Props( new UntypedActorFactory(){ public Actor create() { return new TestAckConsumer("direct:testConsumerAckFromUri","mock:mockAckUri"); } }), "testConsumerAckUri"),
timeout, executionContext), timeout, executionContext),
@ -104,7 +105,7 @@ public class CustomRouteTestBase {
@Test(expected=CamelExecutionException.class) @Test(expected=CamelExecutionException.class)
public void testCustomTimeoutConsumerRoute() throws Exception { public void testCustomTimeoutConsumerRoute() throws Exception {
Duration timeout = Duration.create(10, TimeUnit.SECONDS); FiniteDuration timeout = Duration.create(10, TimeUnit.SECONDS);
ExecutionContext executionContext = system.dispatcher(); ExecutionContext executionContext = system.dispatcher();
ActorRef consumer = Await.result( ActorRef consumer = Await.result(
camel.activationFutureFor(system.actorOf(new Props( new UntypedActorFactory(){ public Actor create() { return new TestAckConsumer("direct:testConsumerException","mock:mockException"); } }), "testConsumerException"), camel.activationFutureFor(system.actorOf(new Props( new UntypedActorFactory(){ public Actor create() { return new TestAckConsumer("direct:testConsumerException","mock:mockException"); } }), "testConsumerException"),

View file

@ -1,7 +1,6 @@
package akka.camel.internal package akka.camel.internal
import language.postfixOps import language.postfixOps
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import scala.concurrent.util.duration._ import scala.concurrent.util.duration._
import org.scalatest.{ GivenWhenThen, BeforeAndAfterEach, BeforeAndAfterAll, WordSpec } import org.scalatest.{ GivenWhenThen, BeforeAndAfterEach, BeforeAndAfterAll, WordSpec }
@ -9,6 +8,7 @@ import akka.actor.{ Props, ActorSystem }
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
import akka.camel._ import akka.camel._
import akka.testkit.{ TimingTest, TestProbe, TestKit } import akka.testkit.{ TimingTest, TestProbe, TestKit }
import scala.concurrent.util.FiniteDuration
class ActivationTrackerTest extends TestKit(ActorSystem("test")) with WordSpec with MustMatchers with BeforeAndAfterAll with BeforeAndAfterEach with GivenWhenThen { class ActivationTrackerTest extends TestKit(ActorSystem("test")) with WordSpec with MustMatchers with BeforeAndAfterAll with BeforeAndAfterEach with GivenWhenThen {
@ -115,11 +115,11 @@ class ActivationTrackerTest extends TestKit(ActorSystem("test")) with WordSpec w
val probe = TestProbe() val probe = TestProbe()
def awaitActivation() = at.tell(AwaitActivation(actor.ref), probe.ref) def awaitActivation() = at.tell(AwaitActivation(actor.ref), probe.ref)
def awaitDeActivation() = at.tell(AwaitDeActivation(actor.ref), probe.ref) def awaitDeActivation() = at.tell(AwaitDeActivation(actor.ref), probe.ref)
def verifyActivated(timeout: Duration = 50 millis) = within(timeout) { probe.expectMsg(EndpointActivated(actor.ref)) } def verifyActivated(timeout: FiniteDuration = 50 millis) = within(timeout) { probe.expectMsg(EndpointActivated(actor.ref)) }
def verifyDeActivated(timeout: Duration = 50 millis) = within(timeout) { probe.expectMsg(EndpointDeActivated(actor.ref)) } def verifyDeActivated(timeout: FiniteDuration = 50 millis) = within(timeout) { probe.expectMsg(EndpointDeActivated(actor.ref)) }
def verifyFailedToActivate(timeout: Duration = 50 millis) = within(timeout) { probe.expectMsg(EndpointFailedToActivate(actor.ref, cause)) } def verifyFailedToActivate(timeout: FiniteDuration = 50 millis) = within(timeout) { probe.expectMsg(EndpointFailedToActivate(actor.ref, cause)) }
def verifyFailedToDeActivate(timeout: Duration = 50 millis) = within(timeout) { probe.expectMsg(EndpointFailedToDeActivate(actor.ref, cause)) } def verifyFailedToDeActivate(timeout: FiniteDuration = 50 millis) = within(timeout) { probe.expectMsg(EndpointFailedToDeActivate(actor.ref, cause)) }
} }

View file

@ -5,7 +5,6 @@
package akka.camel.internal.component package akka.camel.internal.component
import language.postfixOps import language.postfixOps
import org.scalatest.mock.MockitoSugar import org.scalatest.mock.MockitoSugar
import org.mockito.Matchers.any import org.mockito.Matchers.any
import org.mockito.Mockito._ import org.mockito.Mockito._
@ -27,6 +26,7 @@ import com.typesafe.config.ConfigFactory
import akka.actor.ActorSystem.Settings import akka.actor.ActorSystem.Settings
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.testkit.{ TimingTest, TestKit, TestProbe } import akka.testkit.{ TimingTest, TestKit, TestProbe }
import scala.concurrent.util.FiniteDuration
class ActorProducerTest extends TestKit(ActorSystem("test")) with WordSpec with MustMatchers with ActorProducerFixture { class ActorProducerTest extends TestKit(ActorSystem("test")) with WordSpec with MustMatchers with ActorProducerFixture {
@ -303,7 +303,7 @@ trait ActorProducerFixture extends MockitoSugar with BeforeAndAfterAll with Befo
def msg(s: String) = CamelMessage(s, Map.empty) def msg(s: String) = CamelMessage(s, Map.empty)
def given(actor: ActorRef = probe.ref, outCapable: Boolean = true, autoAck: Boolean = true, replyTimeout: Duration = Int.MaxValue seconds) = { def given(actor: ActorRef = probe.ref, outCapable: Boolean = true, autoAck: Boolean = true, replyTimeout: FiniteDuration = Int.MaxValue seconds) = {
prepareMocks(actor, outCapable = outCapable) prepareMocks(actor, outCapable = outCapable)
new ActorProducer(configure(isAutoAck = autoAck, _replyTimeout = replyTimeout), camel) new ActorProducer(configure(isAutoAck = autoAck, _replyTimeout = replyTimeout), camel)
} }
@ -325,16 +325,16 @@ trait ActorProducerFixture extends MockitoSugar with BeforeAndAfterAll with Befo
callbackReceived.countDown() callbackReceived.countDown()
} }
private[this] def valueWithin(implicit timeout: Duration) = private[this] def valueWithin(implicit timeout: FiniteDuration) =
if (!callbackReceived.await(timeout.toNanos, TimeUnit.NANOSECONDS)) fail("Callback not received!") if (!callbackReceived.await(timeout.toNanos, TimeUnit.NANOSECONDS)) fail("Callback not received!")
else callbackValue.get else callbackValue.get
def expectDoneSyncWithin(implicit timeout: Duration): Unit = if (!valueWithin(timeout)) fail("Expected to be done Synchronously") def expectDoneSyncWithin(implicit timeout: FiniteDuration): Unit = if (!valueWithin(timeout)) fail("Expected to be done Synchronously")
def expectDoneAsyncWithin(implicit timeout: Duration): Unit = if (valueWithin(timeout)) fail("Expected to be done Asynchronously") def expectDoneAsyncWithin(implicit timeout: FiniteDuration): Unit = if (valueWithin(timeout)) fail("Expected to be done Asynchronously")
} }
def configure(endpointUri: String = "test-uri", isAutoAck: Boolean = true, _replyTimeout: Duration = Int.MaxValue seconds) = { def configure(endpointUri: String = "test-uri", isAutoAck: Boolean = true, _replyTimeout: FiniteDuration = Int.MaxValue seconds) = {
val endpoint = new ActorEndpoint(endpointUri, actorComponent, actorEndpointPath, camel) val endpoint = new ActorEndpoint(endpointUri, actorComponent, actorEndpointPath, camel)
endpoint.autoAck = isAutoAck endpoint.autoAck = isAutoAck
endpoint.replyTimeout = _replyTimeout endpoint.replyTimeout = _replyTimeout

View file

@ -15,7 +15,7 @@ import akka.cluster.routing.ClusterRouterConfig
import akka.event.EventStream import akka.event.EventStream
import akka.remote.RemoteActorRefProvider import akka.remote.RemoteActorRefProvider
import akka.remote.RemoteDeployer import akka.remote.RemoteDeployer
import akka.routing.RemoteRouterConfig import akka.remote.routing.RemoteRouterConfig
import akka.cluster.routing.ClusterRouterSettings import akka.cluster.routing.ClusterRouterSettings
class ClusterActorRefProvider( class ClusterActorRefProvider(

View file

@ -28,7 +28,7 @@ import akka.routing.Route
import akka.routing.RouteeProvider import akka.routing.RouteeProvider
import akka.routing.Router import akka.routing.Router
import akka.routing.RouterConfig import akka.routing.RouterConfig
import akka.routing.RemoteRouterConfig import akka.remote.routing.RemoteRouterConfig
import akka.actor.RootActorPath import akka.actor.RootActorPath
import akka.actor.ActorCell import akka.actor.ActorCell
import akka.actor.RelativeActorPath import akka.actor.RelativeActorPath

View file

@ -51,28 +51,7 @@ What do they do?
* Callbacks can be provided for every state entry via `onOpen`, `onClose`, and `onHalfOpen` * Callbacks can be provided for every state entry via `onOpen`, `onClose`, and `onHalfOpen`
* These are executed in the :class:`ExecutionContext` provided. * These are executed in the :class:`ExecutionContext` provided.
.. graphviz:: .. image:: ../images/circuit-breaker-states.png
digraph circuit_breaker {
rankdir = "LR";
size = "6,5";
graph [ bgcolor = "transparent" ]
node [ fontname = "Helvetica",
fontsize = 14,
shape = circle,
color = white,
style = filled ];
edge [ fontname = "Helvetica", fontsize = 12 ]
Closed [ fillcolor = green2 ];
"Half-Open" [fillcolor = yellow2 ];
Open [ fillcolor = red2 ];
Closed -> Closed [ label = "Success" ];
"Half-Open" -> Open [ label = "Trip Breaker" ];
"Half-Open" -> Closed [ label = "Reset Breaker" ];
Closed -> Open [ label = "Trip Breaker" ];
Open -> Open [ label = "Calls failing fast" ];
Open -> "Half-Open" [ label = "Attempt Reset" ];
}
======== ========
Examples Examples

View file

@ -27,7 +27,7 @@ public class DangerousJavaActor extends UntypedActor {
public DangerousJavaActor() { public DangerousJavaActor() {
this.breaker = new CircuitBreaker( this.breaker = new CircuitBreaker(
getContext().dispatcher(), getContext().system().scheduler(), getContext().dispatcher(), getContext().system().scheduler(),
5, Duration.parse("10s"), Duration.parse("1m")) 5, Duration.create(10, "s"), Duration.create(1, "m"))
.onOpen(new Callable<Object>() { .onOpen(new Callable<Object>() {
public Object call() throws Exception { public Object call() throws Exception {
notifyMeOnOpen(); notifyMeOnOpen();

View file

@ -8,7 +8,7 @@ import sys, os
# -- General configuration ----------------------------------------------------- # -- General configuration -----------------------------------------------------
sys.path.append(os.path.abspath('_sphinx/exts')) sys.path.append(os.path.abspath('_sphinx/exts'))
extensions = ['sphinx.ext.todo', 'includecode', 'sphinx.ext.graphviz'] extensions = ['sphinx.ext.todo', 'includecode']
templates_path = ['_templates'] templates_path = ['_templates']
source_suffix = '.rst' source_suffix = '.rst'

View file

@ -18,7 +18,7 @@ http://github.com/typesafehub/sbt-multi-jvm
You can add it as a plugin by adding the following to your project/plugins.sbt:: You can add it as a plugin by adding the following to your project/plugins.sbt::
addSbtPlugin("com.typesafe.sbt" % "sbt-multi-jvm" % "0.2.0") addSbtPlugin("com.typesafe.sbt" % "sbt-multi-jvm" % "0.3.0")
You can then add multi-JVM testing to ``project/Build.scala`` by including the ``MultiJvm`` You can then add multi-JVM testing to ``project/Build.scala`` by including the ``MultiJvm``
settings and config. For example, here is an example of how the akka-remote-tests project adds settings and config. For example, here is an example of how the akka-remote-tests project adds
@ -47,7 +47,7 @@ multi-JVM testing (Simplified for clarity):
) )
) configs (MultiJvm) ) configs (MultiJvm)
lazy val buildSettings = Defaults.defaultSettings ++ SbtMultiJvm.settings ++ Seq( lazy val buildSettings = Defaults.defaultSettings ++ SbtMultiJvm.multiJvmSettings ++ Seq(
organization := "com.typesafe.akka", organization := "com.typesafe.akka",
version := "2.1-SNAPSHOT", version := "2.1-SNAPSHOT",
scalaVersion := "|scalaVersion|", scalaVersion := "|scalaVersion|",

View file

@ -210,7 +210,7 @@ Querying the Logical Actor Hierarchy
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Since the actor system forms a file-system like hierarchy, matching on paths is Since the actor system forms a file-system like hierarchy, matching on paths is
possible in the same was as supported by Unix shells: you may replace (parts possible in the same way as supported by Unix shells: you may replace (parts
of) path element names with wildcards (`«*»` and `«?»`) to formulate a of) path element names with wildcards (`«*»` and `«?»`) to formulate a
selection which may match zero or more actual actors. Because the result is not selection which may match zero or more actual actors. Because the result is not
a single actor reference, it has a different type :class:`ActorSelection` and a single actor reference, it has a different type :class:`ActorSelection` and

View file

@ -45,8 +45,8 @@ To prevent visibility and reordering problems on actors, Akka guarantees the fol
.. note:: .. note::
In layman's terms this means that changes to internal fields of the actor is visible when the next message In layman's terms this means that changes to internal fields of the actor are visible when the next message
is processed by that actor. So fields in your actor does not need to be volatile or equivalent. is processed by that actor. So fields in your actor need not be volatile or equivalent.
Both rules only apply for the same actor instance and are not valid if different actors are used. Both rules only apply for the same actor instance and are not valid if different actors are used.

View file

@ -0,0 +1,19 @@
digraph circuit_breaker {
rankdir = "LR";
size = "6,5";
graph [ bgcolor = "transparent" ]
node [ fontname = "Helvetica",
fontsize = 14,
shape = circle,
color = white,
style = filled ];
edge [ fontname = "Helvetica", fontsize = 12 ]
Closed [ fillcolor = green2 ];
"Half-Open" [fillcolor = yellow2 ];
Open [ fillcolor = red2 ];
Closed -> Closed [ label = "Success" ];
"Half-Open" -> Open [ label = "Trip Breaker" ];
"Half-Open" -> Closed [ label = "Reset Breaker" ];
Closed -> Open [ label = "Trip Breaker" ];
Open -> Open [ label = "Calls failing fast" ];
Open -> "Half-Open" [ label = "Attempt Reset" ];

Binary file not shown.

After

Width:  |  Height:  |  Size: 22 KiB

View file

@ -109,7 +109,7 @@ public class FaultHandlingDocSample {
*/ */
public static class Worker extends UntypedActor { public static class Worker extends UntypedActor {
final LoggingAdapter log = Logging.getLogger(getContext().system(), this); final LoggingAdapter log = Logging.getLogger(getContext().system(), this);
final Timeout askTimeout = new Timeout(Duration.parse("5 seconds")); final Timeout askTimeout = new Timeout(Duration.create(5, "seconds"));
// The sender of the initial Start message will continuously be notified about progress // The sender of the initial Start message will continuously be notified about progress
ActorRef progressListener; ActorRef progressListener;

View file

@ -106,7 +106,7 @@ public class FutureDocTestBase {
ActorRef actor = system.actorOf(new Props(MyActor.class)); ActorRef actor = system.actorOf(new Props(MyActor.class));
String msg = "hello"; String msg = "hello";
//#ask-blocking //#ask-blocking
Timeout timeout = new Timeout(Duration.parse("5 seconds")); Timeout timeout = new Timeout(Duration.create(5, "seconds"));
Future<Object> future = Patterns.ask(actor, msg, timeout); Future<Object> future = Patterns.ask(actor, msg, timeout);
String result = (String) Await.result(future, timeout.duration()); String result = (String) Await.result(future, timeout.duration());
//#ask-blocking //#ask-blocking

View file

@ -72,7 +72,7 @@ public class CustomRouterDocTestBase {
routedActor.tell(RepublicanVote); routedActor.tell(RepublicanVote);
routedActor.tell(DemocratVote); routedActor.tell(DemocratVote);
routedActor.tell(RepublicanVote); routedActor.tell(RepublicanVote);
Timeout timeout = new Timeout(Duration.parse("1 seconds")); Timeout timeout = new Timeout(Duration.create(1, "seconds"));
Future<Object> democratsResult = ask(routedActor, DemocratCountResult, timeout); Future<Object> democratsResult = ask(routedActor, DemocratCountResult, timeout);
Future<Object> republicansResult = ask(routedActor, RepublicanCountResult, timeout); Future<Object> republicansResult = ask(routedActor, RepublicanCountResult, timeout);

View file

@ -53,8 +53,8 @@ public class ParentActor extends UntypedActor {
//#scatterGatherFirstCompletedRouter //#scatterGatherFirstCompletedRouter
ActorRef scatterGatherFirstCompletedRouter = getContext().actorOf( ActorRef scatterGatherFirstCompletedRouter = getContext().actorOf(
new Props(FibonacciActor.class).withRouter(new ScatterGatherFirstCompletedRouter(5, Duration new Props(FibonacciActor.class).withRouter(new ScatterGatherFirstCompletedRouter(5, Duration
.parse("2 seconds"))), "router"); .create(2, "seconds"))), "router");
Timeout timeout = new Timeout(Duration.parse("5 seconds")); Timeout timeout = new Timeout(Duration.create(5, "seconds"));
Future<Object> futureResult = akka.pattern.Patterns.ask(scatterGatherFirstCompletedRouter, Future<Object> futureResult = akka.pattern.Patterns.ask(scatterGatherFirstCompletedRouter,
new FibonacciActor.FibonacciNumber(10), timeout); new FibonacciActor.FibonacciNumber(10), timeout);
int result = (Integer) Await.result(futureResult, timeout.duration()); int result = (Integer) Await.result(futureResult, timeout.duration());

View file

@ -5,7 +5,7 @@ package docs.jrouting;
import akka.routing.RoundRobinRouter; import akka.routing.RoundRobinRouter;
import akka.routing.DefaultResizer; import akka.routing.DefaultResizer;
import akka.routing.RemoteRouterConfig; import akka.remote.routing.RemoteRouterConfig;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.Props; import akka.actor.Props;
import akka.actor.UntypedActor; import akka.actor.UntypedActor;

View file

@ -98,7 +98,7 @@ public class TestKitDocTest {
//#test-within //#test-within
new JavaTestKit(system) {{ new JavaTestKit(system) {{
getRef().tell(42); getRef().tell(42);
new Within(Duration.Zero(), Duration.parse("1 second")) { new Within(Duration.Zero(), Duration.create(1, "second")) {
// do not put code outside this method, will run afterwards // do not put code outside this method, will run afterwards
public void run() { public void run() {
assertEquals((Integer) 42, expectMsgClass(Integer.class)); assertEquals((Integer) 42, expectMsgClass(Integer.class));

View file

@ -25,7 +25,7 @@ object DurableMailboxDocSpec {
val config = """ val config = """
//#dispatcher-config //#dispatcher-config
my-dispatcher { my-dispatcher {
mailbox-type = akka.actor.mailbox.FileBasedMailboxType mailbox-type = akka.actor.mailbox.filebased.FileBasedMailboxType
} }
//#dispatcher-config //#dispatcher-config
""" """

View file

@ -6,7 +6,8 @@
The 2.1 release contains several structural changes that require some The 2.1 release contains several structural changes that require some
simple, mechanical source-level changes in client code. Several things have simple, mechanical source-level changes in client code. Several things have
been moved to Scala standard library, such as ``Future``. been moved to Scala standard library, such as ``Future``, and some package
names have been changed in Remoting and Durable Mailboxes.
When migrating from 1.3.x to 2.1.x you should first follow the instructions for When migrating from 1.3.x to 2.1.x you should first follow the instructions for
migrating `1.3.x to 2.0.x <http://doc.akka.io/docs/akka/2.0.3/project/migration-guide-1.3.x-2.0.x.html>`_. migrating `1.3.x to 2.0.x <http://doc.akka.io/docs/akka/2.0.3/project/migration-guide-1.3.x-2.0.x.html>`_.
@ -239,7 +240,7 @@ If the target actor of ``akka.pattern.gracefulStop`` isn't terminated within the
timeout the ``Future`` is completed with failure ``akka.pattern.AskTimeoutException``. timeout the ``Future`` is completed with failure ``akka.pattern.AskTimeoutException``.
In 2.0 it was ``akka.actor.ActorTimeoutException``. In 2.0 it was ``akka.actor.ActorTimeoutException``.
getInstance for singeltons - Java getInstance for Singletons - Java
==================================== ====================================
v2.0:: v2.0::
@ -358,4 +359,62 @@ v2.1::
else if (requestedCapacity < 0) routeeProvider.removeRoutees( else if (requestedCapacity < 0) routeeProvider.removeRoutees(
-requestedCapacity, stopDelay) -requestedCapacity, stopDelay)
Duration and Timeout
====================
The Duration class in the scala library is an improved version of the previous
:class:`akka.util.Duration`. Among others it keeps the static type of
:class:`FiniteDuration` more consistently, which has been used to tighten APIs.
The advantage is that instead of runtime exceptions youll get compiler errors
telling you if you try to pass a possibly non-finite duration where it does not
belong.
The main source incompatibility is that you may have to change the declared
type of fields from ``Duration`` to ``FiniteDuration`` (factory methods already
return the more precise type wherever possible).
Another change is that ``Duration.parse`` was not accepted by the scala-library
maintainers, use ``Duration.create`` instead.
v2.0::
final Duration d = Duration.parse("1 second");
final Timeout t = new Timeout(d);
v2.1::
final FiniteDuration d = Duration.create("1 second");
final Timeout t = new Timeout(d); // always required finite duration, now also in type
Package Name Changes in Remoting
================================
The package name of all classes in the ``akka-remote.jar`` artifact now starts with ``akka.remote``.
This has been done to enable OSGi bundles that don't have conflicting package names.
Change the following import statements. Please note that the serializers are often referenced from configuration.
================================================ =======================================================
Search Replace with
================================================ =======================================================
``akka.routing.RemoteRouterConfig`` ``akka.remote.routing.RemoteRouterConfig``
``akka.serialization.ProtobufSerializer`` ``akka.remote.serialization.ProtobufSerializer``
``akka.serialization.DaemonMsgCreateSerializer`` ``akka.remote.serialization.DaemonMsgCreateSerializer``
================================================ =======================================================
Package Name Changes in Durable Mailboxes
=========================================
The package name of all classes in the ``akka-file-mailbox.jar`` artifact now starts with ``akka.actor.mailbox.filebased``.
This has been done to enable OSGi bundles that don't have conflicting package names.
Change the following import statements. Please note that the ``FileBasedMailboxType`` is often referenced from configuration.
================================================ =========================================================
Search Replace with
================================================ =========================================================
``akka.actor.mailbox.FileBasedMailboxType`` ``akka.actor.mailbox.filebased.FileBasedMailboxType``
``akka.actor.mailbox.FileBasedMailboxSettings`` ``akka.actor.mailbox.filebased.FileBasedMailboxSettings``
``akka.actor.mailbox.FileBasedMessageQueue`` ``akka.actor.mailbox.filebased.FileBasedMessageQueue``
``akka.actor.mailbox.filequeue.*`` ``akka.actor.mailbox.filebased.filequeue.*``
================================================ =========================================================

View file

@ -51,6 +51,16 @@ be able to handle unknown messages then you need to have a default case as in
the example above. Otherwise an ``akka.actor.UnhandledMessage(message, sender, recipient)`` will be the example above. Otherwise an ``akka.actor.UnhandledMessage(message, sender, recipient)`` will be
published to the ``ActorSystem``'s ``EventStream``. published to the ``ActorSystem``'s ``EventStream``.
The result of the :meth:`receive` method is a partial function object, which is
stored within the actor as its “initial behavior”, see `Become/Unbecome`_ for
further information on changing the behavior of an actor after its
construction.
.. note::
The initial behavior of an Actor is extracted prior to constructor is run,
so if you want to base your initial behavior on member state, you should
use ``become`` in the constructor.
Creating Actors with default constructor Creating Actors with default constructor
---------------------------------------- ----------------------------------------

View file

@ -6,7 +6,7 @@ package docs.routing
import akka.routing.RoundRobinRouter import akka.routing.RoundRobinRouter
import akka.actor.{ ActorRef, Props, Actor, ActorSystem } import akka.actor.{ ActorRef, Props, Actor, ActorSystem }
import akka.routing.DefaultResizer import akka.routing.DefaultResizer
import akka.routing.RemoteRouterConfig import akka.remote.routing.RemoteRouterConfig
case class Message1(nbr: Int) case class Message1(nbr: Int)

View file

@ -104,7 +104,7 @@ package docs.serialization {
actor { actor {
serializers { serializers {
java = "akka.serialization.JavaSerializer" java = "akka.serialization.JavaSerializer"
proto = "akka.serialization.ProtobufSerializer" proto = "akka.remote.serialization.ProtobufSerializer"
myown = "docs.serialization.MyOwnSerializer" myown = "docs.serialization.MyOwnSerializer"
} }
} }
@ -122,7 +122,7 @@ package docs.serialization {
actor { actor {
serializers { serializers {
java = "akka.serialization.JavaSerializer" java = "akka.serialization.JavaSerializer"
proto = "akka.serialization.ProtobufSerializer" proto = "akka.remote.serialization.ProtobufSerializer"
myown = "docs.serialization.MyOwnSerializer" myown = "docs.serialization.MyOwnSerializer"
} }

View file

@ -2,8 +2,9 @@
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.actor.mailbox package akka.actor.mailbox.filebased
import akka.actor.mailbox._
import akka.actor.{ ActorContext, ActorRef, ActorSystem, ExtendedActorSystem } import akka.actor.{ ActorContext, ActorRef, ActorSystem, ExtendedActorSystem }
import akka.event.Logging import akka.event.Logging
import com.typesafe.config.Config import com.typesafe.config.Config

View file

@ -1,8 +1,9 @@
/** /**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.actor.mailbox package akka.actor.mailbox.filebased
import akka.actor.mailbox._
import com.typesafe.config.Config import com.typesafe.config.Config
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
import java.util.concurrent.TimeUnit.MILLISECONDS import java.util.concurrent.TimeUnit.MILLISECONDS

View file

@ -15,7 +15,7 @@
* limitations under the License. * limitations under the License.
*/ */
package akka.actor.mailbox.filequeue package akka.actor.mailbox.filebased.filequeue
import java.io.IOException import java.io.IOException

View file

@ -15,7 +15,7 @@
* limitations under the License. * limitations under the License.
*/ */
package akka.actor.mailbox.filequeue package akka.actor.mailbox.filebased.filequeue
import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicLong

View file

@ -15,7 +15,7 @@
* limitations under the License. * limitations under the License.
*/ */
package akka.actor.mailbox.filequeue package akka.actor.mailbox.filebased.filequeue
import java.io._ import java.io._
import java.nio.{ ByteBuffer, ByteOrder } import java.nio.{ ByteBuffer, ByteOrder }

View file

@ -15,14 +15,14 @@
* limitations under the License. * limitations under the License.
*/ */
package akka.actor.mailbox.filequeue package akka.actor.mailbox.filebased.filequeue
import java.io._ import java.io._
import scala.collection.mutable import scala.collection.mutable
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import akka.actor.mailbox.FileBasedMailboxSettings import akka.actor.mailbox.filebased.FileBasedMailboxSettings
// a config value that's backed by a global setting but may be locally overridden // a config value that's backed by a global setting but may be locally overridden
class OverlaySetting[T](base: T) { class OverlaySetting[T](base: T) {

View file

@ -15,7 +15,7 @@
* limitations under the License. * limitations under the License.
*/ */
package akka.actor.mailbox.filequeue package akka.actor.mailbox.filebased.filequeue
import java.nio.{ ByteBuffer, ByteOrder } import java.nio.{ ByteBuffer, ByteOrder }

View file

@ -15,13 +15,13 @@
* limitations under the License. * limitations under the License.
*/ */
package akka.actor.mailbox.filequeue package akka.actor.mailbox.filebased.filequeue
import java.io.File import java.io.File
import java.util.concurrent.CountDownLatch import java.util.concurrent.CountDownLatch
import scala.collection.mutable import scala.collection.mutable
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.actor.mailbox.FileBasedMailboxSettings import akka.actor.mailbox.filebased.FileBasedMailboxSettings
class InaccessibleQueuePath extends Exception("Inaccessible queue path: Must be a directory and writable") class InaccessibleQueuePath extends Exception("Inaccessible queue path: Must be a directory and writable")

View file

@ -15,13 +15,13 @@
* limitations under the License. * limitations under the License.
*/ */
package akka.actor.mailbox.filequeue.tools package akka.actor.mailbox.filebased.filequeue.tools
import language.reflectiveCalls import language.reflectiveCalls
import java.io.{ FileNotFoundException, IOException } import java.io.{ FileNotFoundException, IOException }
import scala.collection.mutable import scala.collection.mutable
import akka.actor.mailbox.filequeue._ import akka.actor.mailbox.filebased.filequeue._
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.actor.ActorSystem import akka.actor.ActorSystem

View file

@ -15,7 +15,7 @@
* limitations under the License. * limitations under the License.
*/ */
package akka.actor.mailbox.filequeue.tools package akka.actor.mailbox.filebased.filequeue.tools
object Util { object Util {
val KILOBYTE = 1024L val KILOBYTE = 1024L

View file

@ -1,14 +1,15 @@
package akka.actor.mailbox package akka.actor.mailbox.filebased
import language.postfixOps import language.postfixOps
import akka.actor.mailbox._
import org.apache.commons.io.FileUtils import org.apache.commons.io.FileUtils
import akka.dispatch.Mailbox import akka.dispatch.Mailbox
object FileBasedMailboxSpec { object FileBasedMailboxSpec {
val config = """ val config = """
File-dispatcher { File-dispatcher {
mailbox-type = akka.actor.mailbox.FileBasedMailboxType mailbox-type = akka.actor.mailbox.filebased.FileBasedMailboxType
throughput = 1 throughput = 1
file-based.directory-path = "file-based" file-based.directory-path = "file-based"
file-based.circuit-breaker.max-failures = 5 file-based.circuit-breaker.max-failures = 5

View file

@ -4,7 +4,6 @@
package akka.remote.testconductor package akka.remote.testconductor
import language.postfixOps import language.postfixOps
import akka.actor.{ Actor, ActorRef, ActorSystem, LoggingFSM, Props, PoisonPill, Status, Address, Scheduler } import akka.actor.{ Actor, ActorRef, ActorSystem, LoggingFSM, Props, PoisonPill, Status, Address, Scheduler }
import RemoteConnection.getAddrString import RemoteConnection.getAddrString
import scala.concurrent.util.{ Duration, Deadline } import scala.concurrent.util.{ Duration, Deadline }
@ -20,6 +19,7 @@ import akka.event.{ LoggingAdapter, Logging }
import java.net.{ InetSocketAddress, ConnectException } import java.net.{ InetSocketAddress, ConnectException }
import scala.reflect.classTag import scala.reflect.classTag
import concurrent.{ ExecutionContext, Await, Future } import concurrent.{ ExecutionContext, Await, Future }
import scala.concurrent.util.FiniteDuration
/** /**
* The Player is the client component of the * The Player is the client component of the
@ -81,13 +81,13 @@ trait Player { this: TestConductorExt ⇒
system.log.debug("entering barriers " + name.mkString("(", ", ", ")")) system.log.debug("entering barriers " + name.mkString("(", ", ", ")"))
val stop = Deadline.now + timeout.duration val stop = Deadline.now + timeout.duration
name foreach { b name foreach { b
val barrierTimeout = stop.timeLeft val barrierTimeout = stop.timeLeft.asInstanceOf[FiniteDuration]
if (barrierTimeout < Duration.Zero) { if (barrierTimeout < Duration.Zero) {
client ! ToServer(FailBarrier(b)) client ! ToServer(FailBarrier(b))
throw new TimeoutException("Server timed out while waiting for barrier " + b); throw new TimeoutException("Server timed out while waiting for barrier " + b);
} }
try { try {
implicit val timeout = Timeout(barrierTimeout + Settings.QueryTimeout.duration) implicit val timeout = Timeout((barrierTimeout + Settings.QueryTimeout.duration).asInstanceOf[FiniteDuration])
Await.result(client ? ToServer(EnterBarrier(b, Option(barrierTimeout))), Duration.Inf) Await.result(client ? ToServer(EnterBarrier(b, Option(barrierTimeout))), Duration.Inf)
} catch { } catch {
case e: AskTimeoutException case e: AskTimeoutException

View file

@ -13,8 +13,8 @@ akka {
actor { actor {
serializers { serializers {
proto = "akka.serialization.ProtobufSerializer" proto = "akka.remote.serialization.ProtobufSerializer"
daemon-create = "akka.serialization.DaemonMsgCreateSerializer" daemon-create = "akka.remote.serialization.DaemonMsgCreateSerializer"
} }

View file

@ -5,6 +5,7 @@ package akka.remote
import akka.actor._ import akka.actor._
import akka.routing._ import akka.routing._
import akka.remote.routing._
import com.typesafe.config._ import com.typesafe.config._
import akka.ConfigurationException import akka.ConfigurationException

View file

@ -9,7 +9,7 @@ import javax.net.ssl.{ KeyManagerFactory, TrustManager, TrustManagerFactory, SSL
import akka.remote.RemoteTransportException import akka.remote.RemoteTransportException
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import java.io.{ IOException, FileNotFoundException, FileInputStream } import java.io.{ IOException, FileNotFoundException, FileInputStream }
import akka.security.provider.AkkaProvider import akka.remote.security.provider.AkkaProvider
import java.security._ import java.security._
/** /**

View file

@ -1,8 +1,9 @@
/** /**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.routing package akka.remote.routing
import akka.routing.{ Route, Router, RouterConfig, RouteeProvider, Resizer }
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.actor.ActorContext import akka.actor.ActorContext
import akka.actor.ActorRef import akka.actor.ActorRef

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.security.provider package akka.remote.security.provider
import org.uncommons.maths.random.{ AESCounterRNG } import org.uncommons.maths.random.{ AESCounterRNG }
import SeedSize.Seed128 import SeedSize.Seed128

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.security.provider package akka.remote.security.provider
import org.uncommons.maths.random.{ AESCounterRNG, SecureRandomSeedGenerator } import org.uncommons.maths.random.{ AESCounterRNG, SecureRandomSeedGenerator }
import SeedSize.Seed128 import SeedSize.Seed128

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.security.provider package akka.remote.security.provider
import org.uncommons.maths.random.{ AESCounterRNG } import org.uncommons.maths.random.{ AESCounterRNG }
import SeedSize.Seed256 import SeedSize.Seed256

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.security.provider package akka.remote.security.provider
import org.uncommons.maths.random.{ AESCounterRNG, SecureRandomSeedGenerator } import org.uncommons.maths.random.{ AESCounterRNG, SecureRandomSeedGenerator }
import SeedSize.Seed256 import SeedSize.Seed256

View file

@ -1,7 +1,7 @@
/** /**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.security.provider package akka.remote.security.provider
import java.security.{ PrivilegedAction, AccessController, Provider, Security } import java.security.{ PrivilegedAction, AccessController, Provider, Security }

View file

@ -13,7 +13,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
// ============================================================================ // ============================================================================
package akka.security.provider package akka.remote.security.provider
import org.uncommons.maths.random.{ SeedGenerator, SeedException, SecureRandomSeedGenerator, RandomDotOrgSeedGenerator, DevRandomSeedGenerator } import org.uncommons.maths.random.{ SeedGenerator, SeedException, SecureRandomSeedGenerator, RandomDotOrgSeedGenerator, DevRandomSeedGenerator }

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.security.provider package akka.remote.security.provider
/** /**
* Internal API * Internal API

View file

@ -2,8 +2,9 @@
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.serialization package akka.remote.serialization
import akka.serialization.{ Serializer, SerializationExtension }
import java.io.Serializable import java.io.Serializable
import com.google.protobuf.ByteString import com.google.protobuf.ByteString
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }

View file

@ -2,8 +2,9 @@
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.serialization package akka.remote.serialization
import akka.serialization.{ Serializer, Serialization }
import com.google.protobuf.Message import com.google.protobuf.Message
import akka.actor.DynamicAccess import akka.actor.DynamicAccess
import akka.remote.RemoteProtocol.ActorRefProtocol import akka.remote.RemoteProtocol.ActorRefProtocol

View file

@ -1,7 +1,9 @@
// Generated by the protocol buffer compiler. DO NOT EDIT! // Generated by the protocol buffer compiler. DO NOT EDIT!
// source: ProtobufProtocol.proto // source: ProtobufProtocol.proto
package akka.actor; package akka.remote;
import com.google.protobuf.AbstractMessage;
public final class ProtobufProtocol { public final class ProtobufProtocol {
private ProtobufProtocol() {} private ProtobufProtocol() {}
@ -43,12 +45,12 @@ public final class ProtobufProtocol {
public static final com.google.protobuf.Descriptors.Descriptor public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() { getDescriptor() {
return akka.actor.ProtobufProtocol.internal_static_akka_actor_MyMessage_descriptor; return ProtobufProtocol.internal_static_akka_actor_MyMessage_descriptor;
} }
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() { internalGetFieldAccessorTable() {
return akka.actor.ProtobufProtocol.internal_static_akka_actor_MyMessage_fieldAccessorTable; return ProtobufProtocol.internal_static_akka_actor_MyMessage_fieldAccessorTable;
} }
private int bitField0_; private int bitField0_;
@ -175,41 +177,41 @@ public final class ProtobufProtocol {
return super.writeReplace(); return super.writeReplace();
} }
public static akka.actor.ProtobufProtocol.MyMessage parseFrom( public static ProtobufProtocol.MyMessage parseFrom(
com.google.protobuf.ByteString data) com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data).buildParsed(); return newBuilder().mergeFrom(data).buildParsed();
} }
public static akka.actor.ProtobufProtocol.MyMessage parseFrom( public static ProtobufProtocol.MyMessage parseFrom(
com.google.protobuf.ByteString data, com.google.protobuf.ByteString data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data, extensionRegistry) return newBuilder().mergeFrom(data, extensionRegistry)
.buildParsed(); .buildParsed();
} }
public static akka.actor.ProtobufProtocol.MyMessage parseFrom(byte[] data) public static ProtobufProtocol.MyMessage parseFrom(byte[] data)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data).buildParsed(); return newBuilder().mergeFrom(data).buildParsed();
} }
public static akka.actor.ProtobufProtocol.MyMessage parseFrom( public static ProtobufProtocol.MyMessage parseFrom(
byte[] data, byte[] data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data, extensionRegistry) return newBuilder().mergeFrom(data, extensionRegistry)
.buildParsed(); .buildParsed();
} }
public static akka.actor.ProtobufProtocol.MyMessage parseFrom(java.io.InputStream input) public static ProtobufProtocol.MyMessage parseFrom(java.io.InputStream input)
throws java.io.IOException { throws java.io.IOException {
return newBuilder().mergeFrom(input).buildParsed(); return newBuilder().mergeFrom(input).buildParsed();
} }
public static akka.actor.ProtobufProtocol.MyMessage parseFrom( public static ProtobufProtocol.MyMessage parseFrom(
java.io.InputStream input, java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException { throws java.io.IOException {
return newBuilder().mergeFrom(input, extensionRegistry) return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed(); .buildParsed();
} }
public static akka.actor.ProtobufProtocol.MyMessage parseDelimitedFrom(java.io.InputStream input) public static ProtobufProtocol.MyMessage parseDelimitedFrom(java.io.InputStream input)
throws java.io.IOException { throws java.io.IOException {
Builder builder = newBuilder(); Builder builder = newBuilder();
if (builder.mergeDelimitedFrom(input)) { if (builder.mergeDelimitedFrom(input)) {
@ -218,7 +220,7 @@ public final class ProtobufProtocol {
return null; return null;
} }
} }
public static akka.actor.ProtobufProtocol.MyMessage parseDelimitedFrom( public static ProtobufProtocol.MyMessage parseDelimitedFrom(
java.io.InputStream input, java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException { throws java.io.IOException {
@ -229,12 +231,12 @@ public final class ProtobufProtocol {
return null; return null;
} }
} }
public static akka.actor.ProtobufProtocol.MyMessage parseFrom( public static ProtobufProtocol.MyMessage parseFrom(
com.google.protobuf.CodedInputStream input) com.google.protobuf.CodedInputStream input)
throws java.io.IOException { throws java.io.IOException {
return newBuilder().mergeFrom(input).buildParsed(); return newBuilder().mergeFrom(input).buildParsed();
} }
public static akka.actor.ProtobufProtocol.MyMessage parseFrom( public static ProtobufProtocol.MyMessage parseFrom(
com.google.protobuf.CodedInputStream input, com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException { throws java.io.IOException {
@ -244,7 +246,7 @@ public final class ProtobufProtocol {
public static Builder newBuilder() { return Builder.create(); } public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); } public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(akka.actor.ProtobufProtocol.MyMessage prototype) { public static Builder newBuilder(ProtobufProtocol.MyMessage prototype) {
return newBuilder().mergeFrom(prototype); return newBuilder().mergeFrom(prototype);
} }
public Builder toBuilder() { return newBuilder(this); } public Builder toBuilder() { return newBuilder(this); }
@ -257,15 +259,15 @@ public final class ProtobufProtocol {
} }
public static final class Builder extends public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder<Builder> com.google.protobuf.GeneratedMessage.Builder<Builder>
implements akka.actor.ProtobufProtocol.MyMessageOrBuilder { implements ProtobufProtocol.MyMessageOrBuilder {
public static final com.google.protobuf.Descriptors.Descriptor public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() { getDescriptor() {
return akka.actor.ProtobufProtocol.internal_static_akka_actor_MyMessage_descriptor; return ProtobufProtocol.internal_static_akka_actor_MyMessage_descriptor;
} }
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() { internalGetFieldAccessorTable() {
return akka.actor.ProtobufProtocol.internal_static_akka_actor_MyMessage_fieldAccessorTable; return ProtobufProtocol.internal_static_akka_actor_MyMessage_fieldAccessorTable;
} }
// Construct using akka.actor.ProtobufProtocol.MyMessage.newBuilder() // Construct using akka.actor.ProtobufProtocol.MyMessage.newBuilder()
@ -302,33 +304,33 @@ public final class ProtobufProtocol {
public com.google.protobuf.Descriptors.Descriptor public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() { getDescriptorForType() {
return akka.actor.ProtobufProtocol.MyMessage.getDescriptor(); return ProtobufProtocol.MyMessage.getDescriptor();
} }
public akka.actor.ProtobufProtocol.MyMessage getDefaultInstanceForType() { public ProtobufProtocol.MyMessage getDefaultInstanceForType() {
return akka.actor.ProtobufProtocol.MyMessage.getDefaultInstance(); return ProtobufProtocol.MyMessage.getDefaultInstance();
} }
public akka.actor.ProtobufProtocol.MyMessage build() { public ProtobufProtocol.MyMessage build() {
akka.actor.ProtobufProtocol.MyMessage result = buildPartial(); ProtobufProtocol.MyMessage result = buildPartial();
if (!result.isInitialized()) { if (!result.isInitialized()) {
throw newUninitializedMessageException(result); throw AbstractMessage.Builder.newUninitializedMessageException(result);
} }
return result; return result;
} }
private akka.actor.ProtobufProtocol.MyMessage buildParsed() private ProtobufProtocol.MyMessage buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
akka.actor.ProtobufProtocol.MyMessage result = buildPartial(); ProtobufProtocol.MyMessage result = buildPartial();
if (!result.isInitialized()) { if (!result.isInitialized()) {
throw newUninitializedMessageException( throw AbstractMessage.Builder.newUninitializedMessageException(
result).asInvalidProtocolBufferException(); result).asInvalidProtocolBufferException();
} }
return result; return result;
} }
public akka.actor.ProtobufProtocol.MyMessage buildPartial() { public ProtobufProtocol.MyMessage buildPartial() {
akka.actor.ProtobufProtocol.MyMessage result = new akka.actor.ProtobufProtocol.MyMessage(this); ProtobufProtocol.MyMessage result = new ProtobufProtocol.MyMessage(this);
int from_bitField0_ = bitField0_; int from_bitField0_ = bitField0_;
int to_bitField0_ = 0; int to_bitField0_ = 0;
if (((from_bitField0_ & 0x00000001) == 0x00000001)) { if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
@ -349,16 +351,16 @@ public final class ProtobufProtocol {
} }
public Builder mergeFrom(com.google.protobuf.Message other) { public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof akka.actor.ProtobufProtocol.MyMessage) { if (other instanceof ProtobufProtocol.MyMessage) {
return mergeFrom((akka.actor.ProtobufProtocol.MyMessage)other); return mergeFrom((ProtobufProtocol.MyMessage)other);
} else { } else {
super.mergeFrom(other); super.mergeFrom(other);
return this; return this;
} }
} }
public Builder mergeFrom(akka.actor.ProtobufProtocol.MyMessage other) { public Builder mergeFrom(ProtobufProtocol.MyMessage other) {
if (other == akka.actor.ProtobufProtocol.MyMessage.getDefaultInstance()) return this; if (other == ProtobufProtocol.MyMessage.getDefaultInstance()) return this;
if (other.hasId()) { if (other.hasId()) {
setId(other.getId()); setId(other.getId());
} }
@ -550,8 +552,8 @@ public final class ProtobufProtocol {
com.google.protobuf.GeneratedMessage.FieldAccessorTable( com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_akka_actor_MyMessage_descriptor, internal_static_akka_actor_MyMessage_descriptor,
new java.lang.String[] { "Id", "Name", "Status", }, new java.lang.String[] { "Id", "Name", "Status", },
akka.actor.ProtobufProtocol.MyMessage.class, ProtobufProtocol.MyMessage.class,
akka.actor.ProtobufProtocol.MyMessage.Builder.class); ProtobufProtocol.MyMessage.Builder.class);
return null; return null;
} }
}; };

View file

@ -6,6 +6,7 @@ package akka.remote
import akka.testkit._ import akka.testkit._
import akka.routing._ import akka.routing._
import akka.actor._ import akka.actor._
import akka.remote.routing._
import com.typesafe.config._ import com.typesafe.config._
object RemoteRouterSpec { object RemoteRouterSpec {

View file

@ -2,10 +2,11 @@
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.serialization package akka.remote.serialization
import language.postfixOps import language.postfixOps
import akka.serialization.SerializationExtension
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.actor.{ Actor, Address, Props, Deploy, OneForOneStrategy, SupervisorStrategy, FromClassCreator } import akka.actor.{ Actor, Address, Props, Deploy, OneForOneStrategy, SupervisorStrategy, FromClassCreator }

View file

@ -2,11 +2,12 @@
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.serialization package akka.remote.serialization
import akka.serialization.SerializationExtension
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.remote.RemoteProtocol.MessageProtocol import akka.remote.RemoteProtocol.MessageProtocol
import akka.actor.ProtobufProtocol.MyMessage import akka.remote.ProtobufProtocol.MyMessage
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ProtobufSerializerSpec extends AkkaSpec { class ProtobufSerializerSpec extends AkkaSpec {

View file

@ -11,6 +11,7 @@ import akka.event.Logging.LogEvent;
import akka.japi.JavaPartialFunction; import akka.japi.JavaPartialFunction;
import akka.japi.Util; import akka.japi.Util;
import scala.concurrent.util.Duration; import scala.concurrent.util.Duration;
import scala.concurrent.util.FiniteDuration;
/** /**
* Java API for the TestProbe. Proper JavaDocs to come once JavaDoccing is implemented. * Java API for the TestProbe. Proper JavaDocs to come once JavaDoccing is implemented.
@ -30,8 +31,10 @@ public class JavaTestKit {
return p.system(); return p.system();
} }
static public Duration duration(String s) { static public FiniteDuration duration(String s) {
return Duration.parse(s); final Duration ret = Duration.apply(s);
if (ret instanceof FiniteDuration) return (FiniteDuration) ret;
else throw new IllegalArgumentException("duration() is only for finite durations, use Duration.Inf() and friends");
} }
public Duration dilated(Duration d) { public Duration dilated(Duration d) {
@ -58,11 +61,11 @@ public class JavaTestKit {
p.lastMessage().sender().tell(msg, p.ref()); p.lastMessage().sender().tell(msg, p.ref());
} }
public Duration getRemainingTime() { public FiniteDuration getRemainingTime() {
return p.remaining(); return p.remaining();
} }
public Duration getRemainingTimeOr(Duration def) { public FiniteDuration getRemainingTimeOr(FiniteDuration def) {
return p.remainingOr(def); return p.remainingOr(def);
} }
@ -97,7 +100,7 @@ public class JavaTestKit {
public abstract class Within { public abstract class Within {
protected abstract void run(); protected abstract void run();
public Within(Duration max) { public Within(FiniteDuration max) {
p.within(max, new AbstractFunction0<Object>() { p.within(max, new AbstractFunction0<Object>() {
public Object apply() { public Object apply() {
run(); run();
@ -106,7 +109,7 @@ public class JavaTestKit {
}); });
} }
public Within(Duration min, Duration max) { public Within(FiniteDuration min, FiniteDuration max) {
p.within(min, max, new AbstractFunction0<Object>() { p.within(min, max, new AbstractFunction0<Object>() {
public Object apply() { public Object apply() {
run(); run();
@ -168,7 +171,7 @@ public class JavaTestKit {
return p.expectMsg(msg); return p.expectMsg(msg);
} }
public <T> T expectMsgEquals(Duration max, T msg) { public <T> T expectMsgEquals(FiniteDuration max, T msg) {
return p.expectMsg(max, msg); return p.expectMsg(max, msg);
} }
@ -176,7 +179,7 @@ public class JavaTestKit {
return p.expectMsgClass(clazz); return p.expectMsgClass(clazz);
} }
public <T> T expectMsgClass(Duration max, Class<T> clazz) { public <T> T expectMsgClass(FiniteDuration max, Class<T> clazz) {
return p.expectMsgClass(max, clazz); return p.expectMsgClass(max, clazz);
} }
@ -184,7 +187,7 @@ public class JavaTestKit {
return p.expectMsgAnyOf(Util.arrayToSeq(msgs)); return p.expectMsgAnyOf(Util.arrayToSeq(msgs));
} }
public Object expectMsgAnyOf(Duration max, Object... msgs) { public Object expectMsgAnyOf(FiniteDuration max, Object... msgs) {
return p.expectMsgAnyOf(max, Util.arrayToSeq(msgs)); return p.expectMsgAnyOf(max, Util.arrayToSeq(msgs));
} }
@ -193,7 +196,7 @@ public class JavaTestKit {
Util.classTag(Object.class)); Util.classTag(Object.class));
} }
public Object[] expectMsgAllOf(Duration max, Object... msgs) { public Object[] expectMsgAllOf(FiniteDuration max, Object... msgs) {
return (Object[]) p.expectMsgAllOf(max, Util.arrayToSeq(msgs)).toArray( return (Object[]) p.expectMsgAllOf(max, Util.arrayToSeq(msgs)).toArray(
Util.classTag(Object.class)); Util.classTag(Object.class));
} }
@ -204,7 +207,7 @@ public class JavaTestKit {
return (T) result; return (T) result;
} }
public Object expectMsgAnyClassOf(Duration max, Class<?>... classes) { public Object expectMsgAnyClassOf(FiniteDuration max, Class<?>... classes) {
return p.expectMsgAnyClassOf(max, Util.arrayToSeq(classes)); return p.expectMsgAnyClassOf(max, Util.arrayToSeq(classes));
} }
@ -212,7 +215,7 @@ public class JavaTestKit {
p.expectNoMsg(); p.expectNoMsg();
} }
public void expectNoMsg(Duration max) { public void expectNoMsg(FiniteDuration max) {
p.expectNoMsg(max); p.expectNoMsg(max);
} }

View file

@ -7,6 +7,7 @@ package akka.testkit
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
import java.util.concurrent.{ CyclicBarrier, TimeUnit, TimeoutException } import java.util.concurrent.{ CyclicBarrier, TimeUnit, TimeoutException }
import akka.actor.ActorSystem import akka.actor.ActorSystem
import scala.concurrent.util.FiniteDuration
class TestBarrierTimeoutException(message: String) extends RuntimeException(message) class TestBarrierTimeoutException(message: String) extends RuntimeException(message)
@ -27,7 +28,7 @@ class TestBarrier(count: Int) {
def await()(implicit system: ActorSystem): Unit = await(TestBarrier.DefaultTimeout) def await()(implicit system: ActorSystem): Unit = await(TestBarrier.DefaultTimeout)
def await(timeout: Duration)(implicit system: ActorSystem) { def await(timeout: FiniteDuration)(implicit system: ActorSystem) {
try { try {
barrier.await(timeout.dilated.toNanos, TimeUnit.NANOSECONDS) barrier.await(timeout.dilated.toNanos, TimeUnit.NANOSECONDS)
} catch { } catch {

View file

@ -7,7 +7,7 @@ import language.postfixOps
import akka.actor._ import akka.actor._
import akka.actor.Actor._ import akka.actor.Actor._
import scala.concurrent.util.Duration import scala.concurrent.util.{ Duration, FiniteDuration }
import scala.concurrent.util.duration._ import scala.concurrent.util.duration._
import java.util.concurrent.{ BlockingDeque, LinkedBlockingDeque, TimeUnit, atomic } import java.util.concurrent.{ BlockingDeque, LinkedBlockingDeque, TimeUnit, atomic }
import atomic.AtomicInteger import atomic.AtomicInteger
@ -176,20 +176,30 @@ trait TestKitBase {
/** /**
* Obtain current time (`System.nanoTime`) as Duration. * Obtain current time (`System.nanoTime`) as Duration.
*/ */
def now: Duration = System.nanoTime.nanos def now: FiniteDuration = System.nanoTime.nanos
/** /**
* Obtain time remaining for execution of the innermost enclosing `within` * Obtain time remaining for execution of the innermost enclosing `within`
* block or missing that it returns the properly dilated default for this * block or missing that it returns the properly dilated default for this
* case from settings (key "akka.test.single-expect-default"). * case from settings (key "akka.test.single-expect-default").
*/ */
def remaining: Duration = remainingOr(testKitSettings.SingleExpectDefaultTimeout.dilated) def remaining: FiniteDuration = remainingOr(testKitSettings.SingleExpectDefaultTimeout.dilated)
/** /**
* Obtain time remaining for execution of the innermost enclosing `within` * Obtain time remaining for execution of the innermost enclosing `within`
* block or missing that it returns the given duration. * block or missing that it returns the given duration.
*/ */
def remainingOr(duration: Duration): Duration = if (end == Duration.Undefined) duration else end - now def remainingOr(duration: FiniteDuration): FiniteDuration = end match {
case x if x eq Duration.Undefined duration
case x if !x.isFinite throw new IllegalArgumentException("`end` cannot be infinite")
case f: FiniteDuration (end - now).asInstanceOf[FiniteDuration] // RK FIXME after next Scala milestone
}
private def remainingOrDilated(max: Duration): FiniteDuration = max match {
case x if x eq Duration.Undefined remaining
case x if !x.isFinite throw new IllegalArgumentException("max duration cannot be infinite")
case f: FiniteDuration f.dilated
}
/** /**
* Query queue status. * Query queue status.
@ -207,7 +217,7 @@ trait TestKitBase {
* which uses the configuration entry "akka.test.timefactor". * which uses the configuration entry "akka.test.timefactor".
*/ */
def awaitCond(p: Boolean, max: Duration = Duration.Undefined, interval: Duration = 100.millis) { def awaitCond(p: Boolean, max: Duration = Duration.Undefined, interval: Duration = 100.millis) {
val _max = if (max eq Duration.Undefined) remaining else max.dilated val _max = remainingOrDilated(max)
val stop = now + _max val stop = now + _max
@tailrec @tailrec
@ -238,7 +248,7 @@ trait TestKitBase {
* } * }
* </pre> * </pre>
*/ */
def within[T](min: Duration, max: Duration)(f: T): T = { def within[T](min: FiniteDuration, max: FiniteDuration)(f: T): T = {
val _max = max.dilated val _max = max.dilated
val start = now val start = now
val rem = if (end == Duration.Undefined) Duration.Inf else end - start val rem = if (end == Duration.Undefined) Duration.Inf else end - start
@ -264,7 +274,7 @@ trait TestKitBase {
/** /**
* Same as calling `within(0 seconds, max)(f)`. * Same as calling `within(0 seconds, max)(f)`.
*/ */
def within[T](max: Duration)(f: T): T = within(0 seconds, max)(f) def within[T](max: FiniteDuration)(f: T): T = within(0 seconds, max)(f)
/** /**
* Same as `expectMsg(remaining, obj)`, but correctly treating the timeFactor. * Same as `expectMsg(remaining, obj)`, but correctly treating the timeFactor.
@ -278,7 +288,7 @@ trait TestKitBase {
* *
* @return the received object * @return the received object
*/ */
def expectMsg[T](max: Duration, obj: T): T = expectMsg_internal(max.dilated, obj) def expectMsg[T](max: FiniteDuration, obj: T): T = expectMsg_internal(max.dilated, obj)
private def expectMsg_internal[T](max: Duration, obj: T): T = { private def expectMsg_internal[T](max: Duration, obj: T): T = {
val o = receiveOne(max) val o = receiveOne(max)
@ -298,7 +308,7 @@ trait TestKitBase {
* @return the received object as transformed by the partial function * @return the received object as transformed by the partial function
*/ */
def expectMsgPF[T](max: Duration = Duration.Undefined, hint: String = "")(f: PartialFunction[Any, T]): T = { def expectMsgPF[T](max: Duration = Duration.Undefined, hint: String = "")(f: PartialFunction[Any, T]): T = {
val _max = if (max eq Duration.Undefined) remaining else max.dilated val _max = remainingOrDilated(max)
val o = receiveOne(_max) val o = receiveOne(_max)
assert(o ne null, "timeout (" + _max + ") during expectMsg: " + hint) assert(o ne null, "timeout (" + _max + ") during expectMsg: " + hint)
assert(f.isDefinedAt(o), "expected: " + hint + " but got unexpected message " + o) assert(f.isDefinedAt(o), "expected: " + hint + " but got unexpected message " + o)
@ -314,7 +324,7 @@ trait TestKitBase {
* partial function returned true * partial function returned true
*/ */
def fishForMessage(max: Duration = Duration.Undefined, hint: String = "")(f: PartialFunction[Any, Boolean]): Any = { def fishForMessage(max: Duration = Duration.Undefined, hint: String = "")(f: PartialFunction[Any, Boolean]): Any = {
val _max = if (max eq Duration.Undefined) remaining else max.dilated val _max = remainingOrDilated(max)
val end = now + _max val end = now + _max
@tailrec @tailrec
def recv: Any = { def recv: Any = {
@ -338,7 +348,7 @@ trait TestKitBase {
* *
* @return the received object * @return the received object
*/ */
def expectMsgType[T](max: Duration)(implicit t: ClassTag[T]): T = expectMsgClass_internal(max.dilated, t.runtimeClass.asInstanceOf[Class[T]]) def expectMsgType[T](max: FiniteDuration)(implicit t: ClassTag[T]): T = expectMsgClass_internal(max.dilated, t.runtimeClass.asInstanceOf[Class[T]])
/** /**
* Same as `expectMsgClass(remaining, c)`, but correctly treating the timeFactor. * Same as `expectMsgClass(remaining, c)`, but correctly treating the timeFactor.
@ -352,9 +362,9 @@ trait TestKitBase {
* *
* @return the received object * @return the received object
*/ */
def expectMsgClass[C](max: Duration, c: Class[C]): C = expectMsgClass_internal(max.dilated, c) def expectMsgClass[C](max: FiniteDuration, c: Class[C]): C = expectMsgClass_internal(max.dilated, c)
private def expectMsgClass_internal[C](max: Duration, c: Class[C]): C = { private def expectMsgClass_internal[C](max: FiniteDuration, c: Class[C]): C = {
val o = receiveOne(max) val o = receiveOne(max)
assert(o ne null, "timeout (" + max + ") during expectMsgClass waiting for " + c) assert(o ne null, "timeout (" + max + ") during expectMsgClass waiting for " + c)
assert(BoxedType(c) isInstance o, "expected " + c + ", found " + o.getClass) assert(BoxedType(c) isInstance o, "expected " + c + ", found " + o.getClass)
@ -373,9 +383,9 @@ trait TestKitBase {
* *
* @return the received object * @return the received object
*/ */
def expectMsgAnyOf[T](max: Duration, obj: T*): T = expectMsgAnyOf_internal(max.dilated, obj: _*) def expectMsgAnyOf[T](max: FiniteDuration, obj: T*): T = expectMsgAnyOf_internal(max.dilated, obj: _*)
private def expectMsgAnyOf_internal[T](max: Duration, obj: T*): T = { private def expectMsgAnyOf_internal[T](max: FiniteDuration, obj: T*): T = {
val o = receiveOne(max) val o = receiveOne(max)
assert(o ne null, "timeout (" + max + ") during expectMsgAnyOf waiting for " + obj.mkString("(", ", ", ")")) assert(o ne null, "timeout (" + max + ") during expectMsgAnyOf waiting for " + obj.mkString("(", ", ", ")"))
assert(obj exists (_ == o), "found unexpected " + o) assert(obj exists (_ == o), "found unexpected " + o)
@ -394,9 +404,9 @@ trait TestKitBase {
* *
* @return the received object * @return the received object
*/ */
def expectMsgAnyClassOf[C](max: Duration, obj: Class[_ <: C]*): C = expectMsgAnyClassOf_internal(max.dilated, obj: _*) def expectMsgAnyClassOf[C](max: FiniteDuration, obj: Class[_ <: C]*): C = expectMsgAnyClassOf_internal(max.dilated, obj: _*)
private def expectMsgAnyClassOf_internal[C](max: Duration, obj: Class[_ <: C]*): C = { private def expectMsgAnyClassOf_internal[C](max: FiniteDuration, obj: Class[_ <: C]*): C = {
val o = receiveOne(max) val o = receiveOne(max)
assert(o ne null, "timeout (" + max + ") during expectMsgAnyClassOf waiting for " + obj.mkString("(", ", ", ")")) assert(o ne null, "timeout (" + max + ") during expectMsgAnyClassOf waiting for " + obj.mkString("(", ", ", ")"))
assert(obj exists (c BoxedType(c) isInstance o), "found unexpected " + o) assert(obj exists (c BoxedType(c) isInstance o), "found unexpected " + o)
@ -421,9 +431,9 @@ trait TestKitBase {
* expectMsgAllOf(1 second, Result1(), Result2()) * expectMsgAllOf(1 second, Result1(), Result2())
* </pre> * </pre>
*/ */
def expectMsgAllOf[T](max: Duration, obj: T*): Seq[T] = expectMsgAllOf_internal(max.dilated, obj: _*) def expectMsgAllOf[T](max: FiniteDuration, obj: T*): Seq[T] = expectMsgAllOf_internal(max.dilated, obj: _*)
private def expectMsgAllOf_internal[T](max: Duration, obj: T*): Seq[T] = { private def expectMsgAllOf_internal[T](max: FiniteDuration, obj: T*): Seq[T] = {
val recv = receiveN_internal(obj.size, max) val recv = receiveN_internal(obj.size, max)
obj foreach (x assert(recv exists (x == _), "not found " + x)) obj foreach (x assert(recv exists (x == _), "not found " + x))
recv foreach (x assert(obj exists (x == _), "found unexpected " + x)) recv foreach (x assert(obj exists (x == _), "found unexpected " + x))
@ -443,9 +453,9 @@ trait TestKitBase {
* Wait time is bounded by the given duration, with an AssertionFailure * Wait time is bounded by the given duration, with an AssertionFailure
* being thrown in case of timeout. * being thrown in case of timeout.
*/ */
def expectMsgAllClassOf[T](max: Duration, obj: Class[_ <: T]*): Seq[T] = internalExpectMsgAllClassOf(max.dilated, obj: _*) def expectMsgAllClassOf[T](max: FiniteDuration, obj: Class[_ <: T]*): Seq[T] = internalExpectMsgAllClassOf(max.dilated, obj: _*)
private def internalExpectMsgAllClassOf[T](max: Duration, obj: Class[_ <: T]*): Seq[T] = { private def internalExpectMsgAllClassOf[T](max: FiniteDuration, obj: Class[_ <: T]*): Seq[T] = {
val recv = receiveN_internal(obj.size, max) val recv = receiveN_internal(obj.size, max)
obj foreach (x assert(recv exists (_.getClass eq BoxedType(x)), "not found " + x)) obj foreach (x assert(recv exists (_.getClass eq BoxedType(x)), "not found " + x))
recv foreach (x assert(obj exists (c BoxedType(c) eq x.getClass), "found non-matching object " + x)) recv foreach (x assert(obj exists (c BoxedType(c) eq x.getClass), "found non-matching object " + x))
@ -468,9 +478,9 @@ trait TestKitBase {
* Beware that one object may satisfy all given class constraints, which * Beware that one object may satisfy all given class constraints, which
* may be counter-intuitive. * may be counter-intuitive.
*/ */
def expectMsgAllConformingOf[T](max: Duration, obj: Class[_ <: T]*): Seq[T] = internalExpectMsgAllConformingOf(max.dilated, obj: _*) def expectMsgAllConformingOf[T](max: FiniteDuration, obj: Class[_ <: T]*): Seq[T] = internalExpectMsgAllConformingOf(max.dilated, obj: _*)
private def internalExpectMsgAllConformingOf[T](max: Duration, obj: Class[_ <: T]*): Seq[T] = { private def internalExpectMsgAllConformingOf[T](max: FiniteDuration, obj: Class[_ <: T]*): Seq[T] = {
val recv = receiveN_internal(obj.size, max) val recv = receiveN_internal(obj.size, max)
obj foreach (x assert(recv exists (BoxedType(x) isInstance _), "not found " + x)) obj foreach (x assert(recv exists (BoxedType(x) isInstance _), "not found " + x))
recv foreach (x assert(obj exists (c BoxedType(c) isInstance x), "found non-matching object " + x)) recv foreach (x assert(obj exists (c BoxedType(c) isInstance x), "found non-matching object " + x))
@ -485,9 +495,9 @@ trait TestKitBase {
/** /**
* Assert that no message is received for the specified time. * Assert that no message is received for the specified time.
*/ */
def expectNoMsg(max: Duration) { expectNoMsg_internal(max.dilated) } def expectNoMsg(max: FiniteDuration) { expectNoMsg_internal(max.dilated) }
private def expectNoMsg_internal(max: Duration) { private def expectNoMsg_internal(max: FiniteDuration) {
val o = receiveOne(max) val o = receiveOne(max)
assert(o eq null, "received unexpected message " + o) assert(o eq null, "received unexpected message " + o)
lastWasNoMsg = true lastWasNoMsg = true
@ -512,7 +522,7 @@ trait TestKitBase {
* </pre> * </pre>
*/ */
def receiveWhile[T](max: Duration = Duration.Undefined, idle: Duration = Duration.Inf, messages: Int = Int.MaxValue)(f: PartialFunction[AnyRef, T]): Seq[T] = { def receiveWhile[T](max: Duration = Duration.Undefined, idle: Duration = Duration.Inf, messages: Int = Int.MaxValue)(f: PartialFunction[AnyRef, T]): Seq[T] = {
val stop = now + (if (max eq Duration.Undefined) remaining else max.dilated) val stop = now + remainingOrDilated(max)
var msg: Message = NullMessage var msg: Message = NullMessage
@tailrec @tailrec
@ -549,7 +559,7 @@ trait TestKitBase {
/** /**
* Receive N messages in a row before the given deadline. * Receive N messages in a row before the given deadline.
*/ */
def receiveN(n: Int, max: Duration): Seq[AnyRef] = receiveN_internal(n, max.dilated) def receiveN(n: Int, max: FiniteDuration): Seq[AnyRef] = receiveN_internal(n, max.dilated)
private def receiveN_internal(n: Int, max: Duration): Seq[AnyRef] = { private def receiveN_internal(n: Int, max: Duration): Seq[AnyRef] = {
val stop = max + now val stop = max + now

View file

@ -8,6 +8,7 @@ import scala.concurrent.util.Duration
import akka.actor.ActorSystem import akka.actor.ActorSystem
import scala.concurrent.{ Await, CanAwait, Awaitable } import scala.concurrent.{ Await, CanAwait, Awaitable }
import java.util.concurrent.{ TimeoutException, CountDownLatch, TimeUnit } import java.util.concurrent.{ TimeoutException, CountDownLatch, TimeUnit }
import scala.concurrent.util.FiniteDuration
/** /**
* A count down latch wrapper for use in testing. * A count down latch wrapper for use in testing.
@ -32,7 +33,11 @@ class TestLatch(count: Int = 1)(implicit system: ActorSystem) extends Awaitable[
@throws(classOf[TimeoutException]) @throws(classOf[TimeoutException])
def ready(atMost: Duration)(implicit permit: CanAwait) = { def ready(atMost: Duration)(implicit permit: CanAwait) = {
val opened = latch.await(atMost.dilated.toNanos, TimeUnit.NANOSECONDS) val waitTime = atMost match {
case f: FiniteDuration f
case _ throw new IllegalArgumentException("TestLatch does not support waiting for " + atMost)
}
val opened = latch.await(waitTime.dilated.toNanos, TimeUnit.NANOSECONDS)
if (!opened) throw new TimeoutException( if (!opened) throw new TimeoutException(
"Timeout of %s with time factor of %s" format (atMost.toString, TestKitExtension(system).TestTimeFactor)) "Timeout of %s with time factor of %s" format (atMost.toString, TestKitExtension(system).TestTimeFactor))
this this

View file

@ -3,7 +3,7 @@ package akka
import language.implicitConversions import language.implicitConversions
import akka.actor.ActorSystem import akka.actor.ActorSystem
import scala.concurrent.util.Duration import scala.concurrent.util.{ Duration, FiniteDuration }
import java.util.concurrent.TimeUnit.MILLISECONDS import java.util.concurrent.TimeUnit.MILLISECONDS
import scala.reflect.ClassTag import scala.reflect.ClassTag
@ -41,14 +41,15 @@ package object testkit {
* *
* Corresponding Java API is available in TestKit.dilated * Corresponding Java API is available in TestKit.dilated
*/ */
implicit def duration2TestDuration(duration: Duration) = new TestDuration(duration) implicit def duration2TestDuration(duration: FiniteDuration) = new TestDuration(duration)
/** /**
* Wrapper for implicit conversion to add dilated function to Duration. * Wrapper for implicit conversion to add dilated function to Duration.
*/ */
class TestDuration(duration: Duration) { class TestDuration(duration: FiniteDuration) {
def dilated(implicit system: ActorSystem): Duration = { def dilated(implicit system: ActorSystem): FiniteDuration = {
duration * TestKitExtension(system).TestTimeFactor // this cast will succeed unless TestTimeFactor is non-finite (which would be a misconfiguration)
(duration * TestKitExtension(system).TestTimeFactor).asInstanceOf[FiniteDuration]
} }
} }
} }

View file

@ -36,7 +36,7 @@ object AkkaBuild extends Build {
parallelExecution in GlobalScope := System.getProperty("akka.parallelExecution", "false").toBoolean, parallelExecution in GlobalScope := System.getProperty("akka.parallelExecution", "false").toBoolean,
Publish.defaultPublishTo in ThisBuild <<= crossTarget / "repository", Publish.defaultPublishTo in ThisBuild <<= crossTarget / "repository",
Unidoc.unidocExclude := Seq(samples.id), Unidoc.unidocExclude := Seq(samples.id),
Dist.distExclude := Seq(actorTests.id, akkaSbtPlugin.id, docs.id), Dist.distExclude := Seq(actorTests.id, akkaSbtPlugin.id, docs.id, samples.id),
initialCommands in ThisBuild := initialCommands in ThisBuild :=
"""|import language.postfixOps """|import language.postfixOps
|import akka.actor._ |import akka.actor._
@ -69,7 +69,7 @@ object AkkaBuild extends Build {
sphinxLatex <<= sphinxLatex in LocalProject(docs.id) map identity, sphinxLatex <<= sphinxLatex in LocalProject(docs.id) map identity,
sphinxPdf <<= sphinxPdf in LocalProject(docs.id) map identity sphinxPdf <<= sphinxPdf in LocalProject(docs.id) map identity
), ),
aggregate = Seq(actor, testkit, actorTests, dataflow, remote, remoteTests, camel, cluster, slf4j, agent, transactor, mailboxes, zeroMQ, kernel, akkaSbtPlugin, samples, osgi, osgiAries, docs) aggregate = Seq(actor, testkit, actorTests, dataflow, remote, remoteTests, camel, cluster, slf4j, agent, transactor, mailboxes, zeroMQ, kernel, akkaSbtPlugin, osgi, osgiAries, docs)
) )
lazy val actor = Project( lazy val actor = Project(
@ -275,12 +275,7 @@ object AkkaBuild extends Build {
settings = defaultSettings ++ Seq( settings = defaultSettings ++ Seq(
sbtPlugin := true, sbtPlugin := true,
publishMavenStyle := false, // SBT Plugins should be published as Ivy publishMavenStyle := false, // SBT Plugins should be published as Ivy
publishTo <<= (version) { version: String => publishTo <<= Publish.akkaPluginPublishTo,
val scalasbt = "http://scalasbt.artifactoryonline.com/scalasbt/"
val (name, u) = if (version.contains("-SNAPSHOT")) ("sbt-plugin-snapshots", scalasbt+"sbt-plugin-snapshots")
else ("sbt-plugin-releases", scalasbt+"sbt-plugin-releases")
Some(Resolver.url(name, url(u))(Resolver.ivyStylePatterns))
},
scalacOptions in Compile := Seq("-encoding", "UTF-8", "-deprecation", "-unchecked"), scalacOptions in Compile := Seq("-encoding", "UTF-8", "-deprecation", "-unchecked"),
scalaVersion := "2.9.1", scalaVersion := "2.9.1",
scalaBinaryVersion <<= scalaVersion scalaBinaryVersion <<= scalaVersion
@ -299,7 +294,8 @@ object AkkaBuild extends Build {
base = file("akka-samples/akka-sample-camel"), base = file("akka-samples/akka-sample-camel"),
dependencies = Seq(actor, camel), dependencies = Seq(actor, camel),
settings = defaultSettings ++ Seq( settings = defaultSettings ++ Seq(
libraryDependencies ++= Dependencies.camelSample libraryDependencies ++= Dependencies.camelSample,
publishArtifact in Compile := false
) )
) )
@ -307,35 +303,35 @@ object AkkaBuild extends Build {
id = "akka-sample-fsm", id = "akka-sample-fsm",
base = file("akka-samples/akka-sample-fsm"), base = file("akka-samples/akka-sample-fsm"),
dependencies = Seq(actor), dependencies = Seq(actor),
settings = defaultSettings settings = defaultSettings ++ Seq( publishArtifact in Compile := false )
) )
lazy val helloSample = Project( lazy val helloSample = Project(
id = "akka-sample-hello", id = "akka-sample-hello",
base = file("akka-samples/akka-sample-hello"), base = file("akka-samples/akka-sample-hello"),
dependencies = Seq(actor), dependencies = Seq(actor),
settings = defaultSettings settings = defaultSettings ++ Seq( publishArtifact in Compile := false )
) )
lazy val helloKernelSample = Project( lazy val helloKernelSample = Project(
id = "akka-sample-hello-kernel", id = "akka-sample-hello-kernel",
base = file("akka-samples/akka-sample-hello-kernel"), base = file("akka-samples/akka-sample-hello-kernel"),
dependencies = Seq(kernel), dependencies = Seq(kernel),
settings = defaultSettings settings = defaultSettings ++ Seq( publishArtifact in Compile := false )
) )
lazy val remoteSample = Project( lazy val remoteSample = Project(
id = "akka-sample-remote", id = "akka-sample-remote",
base = file("akka-samples/akka-sample-remote"), base = file("akka-samples/akka-sample-remote"),
dependencies = Seq(actor, remote, kernel), dependencies = Seq(actor, remote, kernel),
settings = defaultSettings settings = defaultSettings ++ Seq( publishArtifact in Compile := false )
) )
lazy val clusterSample = Project( lazy val clusterSample = Project(
id = "akka-sample-cluster", id = "akka-sample-cluster",
base = file("akka-samples/akka-sample-cluster"), base = file("akka-samples/akka-sample-cluster"),
dependencies = Seq(cluster), dependencies = Seq(cluster),
settings = defaultSettings settings = defaultSettings ++ Seq( publishArtifact in Compile := false )
) )
lazy val docs = Project( lazy val docs = Project(
@ -471,7 +467,7 @@ object AkkaBuild extends Build {
.setPreference(AlignSingleLineCaseStatements, true) .setPreference(AlignSingleLineCaseStatements, true)
} }
lazy val multiJvmSettings = SbtMultiJvm.settings ++ inConfig(MultiJvm)(ScalariformPlugin.scalariformSettings) ++ Seq( lazy val multiJvmSettings = SbtMultiJvm.multiJvmSettings ++ inConfig(MultiJvm)(ScalariformPlugin.scalariformSettings) ++ Seq(
compileInputs in MultiJvm <<= (compileInputs in MultiJvm) dependsOn (ScalariformKeys.format in MultiJvm), compileInputs in MultiJvm <<= (compileInputs in MultiJvm) dependsOn (ScalariformKeys.format in MultiJvm),
compile in MultiJvm <<= (compile in MultiJvm) triggeredBy (compile in Test), compile in MultiJvm <<= (compile in MultiJvm) triggeredBy (compile in Test),
ScalariformKeys.preferences in MultiJvm := formattingPreferences) ++ ScalariformKeys.preferences in MultiJvm := formattingPreferences) ++
@ -514,7 +510,7 @@ object AkkaBuild extends Build {
val cluster = exports(Seq("akka.cluster.*")) val cluster = exports(Seq("akka.cluster.*"))
val fileMailbox = exports(Seq("akka.actor.mailbox.*")) val fileMailbox = exports(Seq("akka.actor.mailbox.filebased.*"))
val mailboxesCommon = exports(Seq("akka.actor.mailbox.*")) val mailboxesCommon = exports(Seq("akka.actor.mailbox.*"))
@ -522,7 +518,7 @@ object AkkaBuild extends Build {
val osgiAries = exports() ++ Seq(OsgiKeys.privatePackage := Seq("akka.osgi.aries.*")) val osgiAries = exports() ++ Seq(OsgiKeys.privatePackage := Seq("akka.osgi.aries.*"))
val remote = exports(Seq("akka.remote.*", "akka.routing.*", "akka.serialization.*")) val remote = exports(Seq("akka.remote.*"))
val slf4j = exports(Seq("akka.event.slf4j.*")) val slf4j = exports(Seq("akka.event.slf4j.*"))
@ -534,16 +530,13 @@ object AkkaBuild extends Build {
def exports(packages: Seq[String] = Seq()) = osgiSettings ++ Seq( def exports(packages: Seq[String] = Seq()) = osgiSettings ++ Seq(
OsgiKeys.importPackage := defaultImports, OsgiKeys.importPackage := defaultImports,
OsgiKeys.exportPackage := packages, OsgiKeys.exportPackage := packages
packagedArtifact in (Compile, packageBin) <<= (artifact in (Compile, packageBin), OsgiKeys.bundle).identityMap,
artifact in (Compile, packageBin) ~= (_.copy(`type` = "bundle"))
) )
def defaultImports = Seq("!sun.misc", akkaImport(), configImport(), scalaImport(), "*") def defaultImports = Seq("!sun.misc", akkaImport(), configImport(), scalaImport(), "*")
def akkaImport(packageName: String = "akka.*") = "%s;version=\"[2.1,2.2)\"".format(packageName) def akkaImport(packageName: String = "akka.*") = "%s;version=\"[2.1,2.2)\"".format(packageName)
def configImport(packageName: String = "com.typesafe.config.*") = "%s;version=\"[0.4.1,0.5)\"".format(packageName) def configImport(packageName: String = "com.typesafe.config.*") = "%s;version=\"[0.4.1,0.5)\"".format(packageName)
def scalaImport(packageName: String = "scala.*") = "%s;version=\"[2.10,2.11)\"".format(packageName) def scalaImport(packageName: String = "scala.*") = "%s;version=\"[2.10,2.11)\"".format(packageName)
} }
} }

View file

@ -74,6 +74,15 @@ object Publish {
} }
} }
def akkaPluginPublishTo: Initialize[Option[Resolver]] = {
(version) { version: String =>
akkaPublishRepository orElse {
val name = if (version.contains("-SNAPSHOT")) "sbt-plugin-snapshots" else "sbt-plugin-releases"
Some(Resolver.url(name, url("http://scalasbt.artifactoryonline.com/scalasbt/" + name))(Resolver.ivyStylePatterns))
}
}
}
def sonatypeRepo(version: String): Option[Resolver] = { def sonatypeRepo(version: String): Option[Resolver] = {
Option(sys.props("publish.maven.central")) filter (_.toLowerCase == "true") map { _ => Option(sys.props("publish.maven.central")) filter (_.toLowerCase == "true") map { _ =>
val nexus = "https://oss.sonatype.org/" val nexus = "https://oss.sonatype.org/"
@ -83,15 +92,11 @@ object Publish {
} }
def akkaPublishRepository: Option[Resolver] = { def akkaPublishRepository: Option[Resolver] =
val property = Option(System.getProperty("akka.publish.repository")) Option(System.getProperty("akka.publish.repository", null)) map { "Akka Publish Repository" at _ }
property map { "Akka Publish Repository" at _ }
}
def akkaCredentials: Seq[Credentials] = { def akkaCredentials: Seq[Credentials] =
val property = Option(System.getProperty("akka.publish.credentials")) Option(System.getProperty("akka.publish.credentials", null)) map (f => Credentials(new File(f))) toSeq
property map (f => Credentials(new File(f))) toSeq
}
// timestamped versions // timestamped versions

View file

@ -1,11 +1,11 @@
resolvers += Classpaths.typesafeResolver resolvers += Classpaths.typesafeResolver
addSbtPlugin("com.typesafe.sbt" % "sbt-multi-jvm" % "0.2.0") addSbtPlugin("com.typesafe.sbt" % "sbt-multi-jvm" % "0.3.0")
addSbtPlugin("com.typesafe.sbtscalariform" % "sbtscalariform" % "0.4.0") addSbtPlugin("com.typesafe.sbtscalariform" % "sbtscalariform" % "0.4.0")
addSbtPlugin("com.typesafe.sbtosgi" % "sbtosgi" % "0.2.0") addSbtPlugin("com.typesafe.sbtosgi" % "sbtosgi" % "0.3.0")
addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.3") addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.3")