2012-01-24 11:59:57 +01:00
|
|
|
/**
|
2012-05-21 16:45:15 +02:00
|
|
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
2009-08-17 20:46:05 +02:00
|
|
|
*/
|
2012-01-24 11:59:57 +01:00
|
|
|
|
2010-10-26 12:49:25 +02:00
|
|
|
package akka.actor
|
2009-08-17 18:42:41 +02:00
|
|
|
|
2012-06-29 13:33:20 +02:00
|
|
|
import scala.concurrent.util.Duration
|
2012-05-20 15:56:52 +02:00
|
|
|
import akka.util.internal.{ TimerTask, HashedWheelTimer, Timeout ⇒ HWTimeout, Timer }
|
2012-01-17 09:34:34 +01:00
|
|
|
import akka.event.LoggingAdapter
|
|
|
|
|
import akka.dispatch.MessageDispatcher
|
|
|
|
|
import java.io.Closeable
|
2012-05-16 16:16:31 +02:00
|
|
|
import java.util.concurrent.atomic.AtomicReference
|
|
|
|
|
import scala.annotation.tailrec
|
2012-05-20 15:56:52 +02:00
|
|
|
import akka.util.internal._
|
2012-08-08 15:57:30 +02:00
|
|
|
import concurrent.ExecutionContext
|
2012-09-18 09:58:30 +02:00
|
|
|
import scala.concurrent.util.FiniteDuration
|
2012-01-17 09:34:34 +01:00
|
|
|
|
2011-12-13 01:44:18 +01:00
|
|
|
//#scheduler
|
2011-12-06 16:27:10 +01:00
|
|
|
/**
|
|
|
|
|
* An Akka scheduler service. This one needs one special behavior: if
|
|
|
|
|
* Closeable, it MUST execute all outstanding tasks upon .close() in order
|
|
|
|
|
* to properly shutdown all dispatchers.
|
|
|
|
|
*
|
|
|
|
|
* Furthermore, this timer service MUST throw IllegalStateException if it
|
|
|
|
|
* cannot schedule a task. Once scheduled, the task MUST be executed. If
|
|
|
|
|
* executed upon close(), the task may execute before its timeout.
|
|
|
|
|
*/
|
2011-11-23 11:07:16 +01:00
|
|
|
trait Scheduler {
|
2011-11-23 15:15:44 +01:00
|
|
|
/**
|
2011-12-15 14:26:17 +01:00
|
|
|
* Schedules a message to be sent repeatedly with an initial delay and
|
|
|
|
|
* frequency. E.g. if you would like a message to be sent immediately and
|
2012-01-18 13:26:11 +01:00
|
|
|
* thereafter every 500ms you would set delay=Duration.Zero and
|
2012-09-18 18:17:44 +02:00
|
|
|
* interval=Duration(500, TimeUnit.MILLISECONDS)
|
2011-12-13 01:44:18 +01:00
|
|
|
*
|
|
|
|
|
* Java & Scala API
|
2011-11-23 15:15:44 +01:00
|
|
|
*/
|
2011-12-15 14:26:17 +01:00
|
|
|
def schedule(
|
2012-09-18 09:58:30 +02:00
|
|
|
initialDelay: FiniteDuration,
|
2012-09-18 18:17:44 +02:00
|
|
|
interval: FiniteDuration,
|
2011-12-15 14:26:17 +01:00
|
|
|
receiver: ActorRef,
|
2012-08-08 15:57:30 +02:00
|
|
|
message: Any)(implicit executor: ExecutionContext): Cancellable
|
2011-11-23 15:15:44 +01:00
|
|
|
|
|
|
|
|
/**
|
2011-12-15 14:26:17 +01:00
|
|
|
* Schedules a function to be run repeatedly with an initial delay and a
|
|
|
|
|
* frequency. E.g. if you would like the function to be run after 2 seconds
|
|
|
|
|
* and thereafter every 100ms you would set delay = Duration(2, TimeUnit.SECONDS)
|
2012-09-18 18:17:44 +02:00
|
|
|
* and interval = Duration(100, TimeUnit.MILLISECONDS)
|
2011-12-13 01:44:18 +01:00
|
|
|
*
|
|
|
|
|
* Scala API
|
2011-11-23 15:15:44 +01:00
|
|
|
*/
|
2011-12-15 14:26:17 +01:00
|
|
|
def schedule(
|
2012-10-01 20:35:19 +02:00
|
|
|
initialDelay: FiniteDuration,
|
|
|
|
|
interval: FiniteDuration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable
|
2011-11-23 15:15:44 +01:00
|
|
|
|
2011-12-14 00:06:36 +01:00
|
|
|
/**
|
2011-12-15 14:26:17 +01:00
|
|
|
* Schedules a function to be run repeatedly with an initial delay and
|
|
|
|
|
* a frequency. E.g. if you would like the function to be run after 2
|
|
|
|
|
* seconds and thereafter every 100ms you would set delay = Duration(2,
|
2012-09-18 18:17:44 +02:00
|
|
|
* TimeUnit.SECONDS) and interval = Duration(100, TimeUnit.MILLISECONDS)
|
2011-12-14 00:06:36 +01:00
|
|
|
*
|
|
|
|
|
* Java API
|
|
|
|
|
*/
|
2011-12-15 14:26:17 +01:00
|
|
|
def schedule(
|
2012-10-01 20:35:19 +02:00
|
|
|
initialDelay: FiniteDuration,
|
|
|
|
|
interval: FiniteDuration,
|
|
|
|
|
runnable: Runnable)(implicit executor: ExecutionContext): Cancellable
|
2011-12-14 00:06:36 +01:00
|
|
|
|
2011-11-23 15:15:44 +01:00
|
|
|
/**
|
2011-12-15 14:26:17 +01:00
|
|
|
* Schedules a Runnable to be run once with a delay, i.e. a time period that
|
|
|
|
|
* has to pass before the runnable is executed.
|
2011-12-13 01:44:18 +01:00
|
|
|
*
|
|
|
|
|
* Java & Scala API
|
2011-11-23 15:15:44 +01:00
|
|
|
*/
|
2012-10-01 20:35:19 +02:00
|
|
|
def scheduleOnce(
|
|
|
|
|
delay: FiniteDuration,
|
|
|
|
|
runnable: Runnable)(implicit executor: ExecutionContext): Cancellable
|
2011-11-23 15:15:44 +01:00
|
|
|
|
|
|
|
|
/**
|
2011-12-15 14:26:17 +01:00
|
|
|
* Schedules a message to be sent once with a delay, i.e. a time period that has
|
|
|
|
|
* to pass before the message is sent.
|
2011-12-13 01:44:18 +01:00
|
|
|
*
|
|
|
|
|
* Java & Scala API
|
2011-11-23 15:15:44 +01:00
|
|
|
*/
|
2012-10-01 20:35:19 +02:00
|
|
|
def scheduleOnce(
|
|
|
|
|
delay: FiniteDuration,
|
|
|
|
|
receiver: ActorRef,
|
|
|
|
|
message: Any)(implicit executor: ExecutionContext): Cancellable
|
2011-11-23 15:15:44 +01:00
|
|
|
|
|
|
|
|
/**
|
2011-12-15 14:26:17 +01:00
|
|
|
* Schedules a function to be run once with a delay, i.e. a time period that has
|
|
|
|
|
* to pass before the function is run.
|
2011-12-13 01:44:18 +01:00
|
|
|
*
|
|
|
|
|
* Scala API
|
2011-11-23 15:15:44 +01:00
|
|
|
*/
|
2012-10-01 20:35:19 +02:00
|
|
|
def scheduleOnce(
|
|
|
|
|
delay: FiniteDuration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable
|
2011-10-17 18:34:34 +02:00
|
|
|
}
|
2011-12-13 01:44:18 +01:00
|
|
|
//#scheduler
|
2011-10-17 18:34:34 +02:00
|
|
|
|
2011-12-13 01:44:18 +01:00
|
|
|
//#cancellable
|
|
|
|
|
/**
|
|
|
|
|
* Signifies something that can be cancelled
|
|
|
|
|
* There is no strict guarantee that the implementation is thread-safe,
|
|
|
|
|
* but it should be good practice to make it so.
|
|
|
|
|
*/
|
2011-11-10 11:53:36 +01:00
|
|
|
trait Cancellable {
|
2011-11-23 15:15:44 +01:00
|
|
|
/**
|
2011-12-13 01:44:18 +01:00
|
|
|
* Cancels this Cancellable
|
|
|
|
|
*
|
|
|
|
|
* Java & Scala API
|
2011-11-23 15:15:44 +01:00
|
|
|
*/
|
2011-11-10 11:53:36 +01:00
|
|
|
def cancel(): Unit
|
2011-11-23 15:15:44 +01:00
|
|
|
|
|
|
|
|
/**
|
2011-12-13 01:44:18 +01:00
|
|
|
* Returns whether this Cancellable has been cancelled
|
|
|
|
|
*
|
|
|
|
|
* Java & Scala API
|
2011-11-23 15:15:44 +01:00
|
|
|
*/
|
2011-11-10 11:53:36 +01:00
|
|
|
def isCancelled: Boolean
|
2011-12-13 01:44:18 +01:00
|
|
|
}
|
2011-12-15 14:26:17 +01:00
|
|
|
//#cancellable
|
2012-01-17 09:34:34 +01:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Scheduled tasks (Runnable and functions) are executed with the supplied dispatcher.
|
|
|
|
|
* Note that dispatcher is by-name parameter, because dispatcher might not be initialized
|
|
|
|
|
* when the scheduler is created.
|
|
|
|
|
*
|
|
|
|
|
* The HashedWheelTimer used by this class MUST throw an IllegalStateException
|
|
|
|
|
* if it does not enqueue a task. Once a task is queued, it MUST be executed or
|
|
|
|
|
* returned from stop().
|
|
|
|
|
*/
|
2012-08-08 15:57:30 +02:00
|
|
|
class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, log: LoggingAdapter) extends Scheduler with Closeable {
|
2012-09-18 09:58:30 +02:00
|
|
|
override def schedule(initialDelay: FiniteDuration,
|
|
|
|
|
delay: FiniteDuration,
|
2012-08-08 15:57:30 +02:00
|
|
|
receiver: ActorRef,
|
|
|
|
|
message: Any)(implicit executor: ExecutionContext): Cancellable = {
|
2012-01-17 09:34:34 +01:00
|
|
|
val continuousCancellable = new ContinuousCancellable
|
2012-05-10 23:55:11 +02:00
|
|
|
continuousCancellable.init(
|
|
|
|
|
hashedWheelTimer.newTimeout(
|
|
|
|
|
new TimerTask with ContinuousScheduling {
|
|
|
|
|
def run(timeout: HWTimeout) {
|
2012-08-08 15:57:30 +02:00
|
|
|
executor execute new Runnable {
|
|
|
|
|
override def run = {
|
|
|
|
|
receiver ! message
|
|
|
|
|
// Check if the receiver is still alive and kicking before reschedule the task
|
|
|
|
|
if (receiver.isTerminated) log.debug("Could not reschedule message to be sent because receiving actor {} has been terminated.", receiver)
|
|
|
|
|
else scheduleNext(timeout, delay, continuousCancellable)
|
|
|
|
|
}
|
|
|
|
|
}
|
2012-05-10 23:55:11 +02:00
|
|
|
}
|
|
|
|
|
},
|
|
|
|
|
initialDelay))
|
2012-01-17 09:34:34 +01:00
|
|
|
}
|
|
|
|
|
|
2012-09-18 09:58:30 +02:00
|
|
|
override def schedule(initialDelay: FiniteDuration,
|
|
|
|
|
delay: FiniteDuration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable =
|
2012-08-08 15:57:30 +02:00
|
|
|
schedule(initialDelay, delay, new Runnable { override def run = f })
|
2012-01-17 09:34:34 +01:00
|
|
|
|
2012-09-18 09:58:30 +02:00
|
|
|
override def schedule(initialDelay: FiniteDuration,
|
|
|
|
|
delay: FiniteDuration,
|
2012-08-08 15:57:30 +02:00
|
|
|
runnable: Runnable)(implicit executor: ExecutionContext): Cancellable = {
|
2012-01-17 09:34:34 +01:00
|
|
|
val continuousCancellable = new ContinuousCancellable
|
2012-05-10 23:55:11 +02:00
|
|
|
continuousCancellable.init(
|
|
|
|
|
hashedWheelTimer.newTimeout(
|
|
|
|
|
new TimerTask with ContinuousScheduling {
|
2012-08-08 15:57:30 +02:00
|
|
|
override def run(timeout: HWTimeout): Unit = executor.execute(new Runnable {
|
|
|
|
|
override def run = {
|
|
|
|
|
runnable.run()
|
|
|
|
|
scheduleNext(timeout, delay, continuousCancellable)
|
|
|
|
|
}
|
|
|
|
|
})
|
2012-05-10 23:55:11 +02:00
|
|
|
},
|
|
|
|
|
initialDelay))
|
2012-01-17 09:34:34 +01:00
|
|
|
}
|
|
|
|
|
|
2012-09-18 09:58:30 +02:00
|
|
|
override def scheduleOnce(delay: FiniteDuration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable =
|
2012-05-10 23:55:11 +02:00
|
|
|
new DefaultCancellable(
|
|
|
|
|
hashedWheelTimer.newTimeout(
|
2012-08-08 15:57:30 +02:00
|
|
|
new TimerTask() { def run(timeout: HWTimeout): Unit = executor.execute(runnable) },
|
2012-05-10 23:55:11 +02:00
|
|
|
delay))
|
2012-01-17 09:34:34 +01:00
|
|
|
|
2012-09-18 09:58:30 +02:00
|
|
|
override def scheduleOnce(delay: FiniteDuration, receiver: ActorRef, message: Any)(implicit executor: ExecutionContext): Cancellable =
|
2012-08-08 15:57:30 +02:00
|
|
|
scheduleOnce(delay, new Runnable { override def run = receiver ! message })
|
2012-01-17 09:34:34 +01:00
|
|
|
|
2012-09-18 09:58:30 +02:00
|
|
|
override def scheduleOnce(delay: FiniteDuration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable =
|
2012-08-08 15:57:30 +02:00
|
|
|
scheduleOnce(delay, new Runnable { override def run = f })
|
2012-01-17 09:34:34 +01:00
|
|
|
|
|
|
|
|
private trait ContinuousScheduling { this: TimerTask ⇒
|
2012-09-18 09:58:30 +02:00
|
|
|
def scheduleNext(timeout: HWTimeout, delay: FiniteDuration, delegator: ContinuousCancellable) {
|
2012-05-16 16:16:31 +02:00
|
|
|
try delegator.swap(timeout.getTimer.newTimeout(this, delay)) catch { case _: IllegalStateException ⇒ } // stop recurring if timer is stopped
|
2012-01-17 09:34:34 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def execDirectly(t: HWTimeout): Unit = {
|
|
|
|
|
try t.getTask.run(t) catch {
|
|
|
|
|
case e: InterruptedException ⇒ throw e
|
|
|
|
|
case e: Exception ⇒ log.error(e, "exception while executing timer task")
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2012-05-16 16:28:43 +02:00
|
|
|
override def close(): Unit = {
|
2012-01-17 09:34:34 +01:00
|
|
|
import scala.collection.JavaConverters._
|
|
|
|
|
hashedWheelTimer.stop().asScala foreach execDirectly
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2012-05-16 16:22:59 +02:00
|
|
|
private[akka] object ContinuousCancellable {
|
|
|
|
|
val initial: HWTimeout = new HWTimeout {
|
|
|
|
|
override def getTimer: Timer = null
|
|
|
|
|
override def getTask: TimerTask = null
|
|
|
|
|
override def isExpired: Boolean = false
|
|
|
|
|
override def isCancelled: Boolean = false
|
|
|
|
|
override def cancel: Unit = ()
|
|
|
|
|
}
|
2012-05-21 14:50:49 +02:00
|
|
|
|
|
|
|
|
val cancelled: HWTimeout = new HWTimeout {
|
|
|
|
|
override def getTimer: Timer = null
|
|
|
|
|
override def getTask: TimerTask = null
|
|
|
|
|
override def isExpired: Boolean = false
|
|
|
|
|
override def isCancelled: Boolean = true
|
|
|
|
|
override def cancel: Unit = ()
|
|
|
|
|
}
|
2012-05-16 16:22:59 +02:00
|
|
|
}
|
2012-01-17 09:34:34 +01:00
|
|
|
/**
|
|
|
|
|
* Wrapper of a [[org.jboss.netty.akka.util.Timeout]] that delegates all
|
|
|
|
|
* methods. Needed to be able to cancel continuous tasks,
|
|
|
|
|
* since they create new Timeout for each tick.
|
|
|
|
|
*/
|
2012-05-16 16:22:59 +02:00
|
|
|
private[akka] class ContinuousCancellable extends AtomicReference[HWTimeout](ContinuousCancellable.initial) with Cancellable {
|
2012-05-10 23:55:11 +02:00
|
|
|
private[akka] def init(initialTimeout: HWTimeout): this.type = {
|
2012-05-16 16:22:59 +02:00
|
|
|
compareAndSet(ContinuousCancellable.initial, initialTimeout)
|
2012-05-10 23:55:11 +02:00
|
|
|
this
|
2012-01-17 09:34:34 +01:00
|
|
|
}
|
|
|
|
|
|
2012-05-16 16:16:31 +02:00
|
|
|
@tailrec private[akka] final def swap(newTimeout: HWTimeout): Unit = get match {
|
2012-05-21 14:50:49 +02:00
|
|
|
case some if some.isCancelled ⇒ try cancel() finally newTimeout.cancel()
|
2012-05-16 16:16:31 +02:00
|
|
|
case some ⇒ if (!compareAndSet(some, newTimeout)) swap(newTimeout)
|
2012-01-17 09:34:34 +01:00
|
|
|
}
|
|
|
|
|
|
2012-05-21 14:50:49 +02:00
|
|
|
def isCancelled(): Boolean = get().isCancelled()
|
|
|
|
|
def cancel(): Unit = getAndSet(ContinuousCancellable.cancelled).cancel()
|
2012-01-17 09:34:34 +01:00
|
|
|
}
|
|
|
|
|
|
2012-05-21 14:50:49 +02:00
|
|
|
private[akka] class DefaultCancellable(timeout: HWTimeout) extends AtomicReference[HWTimeout](timeout) with Cancellable {
|
|
|
|
|
override def cancel(): Unit = getAndSet(ContinuousCancellable.cancelled).cancel()
|
|
|
|
|
override def isCancelled: Boolean = get().isCancelled
|
2012-01-18 14:20:13 +01:00
|
|
|
}
|