Merge pull request #18875 from ktoso/wip-akka.js-cherries-ktoso

Akka.js cherries to master
This commit is contained in:
Roland Kuhn 2015-11-07 18:01:24 +01:00
commit f1abaa1c5e
41 changed files with 630 additions and 580 deletions

View file

@ -10,7 +10,7 @@ import java.util.concurrent._
import atomic.{ AtomicReference, AtomicInteger }
import scala.concurrent.{ future, Await, ExecutionContext }
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import scala.util.Try
import scala.util.control.NonFatal
import org.scalatest.BeforeAndAfterEach

View file

@ -5,7 +5,7 @@ package akka.pattern
import scala.concurrent.Promise
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.Props

View file

@ -112,7 +112,7 @@ abstract class AbstractFSM[S, D] extends FSM[S, D] {
* called, not only the first one matching.</b>
*/
final def onTransition(transitionHandler: UnitApply2[S, S]): Unit =
onTransition(transitionHandler(_: S, _: S))
onTransition(transitionHandler(_: S, _: S))
/**
* Set handler which is called upon reception of unhandled messages. Calling

View file

@ -0,0 +1,72 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor
import java.lang.reflect.{ Modifier, ParameterizedType, TypeVariable }
import akka.dispatch._
import akka.japi.Creator
import akka.routing._
import akka.util.Reflect
import scala.annotation.varargs
import scala.language.existentials
import scala.reflect.ClassTag
import akka.japi.Util.immutableSeq
/**
*
* Java API: Factory for Props instances.
*/
private[akka] trait AbstractProps {
/**
* INTERNAL API
*/
private[akka] def validate(clazz: Class[_]) =
if (Modifier.isAbstract(clazz.getModifiers))
throw new IllegalArgumentException(s"Actor class [${clazz.getName}] must not be abstract")
/**
* Java API: create a Props given a class and its constructor arguments.
*/
@varargs
def create(clazz: Class[_], args: AnyRef*): Props = new Props(deploy = Props.defaultDeploy, clazz = clazz, args = args.toList)
/**
* Create new Props from the given [[akka.japi.Creator]].
*
* You can not use a Java 8 lambda with this method since the generated classes
* don't carry enough type information.
*
* Use the Props.create(actorClass, creator) instead.
*/
def create[T <: Actor](creator: Creator[T]): Props = {
val cc = creator.getClass
if ((cc.getEnclosingClass ne null) && (cc.getModifiers & Modifier.STATIC) == 0)
throw new IllegalArgumentException("cannot use non-static local Creator to create actors; make it static (e.g. local to a static method) or top-level")
val ac = classOf[Actor]
val coc = classOf[Creator[_]]
val actorClass = Reflect.findMarker(cc, coc) match {
case t: ParameterizedType
t.getActualTypeArguments.head match {
case c: Class[_] c // since T <: Actor
case v: TypeVariable[_]
v.getBounds collectFirst { case c: Class[_] if ac.isAssignableFrom(c) && c != ac c } getOrElse ac
case x throw new IllegalArgumentException(s"unsupported type found in Creator argument [$x]")
}
case c: Class[_] if (c == coc)
throw new IllegalArgumentException(s"erased Creator types are unsupported, use Props.create(actorClass, creator) instead")
}
create(classOf[CreatorConsumer], actorClass, creator)
}
/**
* Create new Props from the given [[akka.japi.Creator]] with the type set to the given actorClass.
*/
def create[T <: Actor](actorClass: Class[T], creator: Creator[T]): Props = {
create(classOf[CreatorConsumer], actorClass, creator)
}
}

View file

@ -14,7 +14,7 @@ import scala.annotation.{ switch, tailrec }
import scala.collection.immutable
import scala.concurrent.ExecutionContextExecutor
import scala.concurrent.duration.Duration
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import scala.util.control.NonFatal
import akka.dispatch.MessageDispatcher

View file

@ -4,7 +4,6 @@
package akka.actor
import scala.collection.immutable
import java.lang.reflect.InvocationTargetException
import scala.reflect.ClassTag
import scala.util.Try
@ -53,51 +52,3 @@ abstract class DynamicAccess {
*/
def classLoader: ClassLoader
}
/**
* This is the default [[akka.actor.DynamicAccess]] implementation used by [[akka.actor.ExtendedActorSystem]]
* unless overridden. It uses reflection to turn fully-qualified class names into `Class[_]` objects
* and creates instances from there using `getDeclaredConstructor()` and invoking that. The class loader
* to be used for all this is determined by the actor systems class loader by default.
*/
class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAccess {
override def getClassFor[T: ClassTag](fqcn: String): Try[Class[_ <: T]] =
Try[Class[_ <: T]]({
val c = Class.forName(fqcn, false, classLoader).asInstanceOf[Class[_ <: T]]
val t = implicitly[ClassTag[T]].runtimeClass
if (t.isAssignableFrom(c)) c else throw new ClassCastException(t + " is not assignable from " + c)
})
override def createInstanceFor[T: ClassTag](clazz: Class[_], args: immutable.Seq[(Class[_], AnyRef)]): Try[T] =
Try {
val types = args.map(_._1).toArray
val values = args.map(_._2).toArray
val constructor = clazz.getDeclaredConstructor(types: _*)
constructor.setAccessible(true)
val obj = constructor.newInstance(values: _*)
val t = implicitly[ClassTag[T]].runtimeClass
if (t.isInstance(obj)) obj.asInstanceOf[T] else throw new ClassCastException(clazz.getName + " is not a subtype of " + t)
} recover { case i: InvocationTargetException if i.getTargetException ne null throw i.getTargetException }
override def createInstanceFor[T: ClassTag](fqcn: String, args: immutable.Seq[(Class[_], AnyRef)]): Try[T] =
getClassFor(fqcn) flatMap { c createInstanceFor(c, args) }
override def getObjectFor[T: ClassTag](fqcn: String): Try[T] = {
val classTry =
if (fqcn.endsWith("$")) getClassFor(fqcn)
else getClassFor(fqcn + "$") recoverWith { case _ getClassFor(fqcn) }
classTry flatMap { c
Try {
val module = c.getDeclaredField("MODULE$")
module.setAccessible(true)
val t = implicitly[ClassTag[T]].runtimeClass
module.get(null) match {
case null throw new NullPointerException
case x if !t.isInstance(x) throw new ClassCastException(fqcn + " is not a subtype of " + t)
case x: T x
}
} recover { case i: InvocationTargetException if i.getTargetException ne null throw i.getTargetException }
}
}
}

View file

@ -0,0 +1,108 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor
import java.lang.reflect.Constructor
import akka.japi.Creator
import akka.util.Reflect
import scala.collection.immutable
/**
* This interface defines a class of actor creation strategies deviating from
* the usual default of just reflectively instantiating the [[Actor]]
* subclass. It can be used to allow a dependency injection framework to
* determine the actual actor class and how it shall be instantiated.
*/
trait IndirectActorProducer {
/**
* This factory method must produce a fresh actor instance upon each
* invocation. <b>It is not permitted to return the same instance more than
* once.</b>
*/
def produce(): Actor
/**
* This method is used by [[Props]] to determine the type of actor which will
* be created. This means that an instance of this `IndirectActorProducer`
* will be created in order to call this method during any call to
* [[Props#actorClass]]; it should be noted that such calls may
* performed during actor set-up before the actual actors instantiation, and
* that the instance created for calling `actorClass` is not necessarily reused
* later to produce the actor.
*/
def actorClass: Class[_ <: Actor]
}
private[akka] object IndirectActorProducer {
val CreatorFunctionConsumerClass = classOf[CreatorFunctionConsumer]
val CreatorConsumerClass = classOf[CreatorConsumer]
val TypedCreatorFunctionConsumerClass = classOf[TypedCreatorFunctionConsumer]
def apply(clazz: Class[_], args: immutable.Seq[Any]): IndirectActorProducer = {
if (classOf[IndirectActorProducer].isAssignableFrom(clazz)) {
def get1stArg[T]: T = args.head.asInstanceOf[T]
def get2ndArg[T]: T = args.tail.head.asInstanceOf[T]
// The cost of doing reflection to create these for every props
// is rather high, so we match on them and do new instead
clazz match {
case TypedCreatorFunctionConsumerClass
new TypedCreatorFunctionConsumer(get1stArg, get2ndArg)
case CreatorFunctionConsumerClass
new CreatorFunctionConsumer(get1stArg)
case CreatorConsumerClass
new CreatorConsumer(get1stArg, get2ndArg)
case _
Reflect.instantiate(clazz, args).asInstanceOf[IndirectActorProducer]
}
} else if (classOf[Actor].isAssignableFrom(clazz)) {
if (args.isEmpty) new NoArgsReflectConstructor(clazz.asInstanceOf[Class[_ <: Actor]])
else new ArgsReflectConstructor(clazz.asInstanceOf[Class[_ <: Actor]], args)
} else throw new IllegalArgumentException(s"unknown actor creator [$clazz]")
}
}
/**
* INTERNAL API
*/
private[akka] class CreatorFunctionConsumer(creator: () Actor) extends IndirectActorProducer {
override def actorClass = classOf[Actor]
override def produce() = creator()
}
/**
* INTERNAL API
*/
private[akka] class CreatorConsumer(clazz: Class[_ <: Actor], creator: Creator[Actor]) extends IndirectActorProducer {
override def actorClass = clazz
override def produce() = creator.create()
}
/**
* INTERNAL API
*/
private[akka] class TypedCreatorFunctionConsumer(clz: Class[_ <: Actor], creator: () Actor) extends IndirectActorProducer {
override def actorClass = clz
override def produce() = creator()
}
/**
* INTERNAL API
*/
private[akka] class ArgsReflectConstructor(clz: Class[_ <: Actor], args: immutable.Seq[Any]) extends IndirectActorProducer {
private[this] val constructor: Constructor[_] = Reflect.findConstructor(clz, args)
override def actorClass = clz
override def produce() = Reflect.instantiate(constructor, args).asInstanceOf[Actor]
}
/**
* INTERNAL API
*/
private[akka] class NoArgsReflectConstructor(clz: Class[_ <: Actor]) extends IndirectActorProducer {
Reflect.findConstructor(clz, List.empty)
override def actorClass = clz
override def produce() = Reflect.instantiate(clz)
}

View file

@ -0,0 +1,354 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor
import java.io.Closeable
import java.util.concurrent.ThreadFactory
import java.util.concurrent.atomic.{ AtomicLong, AtomicReference, AtomicReferenceArray }
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.{ Await, ExecutionContext, Future, Promise }
import scala.concurrent.duration._
import scala.util.control.{ NoStackTrace, NonFatal }
import com.typesafe.config.Config
import akka.event.LoggingAdapter
import akka.util.Helpers
import akka.util.Unsafe.{ instance unsafe }
import akka.dispatch.AbstractNodeQueue
/**
* This scheduler implementation is based on a revolving wheel of buckets,
* like Nettys HashedWheelTimer, which it advances at a fixed tick rate and
* dispatches tasks it finds in the current bucket to their respective
* ExecutionContexts. The tasks are held in TaskHolders, which upon
* cancellation null out their reference to the actual task, leaving only this
* shell to be cleaned up when the wheel reaches that bucket next time. This
* enables the use of a simple linked list to chain the TaskHolders off the
* wheel.
*
* Also noteworthy is that this scheduler does not obtain a current time stamp
* when scheduling single-shot tasks, instead it always rounds up the task
* delay to a full multiple of the TickDuration. This means that tasks are
* scheduled possibly one tick later than they could be (if checking that
* now() + delay &lt;= nextTick were done).
*/
class LightArrayRevolverScheduler(config: Config,
log: LoggingAdapter,
threadFactory: ThreadFactory)
extends Scheduler with Closeable {
import Helpers.Requiring
import Helpers.ConfigOps
val WheelSize =
config.getInt("akka.scheduler.ticks-per-wheel")
.requiring(ticks (ticks & (ticks - 1)) == 0, "ticks-per-wheel must be a power of 2")
val TickDuration =
config.getMillisDuration("akka.scheduler.tick-duration")
.requiring(_ >= 10.millis || !Helpers.isWindows, "minimum supported akka.scheduler.tick-duration on Windows is 10ms")
.requiring(_ >= 1.millis, "minimum supported akka.scheduler.tick-duration is 1ms")
val ShutdownTimeout = config.getMillisDuration("akka.scheduler.shutdown-timeout")
import LightArrayRevolverScheduler._
private val oneNs = Duration.fromNanos(1l)
private def roundUp(d: FiniteDuration): FiniteDuration =
try {
((d + TickDuration - oneNs) / TickDuration).toLong * TickDuration
} catch {
case _: IllegalArgumentException d // rounding up Long.MaxValue.nanos overflows
}
/**
* Clock implementation is replaceable (for testing); the implementation must
* return a monotonically increasing series of Long nanoseconds.
*/
protected def clock(): Long = System.nanoTime
/**
* Overridable for tests
*/
protected def getShutdownTimeout: FiniteDuration = ShutdownTimeout
/**
* Overridable for tests
*/
protected def waitNanos(nanos: Long): Unit = {
// see http://www.javamex.com/tutorials/threads/sleep_issues.shtml
val sleepMs = if (Helpers.isWindows) (nanos + 4999999) / 10000000 * 10 else (nanos + 999999) / 1000000
try Thread.sleep(sleepMs) catch {
case _: InterruptedException Thread.currentThread.interrupt() // we got woken up
}
}
override def schedule(initialDelay: FiniteDuration,
delay: FiniteDuration,
runnable: Runnable)(implicit executor: ExecutionContext): Cancellable = {
checkMaxDelay(roundUp(delay).toNanos)
val preparedEC = executor.prepare()
try new AtomicReference[Cancellable](InitialRepeatMarker) with Cancellable { self
compareAndSet(InitialRepeatMarker, schedule(
preparedEC,
new AtomicLong(clock() + initialDelay.toNanos) with Runnable {
override def run(): Unit = {
try {
runnable.run()
val driftNanos = clock() - getAndAdd(delay.toNanos)
if (self.get != null)
swap(schedule(preparedEC, this, Duration.fromNanos(Math.max(delay.toNanos - driftNanos, 1))))
} catch {
case _: SchedulerException // ignore failure to enqueue or terminated target actor
}
}
}, roundUp(initialDelay)))
@tailrec private def swap(c: Cancellable): Unit = {
get match {
case null if (c != null) c.cancel()
case old if (!compareAndSet(old, c)) swap(c)
}
}
@tailrec final def cancel(): Boolean = {
get match {
case null false
case c
if (c.cancel()) compareAndSet(c, null)
else compareAndSet(c, null) || cancel()
}
}
override def isCancelled: Boolean = get == null
} catch {
case SchedulerException(msg) throw new IllegalStateException(msg)
}
}
override def scheduleOnce(delay: FiniteDuration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable =
try schedule(executor.prepare(), runnable, roundUp(delay))
catch {
case SchedulerException(msg) throw new IllegalStateException(msg)
}
override def close(): Unit = Await.result(stop(), getShutdownTimeout) foreach {
task
try task.run() catch {
case e: InterruptedException throw e
case _: SchedulerException // ignore terminated actors
case NonFatal(e) log.error(e, "exception while executing timer task")
}
}
override val maxFrequency: Double = 1.second / TickDuration
/*
* BELOW IS THE ACTUAL TIMER IMPLEMENTATION
*/
private val start = clock()
private val tickNanos = TickDuration.toNanos
private val wheelMask = WheelSize - 1
private val queue = new TaskQueue
private def schedule(ec: ExecutionContext, r: Runnable, delay: FiniteDuration): TimerTask =
if (delay <= Duration.Zero) {
if (stopped.get != null) throw new SchedulerException("cannot enqueue after timer shutdown")
ec.execute(r)
NotCancellable
} else if (stopped.get != null) {
throw new SchedulerException("cannot enqueue after timer shutdown")
} else {
val delayNanos = delay.toNanos
checkMaxDelay(delayNanos)
val ticks = (delayNanos / tickNanos).toInt
val task = new TaskHolder(r, ticks, ec)
queue.add(task)
if (stopped.get != null && task.cancel())
throw new SchedulerException("cannot enqueue after timer shutdown")
task
}
private def checkMaxDelay(delayNanos: Long): Unit =
if (delayNanos / tickNanos > Int.MaxValue)
// 1 second margin in the error message due to rounding
throw new IllegalArgumentException(s"Task scheduled with [${delayNanos.nanos.toSeconds}] seconds delay, " +
s"which is too far in future, maximum delay is [${(tickNanos * Int.MaxValue).nanos.toSeconds - 1}] seconds")
private val stopped = new AtomicReference[Promise[immutable.Seq[TimerTask]]]
private def stop(): Future[immutable.Seq[TimerTask]] = {
val p = Promise[immutable.Seq[TimerTask]]()
if (stopped.compareAndSet(null, p)) {
// Interrupting the timer thread to make it shut down faster is not good since
// it could be in the middle of executing the scheduled tasks, which might not
// respond well to being interrupted.
// Instead we just wait one more tick for it to finish.
p.future
} else Future.successful(Nil)
}
@volatile private var timerThread: Thread = threadFactory.newThread(new Runnable {
var tick = 0
val wheel = Array.fill(WheelSize)(new TaskQueue)
private def clearAll(): immutable.Seq[TimerTask] = {
@tailrec def collect(q: TaskQueue, acc: Vector[TimerTask]): Vector[TimerTask] = {
q.poll() match {
case null acc
case x collect(q, acc :+ x)
}
}
((0 until WheelSize) flatMap (i collect(wheel(i), Vector.empty))) ++ collect(queue, Vector.empty)
}
@tailrec
private def checkQueue(time: Long): Unit = queue.pollNode() match {
case null ()
case node
node.value.ticks match {
case 0 node.value.executeTask()
case ticks
val futureTick = ((
time - start + // calculate the nanos since timer start
(ticks * tickNanos) + // adding the desired delay
tickNanos - 1 // rounding up
) / tickNanos).toInt // and converting to slot number
// tick is an Int that will wrap around, but toInt of futureTick gives us modulo operations
// and the difference (offset) will be correct in any case
val offset = futureTick - tick
val bucket = futureTick & wheelMask
node.value.ticks = offset
wheel(bucket).addNode(node)
}
checkQueue(time)
}
override final def run =
try nextTick()
catch {
case t: Throwable
log.error(t, "exception on LARS timer thread")
stopped.get match {
case null
val thread = threadFactory.newThread(this)
log.info("starting new LARS thread")
try thread.start()
catch {
case e: Throwable
log.error(e, "LARS cannot start new thread, ships going down!")
stopped.set(Promise successful Nil)
clearAll()
}
timerThread = thread
case p
assert(stopped.compareAndSet(p, Promise successful Nil), "Stop signal violated in LARS")
p success clearAll()
}
throw t
}
@tailrec final def nextTick(): Unit = {
val time = clock()
val sleepTime = start + (tick * tickNanos) - time
if (sleepTime > 0) {
// check the queue before taking a nap
checkQueue(time)
waitNanos(sleepTime)
} else {
val bucket = tick & wheelMask
val tasks = wheel(bucket)
val putBack = new TaskQueue
@tailrec def executeBucket(): Unit = tasks.pollNode() match {
case null ()
case node
val task = node.value
if (!task.isCancelled) {
if (task.ticks >= WheelSize) {
task.ticks -= WheelSize
putBack.addNode(node)
} else task.executeTask()
}
executeBucket()
}
executeBucket()
wheel(bucket) = putBack
tick += 1
}
stopped.get match {
case null nextTick()
case p
assert(stopped.compareAndSet(p, Promise successful Nil), "Stop signal violated in LARS")
p success clearAll()
}
}
})
timerThread.start()
}
object LightArrayRevolverScheduler {
private[this] val taskOffset = unsafe.objectFieldOffset(classOf[TaskHolder].getDeclaredField("task"))
private class TaskQueue extends AbstractNodeQueue[TaskHolder]
/**
* INTERNAL API
*/
protected[actor] trait TimerTask extends Runnable with Cancellable
/**
* INTERNAL API
*/
protected[actor] class TaskHolder(@volatile var task: Runnable, var ticks: Int, executionContext: ExecutionContext)
extends TimerTask {
@tailrec
private final def extractTask(replaceWith: Runnable): Runnable =
task match {
case t @ (ExecutedTask | CancelledTask) t
case x if (unsafe.compareAndSwapObject(this, taskOffset, x, replaceWith)) x else extractTask(replaceWith)
}
private[akka] final def executeTask(): Boolean = extractTask(ExecutedTask) match {
case ExecutedTask | CancelledTask false
case other
try {
executionContext execute other
true
} catch {
case _: InterruptedException { Thread.currentThread.interrupt(); false }
case NonFatal(e) { executionContext.reportFailure(e); false }
}
}
// This should only be called in execDirectly
override def run(): Unit = extractTask(ExecutedTask).run()
override def cancel(): Boolean = extractTask(CancelledTask) match {
case ExecutedTask | CancelledTask false
case _ true
}
override def isCancelled: Boolean = task eq CancelledTask
}
private[this] val CancelledTask = new Runnable { def run = () }
private[this] val ExecutedTask = new Runnable { def run = () }
private val NotCancellable: TimerTask = new TimerTask {
def cancel(): Boolean = false
def isCancelled: Boolean = false
def run(): Unit = ()
}
private val InitialRepeatMarker: Cancellable = new Cancellable {
def cancel(): Boolean = false
def isCancelled: Boolean = false
}
}

View file

@ -4,13 +4,9 @@
package akka.actor
import java.lang.reflect.{ Constructor, Modifier, ParameterizedType, TypeVariable }
import akka.actor.Deploy.{ NoDispatcherGiven, NoMailboxGiven }
import akka.dispatch._
import akka.japi.Creator
import akka.routing._
import akka.util.Reflect
import scala.annotation.varargs
import scala.collection.immutable
@ -24,7 +20,7 @@ import scala.reflect.ClassTag
*
* Used when creating new actors through <code>ActorSystem.actorOf</code> and <code>ActorContext.actorOf</code>.
*/
object Props {
object Props extends AbstractProps {
/**
* The defaultCreator, simply throws an UnsupportedOperationException when applied, which is used when creating a Props
@ -90,46 +86,6 @@ object Props {
*/
def apply(clazz: Class[_], args: Any*): Props = apply(defaultDeploy, clazz, args.toList)
/**
* Java API: create a Props given a class and its constructor arguments.
*/
@varargs
def create(clazz: Class[_], args: AnyRef*): Props = apply(defaultDeploy, clazz, args.toList)
/**
* Create new Props from the given [[akka.japi.Creator]].
*
* You can not use a Java 8 lambda with this method since the generated classes
* don't carry enough type information.
*
* Use the Props.create(actorClass, creator) instead.
*/
def create[T <: Actor](creator: Creator[T]): Props = {
val cc = creator.getClass
if ((cc.getEnclosingClass ne null) && (cc.getModifiers & Modifier.STATIC) == 0)
throw new IllegalArgumentException("cannot use non-static local Creator to create actors; make it static (e.g. local to a static method) or top-level")
val ac = classOf[Actor]
val coc = classOf[Creator[_]]
val actorClass = Reflect.findMarker(cc, coc) match {
case t: ParameterizedType
t.getActualTypeArguments.head match {
case c: Class[_] c // since T <: Actor
case v: TypeVariable[_]
v.getBounds collectFirst { case c: Class[_] if ac.isAssignableFrom(c) && c != ac c } getOrElse ac
case x throw new IllegalArgumentException(s"unsupported type found in Creator argument [$x]")
}
case c: Class[_] if (c == coc)
throw new IllegalArgumentException(s"erased Creator types are unsupported, use Props.create(actorClass, creator) instead")
}
apply(defaultDeploy, classOf[CreatorConsumer], actorClass :: creator :: Nil)
}
/**
* Create new Props from the given [[akka.japi.Creator]] with the type set to the given actorClass.
*/
def create[T <: Actor](actorClass: Class[T], creator: Creator[T]): Props = {
apply(defaultDeploy, classOf[CreatorConsumer], actorClass :: creator :: Nil)
}
}
/**
@ -158,8 +114,7 @@ object Props {
@SerialVersionUID(2L)
final case class Props(deploy: Deploy, clazz: Class[_], args: immutable.Seq[Any]) {
if (Modifier.isAbstract(clazz.getModifiers))
throw new IllegalArgumentException(s"Actor class [${clazz.getName}] must not be abstract")
Props.validate(clazz)
// derived property, does not need to be serialized
@transient
@ -259,100 +214,3 @@ final case class Props(deploy: Deploy, clazz: Class[_], args: immutable.Seq[Any]
producer.produce()
}
}
/**
* This interface defines a class of actor creation strategies deviating from
* the usual default of just reflectively instantiating the [[Actor]]
* subclass. It can be used to allow a dependency injection framework to
* determine the actual actor class and how it shall be instantiated.
*/
trait IndirectActorProducer {
/**
* This factory method must produce a fresh actor instance upon each
* invocation. <b>It is not permitted to return the same instance more than
* once.</b>
*/
def produce(): Actor
/**
* This method is used by [[Props]] to determine the type of actor which will
* be created. This means that an instance of this `IndirectActorProducer`
* will be created in order to call this method during any call to
* [[Props#actorClass]]; it should be noted that such calls may
* performed during actor set-up before the actual actors instantiation, and
* that the instance created for calling `actorClass` is not necessarily reused
* later to produce the actor.
*/
def actorClass: Class[_ <: Actor]
}
private[akka] object IndirectActorProducer {
val CreatorFunctionConsumerClass = classOf[CreatorFunctionConsumer]
val CreatorConsumerClass = classOf[CreatorConsumer]
val TypedCreatorFunctionConsumerClass = classOf[TypedCreatorFunctionConsumer]
def apply(clazz: Class[_], args: immutable.Seq[Any]): IndirectActorProducer = {
if (classOf[IndirectActorProducer].isAssignableFrom(clazz)) {
def get1stArg[T]: T = args.head.asInstanceOf[T]
def get2ndArg[T]: T = args.tail.head.asInstanceOf[T]
// The cost of doing reflection to create these for every props
// is rather high, so we match on them and do new instead
clazz match {
case TypedCreatorFunctionConsumerClass
new TypedCreatorFunctionConsumer(get1stArg, get2ndArg)
case CreatorFunctionConsumerClass
new CreatorFunctionConsumer(get1stArg)
case CreatorConsumerClass
new CreatorConsumer(get1stArg, get2ndArg)
case _
Reflect.instantiate(clazz, args).asInstanceOf[IndirectActorProducer]
}
} else if (classOf[Actor].isAssignableFrom(clazz)) {
if (args.isEmpty) new NoArgsReflectConstructor(clazz.asInstanceOf[Class[_ <: Actor]])
else new ArgsReflectConstructor(clazz.asInstanceOf[Class[_ <: Actor]], args)
} else throw new IllegalArgumentException(s"unknown actor creator [$clazz]")
}
}
/**
* INTERNAL API
*/
private[akka] class CreatorFunctionConsumer(creator: () Actor) extends IndirectActorProducer {
override def actorClass = classOf[Actor]
override def produce() = creator()
}
/**
* INTERNAL API
*/
private[akka] class CreatorConsumer(clazz: Class[_ <: Actor], creator: Creator[Actor]) extends IndirectActorProducer {
override def actorClass = clazz
override def produce() = creator.create()
}
/**
* INTERNAL API
*/
private[akka] class TypedCreatorFunctionConsumer(clz: Class[_ <: Actor], creator: () Actor) extends IndirectActorProducer {
override def actorClass = clz
override def produce() = creator()
}
/**
* INTERNAL API
*/
private[akka] class ArgsReflectConstructor(clz: Class[_ <: Actor], args: immutable.Seq[Any]) extends IndirectActorProducer {
private[this] val constructor: Constructor[_] = Reflect.findConstructor(clz, args)
override def actorClass = clz
override def produce() = Reflect.instantiate(constructor, args).asInstanceOf[Actor]
}
/**
* INTERNAL API
*/
private[akka] class NoArgsReflectConstructor(clz: Class[_ <: Actor]) extends IndirectActorProducer {
Reflect.findConstructor(clz, List.empty)
override def actorClass = clz
override def produce() = Reflect.instantiate(clz)
}

View file

@ -0,0 +1,57 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor
import scala.collection.immutable
import java.lang.reflect.InvocationTargetException
import scala.reflect.ClassTag
import scala.util.Try
/**
* This is the default [[akka.actor.DynamicAccess]] implementation used by [[akka.actor.ExtendedActorSystem]]
* unless overridden. It uses reflection to turn fully-qualified class names into `Class[_]` objects
* and creates instances from there using `getDeclaredConstructor()` and invoking that. The class loader
* to be used for all this is determined by the actor systems class loader by default.
*/
class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAccess {
override def getClassFor[T: ClassTag](fqcn: String): Try[Class[_ <: T]] =
Try[Class[_ <: T]]({
val c = Class.forName(fqcn, false, classLoader).asInstanceOf[Class[_ <: T]]
val t = implicitly[ClassTag[T]].runtimeClass
if (t.isAssignableFrom(c)) c else throw new ClassCastException(t + " is not assignable from " + c)
})
override def createInstanceFor[T: ClassTag](clazz: Class[_], args: immutable.Seq[(Class[_], AnyRef)]): Try[T] =
Try {
val types = args.map(_._1).toArray
val values = args.map(_._2).toArray
val constructor = clazz.getDeclaredConstructor(types: _*)
constructor.setAccessible(true)
val obj = constructor.newInstance(values: _*)
val t = implicitly[ClassTag[T]].runtimeClass
if (t.isInstance(obj)) obj.asInstanceOf[T] else throw new ClassCastException(clazz.getName + " is not a subtype of " + t)
} recover { case i: InvocationTargetException if i.getTargetException ne null throw i.getTargetException }
override def createInstanceFor[T: ClassTag](fqcn: String, args: immutable.Seq[(Class[_], AnyRef)]): Try[T] =
getClassFor(fqcn) flatMap { c createInstanceFor(c, args) }
override def getObjectFor[T: ClassTag](fqcn: String): Try[T] = {
val classTry =
if (fqcn.endsWith("$")) getClassFor(fqcn)
else getClassFor(fqcn + "$") recoverWith { case _ getClassFor(fqcn) }
classTry flatMap { c
Try {
val module = c.getDeclaredField("MODULE$")
module.setAccessible(true)
val t = implicitly[ClassTag[T]].runtimeClass
module.get(null) match {
case null throw new NullPointerException
case x if !t.isInstance(x) throw new ClassCastException(fqcn + " is not a subtype of " + t)
case x: T x
}
} recover { case i: InvocationTargetException if i.getTargetException ne null throw i.getTargetException }
}
}
}

View file

@ -4,19 +4,9 @@
package akka.actor
import java.io.Closeable
import java.util.concurrent.ThreadFactory
import java.util.concurrent.atomic.{ AtomicLong, AtomicReference, AtomicReferenceArray }
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.{ Await, ExecutionContext, Future, Promise }
import scala.concurrent.ExecutionContext
import scala.concurrent.duration._
import scala.util.control.{ NoStackTrace, NonFatal }
import com.typesafe.config.Config
import akka.event.LoggingAdapter
import akka.util.Helpers
import akka.util.Unsafe.{ instance unsafe }
import akka.dispatch.AbstractNodeQueue
import scala.util.control.NoStackTrace
/**
* This exception is thrown by Scheduler.schedule* when scheduling is not
@ -177,343 +167,3 @@ trait Cancellable {
def isCancelled: Boolean
}
//#cancellable
/**
* This scheduler implementation is based on a revolving wheel of buckets,
* like Nettys HashedWheelTimer, which it advances at a fixed tick rate and
* dispatches tasks it finds in the current bucket to their respective
* ExecutionContexts. The tasks are held in TaskHolders, which upon
* cancellation null out their reference to the actual task, leaving only this
* shell to be cleaned up when the wheel reaches that bucket next time. This
* enables the use of a simple linked list to chain the TaskHolders off the
* wheel.
*
* Also noteworthy is that this scheduler does not obtain a current time stamp
* when scheduling single-shot tasks, instead it always rounds up the task
* delay to a full multiple of the TickDuration. This means that tasks are
* scheduled possibly one tick later than they could be (if checking that
* now() + delay &lt;= nextTick were done).
*/
class LightArrayRevolverScheduler(config: Config,
log: LoggingAdapter,
threadFactory: ThreadFactory)
extends Scheduler with Closeable {
import Helpers.Requiring
import Helpers.ConfigOps
val WheelSize =
config.getInt("akka.scheduler.ticks-per-wheel")
.requiring(ticks (ticks & (ticks - 1)) == 0, "ticks-per-wheel must be a power of 2")
val TickDuration =
config.getMillisDuration("akka.scheduler.tick-duration")
.requiring(_ >= 10.millis || !Helpers.isWindows, "minimum supported akka.scheduler.tick-duration on Windows is 10ms")
.requiring(_ >= 1.millis, "minimum supported akka.scheduler.tick-duration is 1ms")
val ShutdownTimeout = config.getMillisDuration("akka.scheduler.shutdown-timeout")
import LightArrayRevolverScheduler._
private val oneNs = Duration.fromNanos(1l)
private def roundUp(d: FiniteDuration): FiniteDuration =
try {
((d + TickDuration - oneNs) / TickDuration).toLong * TickDuration
} catch {
case _: IllegalArgumentException d // rounding up Long.MaxValue.nanos overflows
}
/**
* Clock implementation is replaceable (for testing); the implementation must
* return a monotonically increasing series of Long nanoseconds.
*/
protected def clock(): Long = System.nanoTime
/**
* Overridable for tests
*/
protected def getShutdownTimeout: FiniteDuration = ShutdownTimeout
/**
* Overridable for tests
*/
protected def waitNanos(nanos: Long): Unit = {
// see http://www.javamex.com/tutorials/threads/sleep_issues.shtml
val sleepMs = if (Helpers.isWindows) (nanos + 4999999) / 10000000 * 10 else (nanos + 999999) / 1000000
try Thread.sleep(sleepMs) catch {
case _: InterruptedException Thread.currentThread.interrupt() // we got woken up
}
}
override def schedule(initialDelay: FiniteDuration,
delay: FiniteDuration,
runnable: Runnable)(implicit executor: ExecutionContext): Cancellable = {
checkMaxDelay(roundUp(delay).toNanos)
val preparedEC = executor.prepare()
try new AtomicReference[Cancellable](InitialRepeatMarker) with Cancellable { self
compareAndSet(InitialRepeatMarker, schedule(
preparedEC,
new AtomicLong(clock() + initialDelay.toNanos) with Runnable {
override def run(): Unit = {
try {
runnable.run()
val driftNanos = clock() - getAndAdd(delay.toNanos)
if (self.get != null)
swap(schedule(preparedEC, this, Duration.fromNanos(Math.max(delay.toNanos - driftNanos, 1))))
} catch {
case _: SchedulerException // ignore failure to enqueue or terminated target actor
}
}
}, roundUp(initialDelay)))
@tailrec private def swap(c: Cancellable): Unit = {
get match {
case null if (c != null) c.cancel()
case old if (!compareAndSet(old, c)) swap(c)
}
}
@tailrec final def cancel(): Boolean = {
get match {
case null false
case c
if (c.cancel()) compareAndSet(c, null)
else compareAndSet(c, null) || cancel()
}
}
override def isCancelled: Boolean = get == null
} catch {
case SchedulerException(msg) throw new IllegalStateException(msg)
}
}
override def scheduleOnce(delay: FiniteDuration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable =
try schedule(executor.prepare(), runnable, roundUp(delay))
catch {
case SchedulerException(msg) throw new IllegalStateException(msg)
}
/**
* Shutdown the scheduler. All scheduled task will be executed when the scheduler closed,
* i.e. the task may execute before its timeout.
*/
override def close(): Unit = Await.result(stop(), getShutdownTimeout) foreach {
task
try task.run() catch {
case e: InterruptedException throw e
case _: SchedulerException // ignore terminated actors
case NonFatal(e) log.error(e, "exception while executing timer task")
}
}
override val maxFrequency: Double = 1.second / TickDuration
/*
* BELOW IS THE ACTUAL TIMER IMPLEMENTATION
*/
private val start = clock()
private val tickNanos = TickDuration.toNanos
private val wheelMask = WheelSize - 1
private val queue = new TaskQueue
private def schedule(ec: ExecutionContext, r: Runnable, delay: FiniteDuration): TimerTask =
if (delay <= Duration.Zero) {
if (stopped.get != null) throw new SchedulerException("cannot enqueue after timer shutdown")
ec.execute(r)
NotCancellable
} else if (stopped.get != null) {
throw new SchedulerException("cannot enqueue after timer shutdown")
} else {
val delayNanos = delay.toNanos
checkMaxDelay(delayNanos)
val ticks = (delayNanos / tickNanos).toInt
val task = new TaskHolder(r, ticks, ec)
queue.add(task)
if (stopped.get != null && task.cancel())
throw new SchedulerException("cannot enqueue after timer shutdown")
task
}
private def checkMaxDelay(delayNanos: Long): Unit =
if (delayNanos / tickNanos > Int.MaxValue)
// 1 second margin in the error message due to rounding
throw new IllegalArgumentException(s"Task scheduled with [${delayNanos.nanos.toSeconds}] seconds delay, " +
s"which is too far in future, maximum delay is [${(tickNanos * Int.MaxValue).nanos.toSeconds - 1}] seconds")
private val stopped = new AtomicReference[Promise[immutable.Seq[TimerTask]]]
private def stop(): Future[immutable.Seq[TimerTask]] = {
val p = Promise[immutable.Seq[TimerTask]]()
if (stopped.compareAndSet(null, p)) {
// Interrupting the timer thread to make it shut down faster is not good since
// it could be in the middle of executing the scheduled tasks, which might not
// respond well to being interrupted.
// Instead we just wait one more tick for it to finish.
p.future
} else Future.successful(Nil)
}
@volatile private var timerThread: Thread = threadFactory.newThread(new Runnable {
var tick = 0
val wheel = Array.fill(WheelSize)(new TaskQueue)
private def clearAll(): immutable.Seq[TimerTask] = {
@tailrec def collect(q: TaskQueue, acc: Vector[TimerTask]): Vector[TimerTask] = {
q.poll() match {
case null acc
case x collect(q, acc :+ x)
}
}
((0 until WheelSize) flatMap (i collect(wheel(i), Vector.empty))) ++ collect(queue, Vector.empty)
}
@tailrec
private def checkQueue(time: Long): Unit = queue.pollNode() match {
case null ()
case node
node.value.ticks match {
case 0 node.value.executeTask()
case ticks
val futureTick = ((
time - start + // calculate the nanos since timer start
(ticks * tickNanos) + // adding the desired delay
tickNanos - 1 // rounding up
) / tickNanos).toInt // and converting to slot number
// tick is an Int that will wrap around, but toInt of futureTick gives us modulo operations
// and the difference (offset) will be correct in any case
val offset = futureTick - tick
val bucket = futureTick & wheelMask
node.value.ticks = offset
wheel(bucket).addNode(node)
}
checkQueue(time)
}
override final def run =
try nextTick()
catch {
case t: Throwable
log.error(t, "exception on LARS timer thread")
stopped.get match {
case null
val thread = threadFactory.newThread(this)
log.info("starting new LARS thread")
try thread.start()
catch {
case e: Throwable
log.error(e, "LARS cannot start new thread, ships going down!")
stopped.set(Promise successful Nil)
clearAll()
}
timerThread = thread
case p
assert(stopped.compareAndSet(p, Promise successful Nil), "Stop signal violated in LARS")
p success clearAll()
}
throw t
}
@tailrec final def nextTick(): Unit = {
val time = clock()
val sleepTime = start + (tick * tickNanos) - time
if (sleepTime > 0) {
// check the queue before taking a nap
checkQueue(time)
waitNanos(sleepTime)
} else {
val bucket = tick & wheelMask
val tasks = wheel(bucket)
val putBack = new TaskQueue
@tailrec def executeBucket(): Unit = tasks.pollNode() match {
case null ()
case node
val task = node.value
if (!task.isCancelled) {
if (task.ticks >= WheelSize) {
task.ticks -= WheelSize
putBack.addNode(node)
} else task.executeTask()
}
executeBucket()
}
executeBucket()
wheel(bucket) = putBack
tick += 1
}
stopped.get match {
case null nextTick()
case p
assert(stopped.compareAndSet(p, Promise successful Nil), "Stop signal violated in LARS")
p success clearAll()
}
}
})
timerThread.start()
}
object LightArrayRevolverScheduler {
private[this] val taskOffset = unsafe.objectFieldOffset(classOf[TaskHolder].getDeclaredField("task"))
private class TaskQueue extends AbstractNodeQueue[TaskHolder]
/**
* INTERNAL API
*/
protected[actor] trait TimerTask extends Runnable with Cancellable
/**
* INTERNAL API
*/
protected[actor] class TaskHolder(@volatile var task: Runnable, var ticks: Int, executionContext: ExecutionContext)
extends TimerTask {
@tailrec
private final def extractTask(replaceWith: Runnable): Runnable =
task match {
case t @ (ExecutedTask | CancelledTask) t
case x if (unsafe.compareAndSwapObject(this, taskOffset, x, replaceWith)) x else extractTask(replaceWith)
}
private[akka] final def executeTask(): Boolean = extractTask(ExecutedTask) match {
case ExecutedTask | CancelledTask false
case other
try {
executionContext execute other
true
} catch {
case _: InterruptedException { Thread.currentThread.interrupt(); false }
case NonFatal(e) { executionContext.reportFailure(e); false }
}
}
// This should only be called in execDirectly
override def run(): Unit = extractTask(ExecutedTask).run()
override def cancel(): Boolean = extractTask(CancelledTask) match {
case ExecutedTask | CancelledTask false
case _ true
}
override def isCancelled: Boolean = task eq CancelledTask
}
private[this] val CancelledTask = new Runnable { def run = () }
private[this] val ExecutedTask = new Runnable { def run = () }
private val NotCancellable: TimerTask = new TimerTask {
def cancel(): Boolean = false
def isCancelled: Boolean = false
def run(): Unit = ()
}
private val InitialRepeatMarker: Cancellable = new Cancellable {
def cancel(): Boolean = false
def isCancelled: Boolean = false
}
}

View file

@ -4,7 +4,7 @@
package akka.pattern
import scala.concurrent.duration.FiniteDuration
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.DeadLetterSuppression

View file

@ -4,7 +4,7 @@
package akka.routing
import scala.collection.immutable
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import akka.actor.ActorContext
import akka.actor.Props
import akka.dispatch.Dispatchers

View file

@ -5,7 +5,7 @@ package akka.routing
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import com.typesafe.config.Config
import akka.actor.ActorCell
import akka.actor.ActorRefWithCell

View file

@ -13,7 +13,7 @@ import akka.cluster.Member
import akka.cluster.Cluster
import scala.collection.immutable
import akka.cluster.MemberStatus
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import akka.actor.Terminated
import akka.actor.DeadLetterSuppression

View file

@ -7,7 +7,7 @@ import java.util.Arrays
import java.util.concurrent.atomic.AtomicReference
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import com.typesafe.config.Config
import akka.actor.Actor
import akka.actor.ActorSystem

View file

@ -7,7 +7,7 @@ package akka.cluster.metrics
import language.postfixOps
import scala.concurrent.duration._
import akka.testkit.{ LongRunningTest, AkkaSpec }
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class EWMASpec extends AkkaSpec(MetricsConfig.defaultEnabled) with MetricsCollectorFactory {

View file

@ -7,7 +7,7 @@ package akka.cluster.pubsub
import scala.collection.immutable
import scala.collection.immutable.Set
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import java.net.URLEncoder
import java.net.URLDecoder
import akka.actor.Actor

View file

@ -7,7 +7,7 @@ import language.existentials
import language.postfixOps
import scala.collection.immutable
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import scala.util.control.NonFatal
import akka.actor._
import akka.actor.SupervisorStrategy.Stop

View file

@ -13,7 +13,7 @@ import java.lang.reflect.InvocationTargetException
import java.lang.reflect.Method
import scala.collection.immutable
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import scala.util.{ Try, Success, Failure }
import akka.ConfigurationException
import akka.actor.Actor

View file

@ -10,7 +10,7 @@ import java.util.concurrent.atomic.AtomicReference
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import com.typesafe.config.Config

View file

@ -10,7 +10,7 @@ import language.postfixOps
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import java.util.concurrent.atomic.AtomicReference
import org.scalatest.BeforeAndAfterEach
import com.typesafe.config.Config

View file

@ -10,7 +10,7 @@ import akka.remote.transport.ThrottlerTransportAdapter.Direction
import scala.concurrent.duration._
import akka.testkit._
import akka.testkit.TestEvent._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import akka.remote.testconductor.RoleName
import akka.actor.Props
import akka.actor.Actor

View file

@ -11,7 +11,7 @@ import scala.concurrent.duration._
import scala.collection.immutable
import akka.remote.FailureDetector
import akka.remote.DefaultFailureDetectorRegistry
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
object ClusterHeartbeatSenderStateSpec {
class FailureDetectorStub extends FailureDetector {

View file

@ -9,7 +9,7 @@ package akka.cluster
import language.postfixOps
import scala.concurrent.duration._
import akka.testkit.{ LongRunningTest, AkkaSpec }
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class EWMASpec extends AkkaSpec(MetricsEnabledSpec.config) with MetricsCollectorFactory {

View file

@ -10,7 +10,7 @@ import scala.collection.immutable.Queue
import scala.collection.mutable
import scala.concurrent.duration._
import scala.concurrent.duration.FiniteDuration
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import scala.util.Failure
import scala.util.Success
import scala.util.Try

View file

@ -4,7 +4,7 @@
package akka.cluster.ddata
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import akka.cluster.Cluster
import akka.remote.testconductor.RoleName

View file

@ -4,7 +4,7 @@
package akka.cluster.ddata
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorSystem

View file

@ -6,7 +6,7 @@ package docs.ddata;
//#data-bot
import static java.util.concurrent.TimeUnit.SECONDS;
import scala.concurrent.duration.Duration;
import scala.concurrent.forkjoin.ThreadLocalRandom;
import java.util.concurrent.ThreadLocalRandom;
import akka.actor.AbstractActor;
import akka.actor.ActorRef;

View file

@ -18,7 +18,7 @@ import org.junit.Test;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import scala.concurrent.duration.FiniteDuration;
import scala.concurrent.forkjoin.ThreadLocalRandom;
import java.util.concurrent.ThreadLocalRandom;
import akka.actor.Actor;
import akka.actor.ActorLogging;

View file

@ -4,7 +4,7 @@
package docs.ddata
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.cluster.Cluster
@ -47,7 +47,7 @@ object DistributedDataDocSpec {
"""
//#data-bot
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.cluster.Cluster

View file

@ -41,7 +41,7 @@ akka.actor.deployment {
//#routing-logic
import scala.collection.immutable
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import akka.routing.RoundRobinRoutingLogic
import akka.routing.RoutingLogic
import akka.routing.Routee

View file

@ -5,7 +5,7 @@
package akka.persistence
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import scala.language.postfixOps
import com.typesafe.config.ConfigFactory

View file

@ -6,7 +6,7 @@ package akka.persistence.journal.chaos
import scala.collection.immutable
import scala.concurrent.Future
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import akka.persistence._
import akka.persistence.journal.AsyncWriteJournal
import akka.persistence.journal.inmem.InmemMessages

View file

@ -3,7 +3,7 @@
*/
package akka.remote
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import akka.actor.ActorSystem
import akka.actor.ExtendedActorSystem
import akka.actor.Extension

View file

@ -11,7 +11,7 @@ import akka.remote.transport.AssociationHandle.{ HandleEvent, HandleEventListene
import akka.remote.transport.Transport._
import akka.util.ByteString
import java.util.concurrent.ConcurrentHashMap
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import scala.concurrent.{ Future, Promise }
import scala.util.control.NoStackTrace
import scala.util.Try

View file

@ -5,7 +5,7 @@ package akka.remote
import akka.testkit.AkkaSpec
import scala.annotation.tailrec
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
object AckedDeliverySpec {

View file

@ -16,7 +16,7 @@ import java.io.NotSerializableException
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import akka.testkit.SocketUtil.temporaryServerAddress
object RemotingSpec {

View file

@ -9,7 +9,7 @@ import java.util.concurrent.TimeUnit;
import sample.cluster.stats.StatsMessages.JobFailed;
import sample.cluster.stats.StatsMessages.StatsJob;
import sample.cluster.stats.StatsMessages.StatsResult;
import scala.concurrent.forkjoin.ThreadLocalRandom;
import java.util.concurrent.ThreadLocalRandom;
import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import akka.actor.ActorSelection;

View file

@ -1,7 +1,7 @@
package sample.cluster.stats
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import com.typesafe.config.ConfigFactory
import akka.actor.Actor
import akka.actor.ActorSystem

View file

@ -6,7 +6,7 @@ package sample.redelivery
import akka.actor._
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.ThreadLocalRandom
import java.util.UUID
object SimpleOrderedRedeliverer {