Merge with master
This commit is contained in:
commit
d2c24f3034
79 changed files with 367 additions and 286 deletions
|
|
@ -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 =
|
||||||
|
|
|
||||||
|
|
@ -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
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
@ -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")
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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 ⇒
|
||||||
|
|
|
||||||
|
|
@ -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]]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -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) {
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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:
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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"),
|
||||||
|
|
|
||||||
|
|
@ -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)) }
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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(
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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();
|
||||||
|
|
|
||||||
|
|
@ -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'
|
||||||
|
|
|
||||||
|
|
@ -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|",
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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.
|
||||||
|
|
|
||||||
19
akka-docs/images/circuit-breaker-states.dot
Normal file
19
akka-docs/images/circuit-breaker-states.dot
Normal 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" ];
|
||||||
BIN
akka-docs/images/circuit-breaker-states.png
Normal file
BIN
akka-docs/images/circuit-breaker-states.png
Normal file
Binary file not shown.
|
After Width: | Height: | Size: 22 KiB |
|
|
@ -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;
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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);
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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());
|
||||||
|
|
|
||||||
|
|
@ -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;
|
||||||
|
|
|
||||||
|
|
@ -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));
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
"""
|
"""
|
||||||
|
|
|
||||||
|
|
@ -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 you’ll 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.*``
|
||||||
|
================================================ =========================================================
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
----------------------------------------
|
----------------------------------------
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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"
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
@ -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
|
||||||
|
|
@ -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
|
||||||
|
|
||||||
|
|
@ -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
|
||||||
|
|
||||||
|
|
@ -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 }
|
||||||
|
|
@ -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) {
|
||||||
|
|
@ -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 }
|
||||||
|
|
||||||
|
|
@ -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")
|
||||||
|
|
||||||
|
|
@ -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
|
||||||
|
|
||||||
|
|
@ -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
|
||||||
|
|
@ -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
|
||||||
|
|
@ -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 ⇒
|
||||||
|
|
|
||||||
|
|
@ -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"
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
@ -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
|
||||||
|
|
@ -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
|
||||||
|
|
@ -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
|
||||||
|
|
@ -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
|
||||||
|
|
@ -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 }
|
||||||
|
|
||||||
|
|
@ -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 }
|
||||||
|
|
||||||
|
|
@ -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
|
||||||
|
|
@ -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 }
|
||||||
|
|
@ -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
|
||||||
|
|
@ -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;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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 }
|
||||||
|
|
@ -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 {
|
||||||
|
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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]
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
|
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue