scheduleWithFixedDelay vs scheduleAtFixedRate, #26910

* previous `schedule` method is trying to maintain a fixed average frequency
  over time, but that can result in undesired bursts of scheduled tasks after a long
  GC or if the JVM process has been suspended, same with all other periodic
  scheduled message sending via various Timer APIs
* most of the time "fixed delay" is more desirable
* we can't just change because it's too big behavioral change and some might
  depend on previous behavior
* deprecate the old `schedule` and introduce new `scheduleWithFixedDelay`
  and `scheduleAtFixedRate`, when fixing the deprecation warning users should
  make a concious decision of which behavior to use (scheduleWithFixedDelay in
  most cases)

* Streams
* SchedulerSpec
  * test both fixed delay and fixed rate
* TimerSpec
* FSM and PersistentFSM
* mima
* runnable as second parameter list, also in typed.Scheduler
* IllegalStateException vs SchedulerException
* deprecated annotations
* api and reference docs, all places
* migration guide
This commit is contained in:
Patrik Nordwall 2019-05-27 11:53:26 +02:00
parent 72cfc2485e
commit 10d32fceb9
99 changed files with 2285 additions and 909 deletions

View file

@ -47,7 +47,7 @@ class ManualTimerExampleSpec extends ScalaTestWithActorTestKit(ManualTime.config
val probe = TestProbe[Tock.type]()
val behavior = Behaviors.withTimers[Tick.type] { timer =>
timer.startPeriodicTimer("T", Tick, 10.millis)
timer.startTimerWithFixedDelay("T", Tick, 10.millis)
Behaviors.receiveMessage { _ =>
probe.ref ! Tock
Behaviors.same
@ -77,7 +77,7 @@ class ManualTimerExampleSpec extends ScalaTestWithActorTestKit(ManualTime.config
val interval = 10.millis
val behavior = Behaviors.withTimers[Command] { timer =>
timer.startPeriodicTimer("T", Tick(1), interval)
timer.startTimerWithFixedDelay("T", Tick(1), interval)
probe.ref ! Started
Behaviors.receiveMessage {
@ -86,7 +86,7 @@ class ManualTimerExampleSpec extends ScalaTestWithActorTestKit(ManualTime.config
Behaviors.same
case SlowThenBump(nextCount) =>
manualTime.timePasses(interval)
timer.startPeriodicTimer("T", Tick(nextCount), interval)
timer.startTimerWithFixedDelay("T", Tick(nextCount), interval)
probe.ref ! SlowThenBumpAck
Behaviors.same
}

View file

@ -225,7 +225,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
}
onTransition {
case "not-started" -> "started" =>
for (timerName <- timerNames) setTimer(timerName, (), 10 seconds, false)
for (timerName <- timerNames) startSingleTimer(timerName, (), 10 seconds)
}
onTermination {
case _ => {
@ -266,7 +266,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
startWith(1, null)
when(1) {
case Event("go", _) =>
setTimer("t", FSM.Shutdown, 1.5 seconds, false)
startSingleTimer("t", FSM.Shutdown, 1.5 seconds)
goto(2)
}
when(2) {

View file

@ -195,10 +195,10 @@ object FSMTimingSpec {
startWith(Initial, 0)
when(Initial) {
case Event(TestSingleTimer, _) =>
setTimer("tester", Tick, 500.millis.dilated, false)
startSingleTimer("tester", Tick, 500.millis.dilated)
goto(TestSingleTimer)
case Event(TestRepeatedTimer, _) =>
setTimer("tester", Tick, 100.millis.dilated, true)
startTimerWithFixedDelay("tester", Tick, 100.millis.dilated)
goto(TestRepeatedTimer).using(4)
case Event(TestStateTimeoutOverride, _) =>
goto(TestStateTimeout).forMax(Duration.Inf)
@ -214,12 +214,12 @@ object FSMTimingSpec {
goto(Initial)
}
onTransition {
case Initial -> TestSingleTimerResubmit => setTimer("blah", Tick, 500.millis.dilated)
case Initial -> TestSingleTimerResubmit => startSingleTimer("blah", Tick, 500.millis.dilated)
}
when(TestSingleTimerResubmit) {
case Event(Tick, _) =>
tester ! Tick
setTimer("blah", Tock, 500.millis.dilated)
startSingleTimer("blah", Tock, 500.millis.dilated)
stay()
case Event(Tock, _) =>
tester ! Tock
@ -227,11 +227,11 @@ object FSMTimingSpec {
}
when(TestCancelTimer) {
case Event(Tick, _) =>
setTimer("hallo", Tock, 1.milli.dilated)
startSingleTimer("hallo", Tock, 1.milli.dilated)
TestKit.awaitCond(context.asInstanceOf[ActorCell].mailbox.hasMessages, 1.second.dilated)
cancelTimer("hallo")
sender() ! Tick
setTimer("hallo", Tock, 500.millis.dilated)
startSingleTimer("hallo", Tock, 500.millis.dilated)
stay
case Event(Tock, _) =>
tester ! Tock
@ -254,7 +254,7 @@ object FSMTimingSpec {
// FSM is suspended after processing this message and resumed 500ms later
case Event(Tick, _) =>
suspend(self)
setTimer("named", Tock, 1.millis.dilated)
startSingleTimer("named", Tock, 1.millis.dilated)
TestKit.awaitCond(context.asInstanceOf[ActorCell].mailbox.hasMessages, 1.second.dilated)
stay.forMax(1.millis.dilated).replying(Tick)
case Event(Tock, _) =>

View file

@ -102,12 +102,10 @@ class ReceiveTimeoutSpec extends AkkaSpec {
}
}))
val ticks = system.scheduler.schedule(100.millis, 100.millis, new Runnable {
override def run() = {
timeoutActor ! TransparentTick
timeoutActor ! Identify(None)
}
})(system.dispatcher)
val ticks = system.scheduler.scheduleWithFixedDelay(100.millis, 100.millis) { () =>
timeoutActor ! TransparentTick
timeoutActor ! Identify(None)
}(system.dispatcher)
Await.ready(timeoutLatch, TestLatch.DefaultTimeout)
ticks.cancel()
@ -137,8 +135,8 @@ class ReceiveTimeoutSpec extends AkkaSpec {
val count = new AtomicInteger(0)
class ActorWithTimer() extends Actor with Timers {
timers.startPeriodicTimer("transparentTick", TransparentTick, 100.millis)
timers.startPeriodicTimer("identifyTick", Identify(None), 100.millis)
timers.startTimerWithFixedDelay("transparentTick", TransparentTick, 100.millis)
timers.startTimerWithFixedDelay("identifyTick", Identify(None), 100.millis)
context.setReceiveTimeout(1 second)
def receive: Receive = {

File diff suppressed because it is too large Load diff

View file

@ -468,7 +468,7 @@ object SupervisorHierarchySpec {
case Event(Init, _) =>
hierarchy = context.watch(
context.actorOf(Props(new Hierarchy(size, breadth, self, 0, random)).withDispatcher("hierarchy"), "head"))
setTimer("phase", StateTimeout, 5 seconds, false)
startSingleTimer("phase", StateTimeout, 5 seconds)
goto(Init)
}
@ -493,7 +493,7 @@ object SupervisorHierarchySpec {
idleChildren = children
activeChildren = children
// set timeout for completion of the whole test (i.e. including Finishing and Stopping)
setTimer("phase", StateTimeout, 90.seconds.dilated, false)
startSingleTimer("phase", StateTimeout, 90.seconds.dilated)
}
val workSchedule = 50.millis

View file

@ -38,7 +38,7 @@ object TimerSpec {
private var bumpCount = initial()
if (repeat)
timers.startPeriodicTimer("T", Tick(bumpCount), interval)
timers.startTimerWithFixedDelay("T", Tick(bumpCount), interval)
else
timers.startSingleTimer("T", Tick(bumpCount), interval)
@ -53,7 +53,7 @@ object TimerSpec {
def bump(): Unit = {
bumpCount += 1
timers.startPeriodicTimer("T", Tick(bumpCount), interval)
timers.startTimerWithFixedDelay("T", Tick(bumpCount), interval)
}
def autoReceive(): Unit = {
@ -104,19 +104,28 @@ object TimerSpec {
}
def bump(bumpCount: Int): State = {
setTimer("T", Tick(bumpCount + 1), interval, repeat)
if (repeat)
startTimerWithFixedDelay("T", Tick(bumpCount + 1), interval)
else
startSingleTimer("T", Tick(bumpCount + 1), interval)
stay.using(bumpCount + 1)
}
def autoReceive(): State = {
setTimer("A", PoisonPill, interval, repeat)
if (repeat)
startTimerWithFixedDelay("A", PoisonPill, interval)
else
startSingleTimer("A", PoisonPill, interval)
stay
}
{
val i = initial()
startWith(TheState, i)
setTimer("T", Tick(i), interval, repeat)
if (repeat)
startTimerWithFixedDelay("T", Tick(i), interval)
else
startSingleTimer("T", Tick(i), interval)
}
when(TheState) {

View file

@ -88,7 +88,7 @@ public class ActorCompile {
Behavior<MyMsg> b =
Behaviors.withTimers(
timers -> {
timers.startPeriodicTimer("key", new MyMsgB("tick"), Duration.ofSeconds(1));
timers.startTimerWithFixedDelay("key", new MyMsgB("tick"), Duration.ofSeconds(1));
return Behaviors.ignore();
});
}

View file

@ -13,7 +13,8 @@ class SchedulerSpec {
import system.executionContext
// verify a lambda works
system.scheduler.scheduleAtFixedRate(10.milliseconds, 10.milliseconds, () => system.log.info("Woho!"))
system.scheduler.scheduleWithFixedDelay(10.milliseconds, 10.milliseconds)(() => system.log.info("Woho!"))
system.scheduler.scheduleAtFixedRate(10.milliseconds, 10.milliseconds)(() => system.log.info("Woho!"))
system.scheduler.scheduleOnce(10.milliseconds, () => system.log.info("Woho!"))
}

View file

@ -49,7 +49,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
def target(monitor: ActorRef[Event], timer: TimerScheduler[Command], bumpCount: Int): Behavior[Command] = {
def bump(): Behavior[Command] = {
val nextCount = bumpCount + 1
timer.startPeriodicTimer("T", Tick(nextCount), interval)
timer.startTimerWithFixedDelay("T", Tick(nextCount), interval)
target(monitor, timer, nextCount)
}
@ -107,7 +107,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
"schedule repeated ticks" taggedAs TimingTest in {
val probe = TestProbe[Event]("evt")
val behv = Behaviors.withTimers[Command] { timer =>
timer.startPeriodicTimer("T", Tick(1), interval)
timer.startTimerWithFixedDelay("T", Tick(1), interval)
target(probe.ref, timer, 1)
}
@ -125,7 +125,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
"replace timer" taggedAs TimingTest in {
val probe = TestProbe[Event]("evt")
val behv = Behaviors.withTimers[Command] { timer =>
timer.startPeriodicTimer("T", Tick(1), interval)
timer.startTimerWithFixedDelay("T", Tick(1), interval)
target(probe.ref, timer, 1)
}
@ -145,7 +145,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
"cancel timer" taggedAs TimingTest in {
val probe = TestProbe[Event]("evt")
val behv = Behaviors.withTimers[Command] { timer =>
timer.startPeriodicTimer("T", Tick(1), interval)
timer.startTimerWithFixedDelay("T", Tick(1), interval)
target(probe.ref, timer, 1)
}
@ -170,7 +170,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
val startCounter = new AtomicInteger(0)
val behv = Behaviors
.supervise(Behaviors.withTimers[Command] { timer =>
timer.startPeriodicTimer("T", Tick(startCounter.incrementAndGet()), interval)
timer.startTimerWithFixedDelay("T", Tick(startCounter.incrementAndGet()), interval)
target(probe.ref, timer, 1)
})
.onFailure[Exception](SupervisorStrategy.restart)
@ -197,7 +197,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
val probe = TestProbe[Event]("evt")
val behv = Behaviors
.supervise(Behaviors.withTimers[Command] { timer =>
timer.startPeriodicTimer("T", Tick(1), interval)
timer.startTimerWithFixedDelay("T", Tick(1), interval)
target(probe.ref, timer, 1)
})
.onFailure[Exception](SupervisorStrategy.restart)
@ -226,7 +226,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
"cancel timers when stopped from exception" taggedAs TimingTest in {
val probe = TestProbe[Event]()
val behv = Behaviors.withTimers[Command] { timer =>
timer.startPeriodicTimer("T", Tick(1), interval)
timer.startTimerWithFixedDelay("T", Tick(1), interval)
target(probe.ref, timer, 1)
}
val ref = spawn(behv)
@ -239,7 +239,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
"cancel timers when stopped voluntarily" taggedAs TimingTest in {
val probe = TestProbe[Event]()
val behv = Behaviors.withTimers[Command] { timer =>
timer.startPeriodicTimer("T", Tick(1), interval)
timer.startTimerWithFixedDelay("T", Tick(1), interval)
target(probe.ref, timer, 1)
}
val ref = spawn(behv)
@ -250,9 +250,9 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
"allow for nested timers" in {
val probe = TestProbe[String]()
val ref = spawn(Behaviors.withTimers[String] { outerTimer =>
outerTimer.startPeriodicTimer("outer-key", "outer-message", 50.millis)
outerTimer.startTimerWithFixedDelay("outer-key", "outer-message", 50.millis)
Behaviors.withTimers { innerTimer =>
innerTimer.startPeriodicTimer("inner-key", "inner-message", 50.millis)
innerTimer.startTimerWithFixedDelay("inner-key", "inner-message", 50.millis)
Behaviors.receiveMessage { message =>
if (message == "stop") Behaviors.stopped
else {
@ -277,7 +277,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
"keep timers when behavior changes" in {
val probe = TestProbe[String]()
def newBehavior(n: Int): Behavior[String] = Behaviors.withTimers[String] { timers =>
timers.startPeriodicTimer(s"key${n}", s"message${n}", 50.milli)
timers.startTimerWithFixedDelay(s"key${n}", s"message${n}", 50.milli)
Behaviors.receiveMessage { message =>
if (message == "stop") Behaviors.stopped
else {
@ -303,7 +303,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
val probe = TestProbe[Any]()
val ref = spawn(Behaviors.withTimers[String] { timers =>
Behaviors.setup { _ =>
timers.startPeriodicTimer("test", "test", 250.millis)
timers.startTimerWithFixedDelay("test", "test", 250.millis)
Behaviors.receive { (context, message) =>
Behaviors.stopped(() => context.log.info(s"stopping"))
}

View file

@ -21,21 +21,18 @@ import scala.concurrent.duration.FiniteDuration
trait Scheduler {
/**
*
* Schedules a Runnable to be run once with a delay, i.e. a time period that
* Scala API: Schedules a Runnable to be run once with a delay, i.e. a time period that
* has to pass before the runnable is executed.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Note: For scheduling within actors `Behaviors.withTimers` or `ActorContext.scheduleOnce` should be preferred.
*
* Scala API
*/
def scheduleOnce(delay: FiniteDuration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable
/**
* Schedules a Runnable to be run once with a delay, i.e. a time period that
* Java API: Schedules a Runnable to be run once with a delay, i.e. a time period that
* has to pass before the runnable is executed.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
@ -43,20 +40,20 @@ trait Scheduler {
*
* Note: For scheduling within actors `Behaviors.withTimers` or `ActorContext.scheduleOnce` should be preferred.
*
* Java API
*/
def scheduleOnce(delay: java.time.Duration, runnable: Runnable, executor: ExecutionContext): Cancellable
/**
* Schedules a `Runnable` 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) and interval = Duration(100, TimeUnit.MILLISECONDS). If
* the execution of the runnable takes longer than the interval, the
* subsequent execution will start immediately after the prior one completes
* (there will be no overlap of executions of the runnable). In such cases,
* the actual execution interval will differ from the interval passed to this
* method.
* Scala API: Schedules a `Runnable` to be run repeatedly with an initial delay and
* a fixed `delay` between subsequent executions. 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)`
* and `interval=Duration(100, TimeUnit.MILLISECONDS)`.
*
* It will not compensate the delay between tasks if the execution takes a long time or if
* scheduling is delayed longer than specified for some reason. The delay between subsequent
* execution will always be (at least) the given `delay`. In the long run, the
* frequency of execution will generally be slightly lower than the reciprocal of the specified
* `delay`.
*
* If the `Runnable` throws an exception the repeated scheduling is aborted,
* i.e. the function will not be invoked any more.
@ -64,23 +61,24 @@ trait Scheduler {
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Note: For user scheduling needs `Behaviors.withTimers` should be preferred.
* Note: For scheduling within actors `Behaviors.withTimers` should be preferred.
*
* Scala API
*/
def scheduleAtFixedRate(initialDelay: FiniteDuration, interval: FiniteDuration, runnable: Runnable)(
def scheduleWithFixedDelay(initialDelay: FiniteDuration, delay: FiniteDuration)(runnable: Runnable)(
implicit executor: ExecutionContext): Cancellable
/**
* Schedules a `Runnable` 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 to `Duration.ofSeconds(2)`,
* and interval to `Duration.ofMillis(100)`. If
* the execution of the runnable takes longer than the interval, the
* subsequent execution will start immediately after the prior one completes
* (there will be no overlap of executions of the runnable). In such cases,
* the actual execution interval will differ from the interval passed to this
* method.
* Java API: Schedules a `Runnable` to be run repeatedly with an initial delay and
* a fixed `delay` between subsequent executions. E.g. if you would like the function to
* be run after 2 seconds and thereafter every 100ms you would set delay to `Duration.ofSeconds(2)`,
* and interval to `Duration.ofMillis(100)`.
*
* It will not compensate the delay between tasks if the execution takes a long time or if
* scheduling is delayed longer than specified for some reason. The delay between subsequent
* execution will always be (at least) the given `delay`.
*
* In the long run, the frequency of tasks will generally be slightly lower than
* the reciprocal of the specified `delay`.
*
* If the `Runnable` throws an exception the repeated scheduling is aborted,
* i.e. the function will not be invoked any more.
@ -88,9 +86,79 @@ trait Scheduler {
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Note: For user scheduling needs `Behaviors.withTimers` should be preferred.
* Note: For scheduling in actors `Behaviors.withTimers` should be preferred.
*/
def scheduleWithFixedDelay(
initialDelay: java.time.Duration,
delay: java.time.Duration,
runnable: Runnable,
executor: ExecutionContext): Cancellable
/**
* Scala API: Schedules a `Runnable` 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)`
* and `interval=Duration(100, TimeUnit.MILLISECONDS)`.
*
* Java API
* It will compensate the delay for a subsequent task if the previous tasks took
* too long to execute. In such cases, the actual execution interval will differ from
* the interval passed to the method.
*
* If the execution of the tasks takes longer than the `interval`, the subsequent
* execution will start immediately after the prior one completes (there will be
* no overlap of executions). This also has the consequence that after long garbage
* collection pauses or other reasons when the JVM was suspended all "missed" tasks
* will execute when the process wakes up again.
*
* In the long run, the frequency of execution will be exactly the reciprocal of the
* specified `interval`.
*
* Warning: `scheduleAtFixedRate` can result in bursts of scheduled tasks after long
* garbage collection pauses, which may in worst case cause undesired load on the system.
* Therefore `scheduleWithFixedDelay` is often preferred.
*
* If the `Runnable` throws an exception the repeated scheduling is aborted,
* i.e. the function will not be invoked any more.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Note: For scheduling within actors `Behaviors.withTimers` should be preferred.
*
*/
def scheduleAtFixedRate(initialDelay: FiniteDuration, interval: FiniteDuration)(runnable: Runnable)(
implicit executor: ExecutionContext): Cancellable
/**
* Java API: Schedules a `Runnable` 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 to `Duration.ofSeconds(2)`,
* and interval to `Duration.ofMillis(100)`.
*
* It will compensate the delay for a subsequent task if the previous tasks took
* too long to execute. In such cases, the actual execution interval will differ from
* the interval passed to the method.
*
* If the execution of the tasks takes longer than the `interval`, the subsequent
* execution will start immediately after the prior one completes (there will be
* no overlap of executions). This also has the consequence that after long garbage
* collection pauses or other reasons when the JVM was suspended all "missed" tasks
* will execute when the process wakes up again.
*
* In the long run, the frequency of execution will be exactly the reciprocal of the
* specified `interval`.
*
* Warning: `scheduleAtFixedRate` can result in bursts of scheduled tasks after long
* garbage collection pauses, which may in worst case cause undesired load on the system.
* Therefore `scheduleWithFixedDelay` is often preferred.
*
* If the `Runnable` throws an exception the repeated scheduling is aborted,
* i.e. the function will not be invoked any more.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Note: For scheduling in actors `Behaviors.withTimers` should be preferred.
*/
def scheduleAtFixedRate(
initialDelay: java.time.Duration,

View file

@ -5,6 +5,8 @@
package akka.actor.typed
package internal
import java.time.Duration
import scala.concurrent.duration.FiniteDuration
import akka.actor.Cancellable
@ -36,6 +38,18 @@ import akka.util.OptionVal
case _ => throw new IllegalArgumentException(s"timers not supported with [${ctx.getClass}]")
}
private sealed trait TimerMode {
def repeat: Boolean
}
private case object FixedRateMode extends TimerMode {
override def repeat: Boolean = true
}
private case object FixedDelayMode extends TimerMode {
override def repeat: Boolean = true
}
private case object SingleMode extends TimerMode {
override def repeat: Boolean = false
}
}
/**
@ -49,19 +63,31 @@ import akka.util.OptionVal
private var timers: Map[Any, Timer[T]] = Map.empty
private val timerGen = Iterator.from(1)
override def startTimerAtFixedRate(key: Any, msg: T, interval: FiniteDuration): Unit =
startTimer(key, msg, interval, FixedRateMode)
override def startTimerAtFixedRate(key: Any, msg: T, interval: Duration): Unit =
startTimerAtFixedRate(key, msg, interval.asScala)
override def startTimerWithFixedDelay(key: Any, msg: T, delay: FiniteDuration): Unit =
startTimer(key, msg, delay, FixedDelayMode)
override def startTimerWithFixedDelay(key: Any, msg: T, delay: Duration): Unit =
startTimerWithFixedDelay(key, msg, delay.asScala)
override def startPeriodicTimer(key: Any, msg: T, interval: FiniteDuration): Unit =
startTimer(key, msg, interval, repeat = true)
startTimer(key, msg, interval, FixedRateMode)
override def startPeriodicTimer(key: Any, msg: T, interval: java.time.Duration): Unit =
startPeriodicTimer(key, msg, interval.asScala)
override def startSingleTimer(key: Any, msg: T, delay: FiniteDuration): Unit =
startTimer(key, msg, delay, repeat = false)
startTimer(key, msg, delay, SingleMode)
def startSingleTimer(key: Any, msg: T, delay: java.time.Duration): Unit =
startSingleTimer(key, msg, delay.asScala)
private def startTimer(key: Any, msg: T, delay: FiniteDuration, repeat: Boolean): Unit = {
private def startTimer(key: Any, msg: T, delay: FiniteDuration, mode: TimerMode): Unit = {
timers.get(key) match {
case Some(t) => cancelTimer(t)
case None =>
@ -74,15 +100,19 @@ import akka.util.OptionVal
else
new TimerMsg(key, nextGen, this)
val task =
if (repeat)
ctx.system.scheduler.scheduleAtFixedRate(delay, delay, () => ctx.self.unsafeUpcast ! timerMsg)(
ExecutionContexts.sameThreadExecutionContext)
else
val task = mode match {
case SingleMode =>
ctx.system.scheduler
.scheduleOnce(delay, () => ctx.self.unsafeUpcast ! timerMsg)(ExecutionContexts.sameThreadExecutionContext)
case FixedDelayMode =>
ctx.system.scheduler.scheduleWithFixedDelay(delay, delay)(() => ctx.self.unsafeUpcast ! timerMsg)(
ExecutionContexts.sameThreadExecutionContext)
case FixedRateMode =>
ctx.system.scheduler.scheduleAtFixedRate(delay, delay)(() => ctx.self.unsafeUpcast ! timerMsg)(
ExecutionContexts.sameThreadExecutionContext)
}
val nextTimer = Timer(key, msg, repeat, nextGen, task)
val nextTimer = Timer(key, msg, mode.repeat, nextGen, task)
timers = timers.updated(key, nextTimer)
}

View file

@ -25,14 +25,26 @@ private[akka] final class SchedulerAdapter(private[akka] val untypedScheduler: a
override def scheduleOnce(delay: Duration, runnable: Runnable, executor: ExecutionContext): Cancellable =
untypedScheduler.scheduleOnce(delay, runnable)(executor)
override def scheduleAtFixedRate(initialDelay: FiniteDuration, interval: FiniteDuration, runnable: Runnable)(
override def scheduleWithFixedDelay(initialDelay: FiniteDuration, delay: FiniteDuration)(runnable: Runnable)(
implicit executor: ExecutionContext): Cancellable =
untypedScheduler.schedule(initialDelay, interval, runnable)
untypedScheduler.scheduleWithFixedDelay(initialDelay, delay)(runnable)
override def scheduleWithFixedDelay(
initialDelay: Duration,
delay: Duration,
runnable: Runnable,
executor: ExecutionContext): Cancellable =
untypedScheduler.scheduleWithFixedDelay(initialDelay, delay, runnable, executor)
override def scheduleAtFixedRate(initialDelay: FiniteDuration, interval: FiniteDuration)(runnable: Runnable)(
implicit executor: ExecutionContext): Cancellable =
untypedScheduler.scheduleAtFixedRate(initialDelay, interval)(runnable)
override def scheduleAtFixedRate(
initialDelay: Duration,
interval: Duration,
runnable: Runnable,
executor: ExecutionContext): Cancellable =
untypedScheduler.schedule(initialDelay, interval, runnable)(executor)
untypedScheduler.scheduleAtFixedRate(initialDelay, interval, runnable, executor)
}

View file

@ -18,14 +18,57 @@ import java.time.Duration
trait TimerScheduler[T] {
/**
* Start a periodic timer that will send `msg` to the `self` actor at
* a fixed `interval`.
* Schedules a message to be sent repeatedly to the `self` actor with a
* fixed `delay` between messages.
*
* It will not compensate the delay between messages if scheduling is delayed
* longer than specified for some reason. The delay between sending of subsequent
* messages will always be (at least) the given `delay`.
*
* In the long run, the frequency of messages will generally be slightly lower than
* the reciprocal of the specified `delay`.
*
* Each timer has a key and if a new timer with same key is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startTimerWithFixedDelay(key: Any, msg: T, delay: java.time.Duration): Unit
/**
* Schedules a message to be sent repeatedly to the `self` actor with a
* given frequency.
*
* It will compensate the delay for a subsequent message if the sending of previous
* message was delayed more than specified. In such cases, the actual message interval
* will differ from the interval passed to the method.
*
* If the execution is delayed longer than the `interval`, the subsequent message will
* be sent immediately after the prior one. This also has the consequence that after
* long garbage collection pauses or other reasons when the JVM was suspended all
* "missed" messages will be sent when the process wakes up again.
*
* In the long run, the frequency of messages will be exactly the reciprocal of the
* specified `interval`.
*
* Warning: `startTimerAtFixedRate` can result in bursts of scheduled messages after long
* garbage collection pauses, which may in worst case cause undesired load on the system.
* Therefore `startTimerWithFixedDelay` is often preferred.
*
* Each timer has a key and if a new timer with same key is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startTimerAtFixedRate(key: Any, msg: T, interval: java.time.Duration): Unit
/**
* Deprecated API: See [[TimerScheduler#startTimerWithFixedDelay]] or [[TimerScheduler#startTimerAtFixedRate]].
*/
@deprecated(
"Use startTimerWithFixedDelay or startTimerAtFixedRate instead. This has the same semantics as " +
"startTimerAtFixedRate, but startTimerWithFixedDelay is often preferred.",
since = "2.6.0")
def startPeriodicTimer(key: Any, msg: T, interval: Duration): Unit
/**

View file

@ -18,14 +18,57 @@ import scala.concurrent.duration.FiniteDuration
trait TimerScheduler[T] {
/**
* Start a periodic timer that will send `msg` to the `self` actor at
* a fixed `interval`.
* Schedules a message to be sent repeatedly to the `self` actor with a
* fixed `delay` between messages.
*
* It will not compensate the delay between messages if scheduling is delayed
* longer than specified for some reason. The delay between sending of subsequent
* messages will always be (at least) the given `delay`.
*
* In the long run, the frequency of messages will generally be slightly lower than
* the reciprocal of the specified `delay`.
*
* Each timer has a key and if a new timer with same key is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startTimerWithFixedDelay(key: Any, msg: T, delay: FiniteDuration): Unit
/**
* Schedules a message to be sent repeatedly to the `self` actor with a
* given frequency.
*
* It will compensate the delay for a subsequent message if the sending of previous
* message was delayed more than specified. In such cases, the actual message interval
* will differ from the interval passed to the method.
*
* If the execution is delayed longer than the `interval`, the subsequent message will
* be sent immediately after the prior one. This also has the consequence that after
* long garbage collection pauses or other reasons when the JVM was suspended all
* "missed" messages will be sent when the process wakes up again.
*
* In the long run, the frequency of messages will be exactly the reciprocal of the
* specified `interval`.
*
* Warning: `startTimerAtFixedRate` can result in bursts of scheduled messages after long
* garbage collection pauses, which may in worst case cause undesired load on the system.
* Therefore `startTimerWithFixedDelay` is often preferred.
*
* Each timer has a key and if a new timer with same key is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startTimerAtFixedRate(key: Any, msg: T, interval: FiniteDuration): Unit
/**
* Deprecated API: See [[TimerScheduler#startTimerWithFixedDelay]] or [[TimerScheduler#startTimerAtFixedRate]].
*/
@deprecated(
"Use startTimerWithFixedDelay or startTimerAtFixedRate instead. This has the same semantics as " +
"startTimerAtFixedRate, but startTimerWithFixedDelay is often preferred.",
since = "2.6.0")
def startPeriodicTimer(key: Any, msg: T, interval: FiniteDuration): Unit
/**

View file

@ -23,6 +23,8 @@ import scala.concurrent.duration.FiniteDuration;
*/
public abstract class AbstractScheduler extends AbstractSchedulerBase {
// FIXME #26910
/**
* 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

View file

@ -59,3 +59,16 @@ ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.event.LoggingReceive.c
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.event.Logging.getLogger")
ProblemFilters.exclude[MissingTypesProblem]("akka.routing.RoutedActorCell")
ProblemFilters.exclude[MissingTypesProblem]("akka.routing.ResizablePoolCell")
# #26910 scheduleWithFixedDelay vs scheduleAtFixedRate
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.FSM#Timer.repeat")
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.actor.FSM#Timer.copy")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.actor.FSM#Timer.copy$default$3")
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.actor.FSM#Timer.this")
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.actor.FSM#Timer.apply")
# declared final in current version
ProblemFilters.exclude[FinalMethodProblem]("akka.actor.TimerScheduler.startPeriodicTimer")
ProblemFilters.exclude[FinalMethodProblem]("akka.actor.TimerScheduler.startSingleTimer")
# only in current version
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.TimerScheduler.startTimerAtFixedRate")
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.TimerScheduler.startTimerWithFixedDelay")

View file

@ -4,7 +4,7 @@
package akka.actor
import akka.util.JavaDurationConverters
import akka.util.JavaDurationConverters._
import scala.concurrent.duration.FiniteDuration
/**
@ -113,7 +113,6 @@ abstract class AbstractFSM[S, D] extends FSM[S, D] {
stateName: S,
stateTimeout: java.time.Duration,
stateFunctionBuilder: FSMStateFunctionBuilder[S, D]): Unit = {
import JavaDurationConverters._
when(stateName, stateTimeout.asScala, stateFunctionBuilder)
}
@ -150,7 +149,6 @@ abstract class AbstractFSM[S, D] extends FSM[S, D] {
* @param timeout state timeout for the initial state, overriding the default timeout for that state
*/
final def startWith(stateName: S, stateData: D, timeout: java.time.Duration): Unit = {
import JavaDurationConverters._
startWith(stateName, stateData, timeout.asScala)
}
@ -434,6 +432,65 @@ abstract class AbstractFSM[S, D] extends FSM[S, D] {
*/
final def goTo(nextStateName: S): State = goto(nextStateName)
/**
* Schedules a message to be sent repeatedly to the `self` actor with a
* fixed `delay` between messages.
*
* It will not compensate the delay between messages if scheduling is delayed
* longer than specified for some reason. The delay between sending of subsequent
* messages will always be (at least) the given `delay`.
*
* In the long run, the frequency of messages will generally be slightly lower than
* the reciprocal of the specified `delay`.
*
* Each timer has a `name` and if a new timer with same `name` is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startTimerWithFixedDelay(name: String, msg: Any, delay: java.time.Duration): Unit =
startTimerWithFixedDelay(name, msg, delay.asScala)
/**
* Schedules a message to be sent repeatedly to the `self` actor with a
* given frequency.
*
* It will compensate the delay for a subsequent message if the sending of previous
* message was delayed more than specified. In such cases, the actual message interval
* will differ from the interval passed to the method.
*
* If the execution is delayed longer than the `interval`, the subsequent message will
* be sent immediately after the prior one. This also has the consequence that after
* long garbage collection pauses or other reasons when the JVM was suspended all
* "missed" messages will be sent when the process wakes up again.
*
* In the long run, the frequency of messages will be exactly the reciprocal of the
* specified `interval`.
*
* Warning: `startTimerAtFixedRate` can result in bursts of scheduled messages after long
* garbage collection pauses, which may in worst case cause undesired load on the system.
* Therefore `startTimerWithFixedDelay` is often preferred.
*
* Each timer has a `name` and if a new timer with same `name` is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startTimerAtFixedRate(name: String, msg: Any, interval: java.time.Duration): Unit =
startTimerAtFixedRate(name, msg, interval.asScala)
/**
* Start a timer that will send `msg` once to the `self` actor after
* the given `delay`.
*
* Each timer has a `name` and if a new timer with same `name` is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startSingleTimer(name: String, msg: Any, delay: java.time.Duration): Unit =
startSingleTimer(name, msg, delay.asScala)
/**
* Schedule named timer to deliver message after given delay, possibly repeating.
* Any existing timer with the same name will automatically be canceled before
@ -442,6 +499,7 @@ abstract class AbstractFSM[S, D] extends FSM[S, D] {
* @param msg message to be delivered
* @param timeout delay of first message delivery and between subsequent messages
*/
@deprecated("Use startSingleTimer instead.", since = "2.6.0")
final def setTimer(name: String, msg: Any, timeout: FiniteDuration): Unit =
setTimer(name, msg, timeout, repeat = false)
@ -453,8 +511,8 @@ abstract class AbstractFSM[S, D] extends FSM[S, D] {
* @param msg message to be delivered
* @param timeout delay of first message delivery and between subsequent messages
*/
@deprecated("Use startSingleTimer instead.", since = "2.6.0")
final def setTimer(name: String, msg: Any, timeout: java.time.Duration): Unit = {
import JavaDurationConverters._
setTimer(name, msg, timeout.asScala, false)
}
@ -467,8 +525,11 @@ abstract class AbstractFSM[S, D] extends FSM[S, D] {
* @param timeout delay of first message delivery and between subsequent messages
* @param repeat send once if false, scheduleAtFixedRate if true
*/
@deprecated(
"Use startSingleTimer, startTimerWithFixedDelay or startTimerAtFixedRate instead. This has the same semantics as " +
"startTimerAtFixedRate, but startTimerWithFixedDelay is often preferred.",
since = "2.6.0")
final def setTimer(name: String, msg: Any, timeout: java.time.Duration, repeat: Boolean): Unit = {
import JavaDurationConverters._
setTimer(name, msg, timeout.asScala, repeat)
}

View file

@ -531,13 +531,12 @@ abstract class ActorSystem extends ActorRefFactory {
* effort basis and hence not strictly guaranteed.
*/
def deadLetters: ActorRef
//#scheduler
/**
* Light-weight scheduler for running asynchronous tasks after some deadline
* in the future. Not terribly precise but cheap.
*/
def scheduler: Scheduler
//#scheduler
/**
* Java API: Light-weight scheduler for running asynchronous tasks after some deadline

View file

@ -88,11 +88,35 @@ object FSM {
*/
private final case class TimeoutMarker(generation: Long)
/** INTERNAL API */
@InternalApi
private[akka] sealed trait TimerMode {
def repeat: Boolean
}
/** INTERNAL API */
@InternalApi
private[akka] case object FixedRateMode extends TimerMode {
override def repeat: Boolean = true
}
/** INTERNAL API */
@InternalApi
private[akka] case object FixedDelayMode extends TimerMode {
override def repeat: Boolean = true
}
/** INTERNAL API */
@InternalApi
private[akka] case object SingleMode extends TimerMode {
override def repeat: Boolean = false
}
/**
* INTERNAL API
*/
@InternalApi
private[akka] final case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int, owner: AnyRef)(
private[akka] final case class Timer(name: String, msg: Any, mode: TimerMode, generation: Int, owner: AnyRef)(
context: ActorContext)
extends NoSerializationVerificationNeeded {
private var ref: Option[Cancellable] = _
@ -104,9 +128,11 @@ object FSM {
case m: AutoReceivedMessage => m
case _ => this
}
ref = Some(
if (repeat) scheduler.schedule(timeout, timeout, actor, timerMsg)
else scheduler.scheduleOnce(timeout, actor, timerMsg))
ref = Some(mode match {
case SingleMode => scheduler.scheduleOnce(timeout, actor, timerMsg)
case FixedDelayMode => scheduler.scheduleWithFixedDelay(timeout, timeout, actor, timerMsg)
case FixedRateMode => scheduler.scheduleAtFixedRate(timeout, timeout, actor, timerMsg)
})
}
def cancel(): Unit =
if (ref.isDefined) {
@ -448,6 +474,65 @@ trait FSM[S, D] extends Actor with Listeners with ActorLogging {
final def transform(func: StateFunction): TransformHelper = new TransformHelper(func)
/**
* Schedules a message to be sent repeatedly to the `self` actor with a
* fixed `delay` between messages.
*
* It will not compensate the delay between messages if scheduling is delayed
* longer than specified for some reason. The delay between sending of subsequent
* messages will always be (at least) the given `delay`.
*
* In the long run, the frequency of messages will generally be slightly lower than
* the reciprocal of the specified `delay`.
*
* Each timer has a `name` and if a new timer with same `name` is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startTimerWithFixedDelay(name: String, msg: Any, delay: FiniteDuration): Unit =
startTimer(name, msg, delay, FixedDelayMode)
/**
* Schedules a message to be sent repeatedly to the `self` actor with a
* given frequency.
*
* It will compensate the delay for a subsequent message if the sending of previous
* message was delayed more than specified. In such cases, the actual message interval
* will differ from the interval passed to the method.
*
* If the execution is delayed longer than the `interval`, the subsequent message will
* be sent immediately after the prior one. This also has the consequence that after
* long garbage collection pauses or other reasons when the JVM was suspended all
* "missed" messages will be sent when the process wakes up again.
*
* In the long run, the frequency of messages will be exactly the reciprocal of the
* specified `interval`.
*
* Warning: `startTimerAtFixedRate` can result in bursts of scheduled messages after long
* garbage collection pauses, which may in worst case cause undesired load on the system.
* Therefore `startTimerWithFixedDelay` is often preferred.
*
* Each timer has a `name` and if a new timer with same `name` is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startTimerAtFixedRate(name: String, msg: Any, interval: FiniteDuration): Unit =
startTimer(name, msg, interval, FixedRateMode)
/**
* Start a timer that will send `msg` once to the `self` actor after
* the given `delay`.
*
* Each timer has a `name` and if a new timer with same `name` is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startSingleTimer(name: String, msg: Any, delay: FiniteDuration): Unit =
startTimer(name, msg, delay, SingleMode)
/**
* Schedule named timer to deliver message after given delay, possibly repeating.
* Any existing timer with the same name will automatically be canceled before
@ -457,13 +542,23 @@ trait FSM[S, D] extends Actor with Listeners with ActorLogging {
* @param timeout delay of first message delivery and between subsequent messages
* @param repeat send once if false, scheduleAtFixedRate if true
*/
@deprecated(
"Use startSingleTimer, startTimerWithFixedDelay or startTimerAtFixedRate instead. This has the same semantics as " +
"startTimerAtFixedRate, but startTimerWithFixedDelay is often preferred.",
since = "2.6.0")
final def setTimer(name: String, msg: Any, timeout: FiniteDuration, repeat: Boolean = false): Unit = {
// repeat => FixedRateMode for compatibility
val mode = if (repeat) FixedRateMode else SingleMode
startTimer(name, msg, timeout, mode)
}
private def startTimer(name: String, msg: Any, timeout: FiniteDuration, mode: TimerMode): Unit = {
if (debugEvent)
log.debug("setting " + (if (repeat) "repeating " else "") + "timer '" + name + "'/" + timeout + ": " + msg)
log.debug("setting " + (if (mode.repeat) "repeating " else "") + "timer '" + name + "'/" + timeout + ": " + msg)
if (timers contains name) {
timers(name).cancel()
}
val timer = Timer(name, msg, repeat, timerGen.next, this)(context)
val timer = Timer(name, msg, mode, timerGen.next, this)(context)
timer.schedule(self, timeout)
timers(name) = timer
}
@ -660,14 +755,14 @@ trait FSM[S, D] extends Actor with Listeners with ActorLogging {
if (generation == gen) {
processMsg(StateTimeout, "state timeout")
}
case t @ Timer(name, msg, repeat, gen, owner) =>
case t @ Timer(name, msg, mode, gen, owner) =>
if ((owner eq this) && (timers contains name) && (timers(name).generation == gen)) {
if (timeoutFuture.isDefined) {
timeoutFuture.get.cancel()
timeoutFuture = None
}
generation += 1
if (!repeat) {
if (!mode.repeat) {
timers -= name
}
processMsg(msg, t)

View file

@ -90,6 +90,12 @@ class LightArrayRevolverScheduler(config: Config, log: LoggingAdapter, threadFac
}
}
override def scheduleWithFixedDelay(initialDelay: FiniteDuration, delay: FiniteDuration)(runnable: Runnable)(
implicit executor: ExecutionContext): Cancellable = {
checkMaxDelay(roundUp(delay).toNanos)
super.scheduleWithFixedDelay(initialDelay, delay)(runnable)
}
override def schedule(initialDelay: FiniteDuration, delay: FiniteDuration, runnable: Runnable)(
implicit executor: ExecutionContext): Cancellable = {
checkMaxDelay(roundUp(delay).toNanos)
@ -130,7 +136,7 @@ class LightArrayRevolverScheduler(config: Config, log: LoggingAdapter, threadFac
override def isCancelled: Boolean = get == null
} catch {
case SchedulerException(msg) => throw new IllegalStateException(msg)
case cause @ SchedulerException(msg) => throw new IllegalStateException(msg, cause)
}
}
@ -138,7 +144,7 @@ class LightArrayRevolverScheduler(config: Config, log: LoggingAdapter, threadFac
implicit executor: ExecutionContext): Cancellable =
try schedule(executor, runnable, roundUp(delay))
catch {
case SchedulerException(msg) => throw new IllegalStateException(msg)
case cause @ SchedulerException(msg) => throw new IllegalStateException(msg, cause)
}
override def close(): Unit = Await.result(stop(), getShutdownTimeout).foreach { task =>

View file

@ -4,21 +4,25 @@
package akka.actor
import java.util.concurrent.atomic.AtomicReference
import scala.annotation.tailrec
import akka.util.JavaDurationConverters
import com.github.ghik.silencer.silent
import scala.concurrent.ExecutionContext
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
import akka.annotation.InternalApi
import com.github.ghik.silencer.silent
/**
* This exception is thrown by Scheduler.schedule* when scheduling is not
* possible, e.g. after shutting down the Scheduler.
*/
private final case class SchedulerException(msg: String) extends akka.AkkaException(msg) with NoStackTrace
// The Scheduler trait is included in the documentation. KEEP THE LINES SHORT!!!
//#scheduler
/**
* An Akka scheduler service. This one needs one special behavior: if
* Closeable, it MUST execute all outstanding tasks upon .close() in order
@ -34,7 +38,7 @@ private final case class SchedulerException(msg: String) extends akka.AkkaExcept
* 2) a akka.event.LoggingAdapter
* 3) a java.util.concurrent.ThreadFactory
*
* Please note that this scheduler implementation is higly optimised for high-throughput
* Please note that this scheduler implementation is highly optimised for high-throughput
* and high-frequency events. It is not to be confused with long-term schedulers such as
* Quartz. The scheduler will throw an exception if attempts are made to schedule too far
* into the future (which by default is around 8 months (`Int.MaxValue` seconds).
@ -42,13 +46,310 @@ private final case class SchedulerException(msg: String) extends akka.AkkaExcept
trait Scheduler {
/**
* 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
* thereafter every 500ms you would set delay=Duration.Zero and
* interval=Duration(500, TimeUnit.MILLISECONDS)
* Scala API: Schedules a `Runnable` to be run repeatedly with an initial delay and
* a fixed `delay` between subsequent executions. 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)`
* and `interval=Duration(100, TimeUnit.MILLISECONDS)`.
*
* Java & Scala API
* It will not compensate the delay between tasks if the execution takes a long time or if
* scheduling is delayed longer than specified for some reason. The delay between subsequent
* execution will always be (at least) the given `delay`. In the long run, the
* frequency of execution will generally be slightly lower than the reciprocal of the specified
* `delay`.
*
* If the `Runnable` throws an exception the repeated scheduling is aborted,
* i.e. the function will not be invoked any more.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Note: For scheduling within actors `with Timers` should be preferred.
*/
def scheduleWithFixedDelay(initialDelay: FiniteDuration, delay: FiniteDuration)(runnable: Runnable)(
implicit executor: ExecutionContext): Cancellable = {
try new AtomicReference[Cancellable](Cancellable.initialNotCancelled) with Cancellable { self =>
compareAndSet(
Cancellable.initialNotCancelled,
scheduleOnce(
initialDelay,
new Runnable {
override def run(): Unit = {
try {
runnable.run()
if (self.get != null)
swap(scheduleOnce(delay, this))
} catch {
// ignore failure to enqueue or terminated target actor
case _: SchedulerException =>
case e: IllegalStateException if e.getCause != null && e.getCause.isInstanceOf[SchedulerException] =>
}
}
}))
@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)
}
}
/**
* Java API: Schedules a `Runnable` to be run repeatedly with an initial delay and
* a fixed `delay` between subsequent executions. E.g. if you would like the function to
* be run after 2 seconds and thereafter every 100ms you would set delay to `Duration.ofSeconds(2)`,
* and interval to `Duration.ofMillis(100)`.
*
* It will not compensate the delay between tasks if the execution takes a long time or if
* scheduling is delayed longer than specified for some reason. The delay between subsequent
* execution will always be (at least) the given `delay`.
*
* In the long run, the frequency of tasks will generally be slightly lower than
* the reciprocal of the specified `delay`.
*
* If the `Runnable` throws an exception the repeated scheduling is aborted,
* i.e. the function will not be invoked any more.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Note: For scheduling within actors `AbstractActorWithTimers` should be preferred.
*/
final def scheduleWithFixedDelay(
initialDelay: java.time.Duration,
delay: java.time.Duration,
runnable: Runnable,
executor: ExecutionContext): Cancellable = {
import JavaDurationConverters._
scheduleWithFixedDelay(initialDelay.asScala, delay.asScala)(runnable)(executor)
}
/**
* Scala API: Schedules a message to be sent repeatedly with an initial delay and
* a fixed `delay` between messages. E.g. if you would like a message to be sent
* immediately and thereafter every 500ms you would set `delay=Duration.Zero` and
* `interval=Duration(500, TimeUnit.MILLISECONDS)`.
*
* It will not compensate the delay between messages if scheduling is delayed
* longer than specified for some reason. The delay between sending of subsequent
* messages will always be (at least) the given `delay`.
*
* In the long run, the frequency of messages will generally be slightly lower than
* the reciprocal of the specified `delay`.
*
* Note: For scheduling within actors `with Timers` should be preferred.
*/
@silent
final def scheduleWithFixedDelay(
initialDelay: FiniteDuration,
delay: FiniteDuration,
receiver: ActorRef,
message: Any)(
implicit
executor: ExecutionContext,
sender: ActorRef = Actor.noSender): Cancellable = {
scheduleWithFixedDelay(initialDelay, delay)(new Runnable {
def run(): Unit = {
receiver ! message
if (receiver.isTerminated)
throw SchedulerException("timer active for terminated actor")
}
})
}
/**
* Java API: Schedules a message to be sent repeatedly with an initial delay and
* a fixed `delay` between messages. E.g. if you would like a message to be sent
* immediately and thereafter every 500ms you would set `delay=Duration.ZERO` and
* `interval=Duration.ofMillis(500)`.
*
* It will not compensate the delay between messages if scheduling is delayed
* longer than specified for some reason. The delay between sending of subsequent
* messages will always be (at least) the given `delay`.
*
* In the long run, the frequency of messages will generally be slightly lower than
* the reciprocal of the specified `delay`.
*
* Note: For scheduling within actors `AbstractActorWithTimers` should be preferred.
*/
final def scheduleWithFixedDelay(
initialDelay: java.time.Duration,
delay: java.time.Duration,
receiver: ActorRef,
message: Any,
executor: ExecutionContext,
sender: ActorRef): Cancellable = {
import JavaDurationConverters._
scheduleWithFixedDelay(initialDelay.asScala, delay.asScala, receiver, message)(executor, sender)
}
/**
* Scala API: Schedules a `Runnable` 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)`
* and `interval=Duration(100, TimeUnit.MILLISECONDS)`.
*
* It will compensate the delay for a subsequent task if the previous tasks took
* too long to execute. In such cases, the actual execution interval will differ from
* the interval passed to the method.
*
* If the execution of the tasks takes longer than the `interval`, the subsequent
* execution will start immediately after the prior one completes (there will be
* no overlap of executions). This also has the consequence that after long garbage
* collection pauses or other reasons when the JVM was suspended all "missed" tasks
* will execute when the process wakes up again.
*
* In the long run, the frequency of execution will be exactly the reciprocal of the
* specified `interval`.
*
* Warning: `scheduleAtFixedRate` can result in bursts of scheduled tasks after long
* garbage collection pauses, which may in worst case cause undesired load on the system.
* Therefore `scheduleWithFixedDelay` is often preferred.
*
* If the `Runnable` throws an exception the repeated scheduling is aborted,
* i.e. the function will not be invoked any more.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Note: For scheduling within actors `with Timers` should be preferred.
*/
@silent
final def scheduleAtFixedRate(initialDelay: FiniteDuration, interval: FiniteDuration)(runnable: Runnable)(
implicit executor: ExecutionContext): Cancellable =
schedule(initialDelay, interval, runnable)(executor)
/**
* Java API: Schedules a `Runnable` 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 to `Duration.ofSeconds(2)`,
* and interval to `Duration.ofMillis(100)`.
*
* It will compensate the delay for a subsequent task if the previous tasks took
* too long to execute. In such cases, the actual execution interval will differ from
* the interval passed to the method.
*
* If the execution of the tasks takes longer than the `interval`, the subsequent
* execution will start immediately after the prior one completes (there will be
* no overlap of executions). This also has the consequence that after long garbage
* collection pauses or other reasons when the JVM was suspended all "missed" tasks
* will execute when the process wakes up again.
*
* In the long run, the frequency of execution will be exactly the reciprocal of the
* specified `interval`.
*
* Warning: `scheduleAtFixedRate` can result in bursts of scheduled tasks after long
* garbage collection pauses, which may in worst case cause undesired load on the system.
* Therefore `scheduleWithFixedDelay` is often preferred.
*
* If the `Runnable` throws an exception the repeated scheduling is aborted,
* i.e. the function will not be invoked any more.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Note: For scheduling within actors `AbstractActorWithTimers` should be preferred.
*/
final def scheduleAtFixedRate(
initialDelay: java.time.Duration,
interval: java.time.Duration,
runnable: Runnable,
executor: ExecutionContext): Cancellable = {
import JavaDurationConverters._
scheduleAtFixedRate(initialDelay.asScala, interval.asScala)(runnable)(executor)
}
/**
* Scala API: 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
* thereafter every 500ms you would set `delay=Duration.Zero` and
* `interval=Duration(500, TimeUnit.MILLISECONDS)`
*
* It will compensate the delay for a subsequent message if the sending of previous
* message was delayed more than specified. In such cases, the actual message interval
* will differ from the interval passed to the method.
*
* If the execution is delayed longer than the `interval`, the subsequent message will
* be sent immediately after the prior one. This also has the consequence that after
* long garbage collection pauses or other reasons when the JVM was suspended all
* "missed" messages will be sent when the process wakes up again.
*
* In the long run, the frequency of messages will be exactly the reciprocal of the
* specified `interval`.
*
* Warning: `scheduleAtFixedRate` can result in bursts of scheduled messages after long
* garbage collection pauses, which may in worst case cause undesired load on the system.
* Therefore `scheduleWithFixedDelay` is often preferred.
*
* Note: For scheduling within actors `with Timers` should be preferred.
*/
@silent
final def scheduleAtFixedRate(
initialDelay: FiniteDuration,
interval: FiniteDuration,
receiver: ActorRef,
message: Any)(
implicit
executor: ExecutionContext,
sender: ActorRef = Actor.noSender): Cancellable =
schedule(initialDelay, interval, receiver, message)
/**
* Java API: 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
* thereafter every 500ms you would set `delay=Duration.ZERO` and
* `interval=Duration.ofMillis(500)`
*
* It will compensate the delay for a subsequent message if the sending of previous
* message was delayed more than specified. In such cases, the actual message interval
* will differ from the interval passed to the method.
*
* If the execution is delayed longer than the `interval`, the subsequent message will
* be sent immediately after the prior one. This also has the consequence that after
* long garbage collection pauses or other reasons when the JVM was suspended all
* "missed" messages will be sent when the process wakes up again.
*
* In the long run, the frequency of messages will be exactly the reciprocal of the
* specified `interval`.
*
* Warning: `scheduleAtFixedRate` can result in bursts of scheduled messages after long
* garbage collection pauses, which may in worst case cause undesired load on the system.
* Therefore `scheduleWithFixedDelay` is often preferred.
*
* Note: For scheduling within actors `AbstractActorWithTimers` should be preferred.
*/
final def scheduleAtFixedRate(
initialDelay: java.time.Duration,
interval: java.time.Duration,
receiver: ActorRef,
message: Any,
executor: ExecutionContext,
sender: ActorRef): Cancellable = {
import JavaDurationConverters._
scheduleAtFixedRate(initialDelay.asScala, interval.asScala, receiver, message)(executor, sender)
}
/**
* Deprecated API: See [[Scheduler#scheduleWithFixedDelay]] or [[Scheduler#scheduleAtFixedRate]].
*/
@deprecated(
"Use scheduleWithFixedDelay or scheduleAtFixedRate instead. This has the same semantics as " +
"scheduleAtFixedRate, but scheduleWithFixedDelay is often preferred.",
since = "2.6.0")
@silent
final def schedule(initialDelay: FiniteDuration, interval: FiniteDuration, receiver: ActorRef, message: Any)(
implicit
@ -66,13 +367,12 @@ trait Scheduler {
})
/**
* 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
* thereafter every 500ms you would set delay=Duration.Zero and
* interval=Duration(500, TimeUnit.MILLISECONDS)
*
* Java API
* Deprecated API: See [[Scheduler#scheduleWithFixedDelay]] or [[Scheduler#scheduleAtFixedRate]].
*/
@deprecated(
"Use scheduleWithFixedDelay or scheduleAtFixedRate instead. This has the same semantics as " +
"scheduleAtFixedRate, but scheduleWithFixedDelay is often preferred.",
since = "2.6.0")
final def schedule(
initialDelay: java.time.Duration,
interval: java.time.Duration,
@ -85,66 +385,34 @@ trait Scheduler {
}
/**
* 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)
* and interval = Duration(100, TimeUnit.MILLISECONDS). If the execution of
* the function takes longer than the interval, the subsequent execution will
* start immediately after the prior one completes (there will be no overlap
* of the function executions). In such cases, the actual execution interval
* will differ from the interval passed to this method.
*
* If the function throws an exception the repeated scheduling is aborted,
* i.e. the function will not be invoked any more.
*
* Scala API
* Deprecated API: See [[Scheduler#scheduleWithFixedDelay]] or [[Scheduler#scheduleAtFixedRate]].
*/
@deprecated(
"Use scheduleWithFixedDelay or scheduleAtFixedRate instead. This has the same semantics as " +
"scheduleAtFixedRate, but scheduleWithFixedDelay is often preferred.",
since = "2.6.0")
final def schedule(initialDelay: FiniteDuration, interval: FiniteDuration)(f: => Unit)(
implicit
executor: ExecutionContext): Cancellable =
schedule(initialDelay, interval, new Runnable { override def run(): Unit = f })
/**
* Schedules a `Runnable` 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) and interval = Duration(100, TimeUnit.MILLISECONDS). If
* the execution of the runnable takes longer than the interval, the
* subsequent execution will start immediately after the prior one completes
* (there will be no overlap of executions of the runnable). In such cases,
* the actual execution interval will differ from the interval passed to this
* method.
*
* If the `Runnable` throws an exception the repeated scheduling is aborted,
* i.e. the function will not be invoked any more.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Java API
* Deprecated API: See [[Scheduler#scheduleWithFixedDelay]] or [[Scheduler#scheduleAtFixedRate]].
*/
@deprecated(
"Use scheduleWithFixedDelay or scheduleAtFixedRate instead. This has the same semantics as " +
"scheduleAtFixedRate, but scheduleWithFixedDelay is often preferred.",
since = "2.6.0")
def schedule(initialDelay: FiniteDuration, interval: FiniteDuration, runnable: Runnable)(
implicit executor: ExecutionContext): Cancellable
/**
* Schedules a `Runnable` 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) and interval = Duration(100, TimeUnit.MILLISECONDS). If
* the execution of the runnable takes longer than the interval, the
* subsequent execution will start immediately after the prior one completes
* (there will be no overlap of executions of the runnable). In such cases,
* the actual execution interval will differ from the interval passed to this
* method.
*
* If the `Runnable` throws an exception the repeated scheduling is aborted,
* i.e. the function will not be invoked any more.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Java API
* Deprecated API: See [[Scheduler#scheduleWithFixedDelay]] or [[Scheduler#scheduleAtFixedRate]].
*/
@deprecated(
"Use scheduleWithFixedDelay or scheduleAtFixedRate instead. This has the same semantics as " +
"scheduleAtFixedRate, but scheduleWithFixedDelay is often preferred.",
since = "2.6.0")
def schedule(initialDelay: java.time.Duration, interval: java.time.Duration, runnable: Runnable)(
implicit executor: ExecutionContext): Cancellable = {
import JavaDurationConverters._
@ -152,13 +420,13 @@ trait Scheduler {
}
/**
* Schedules a message to be sent once with a delay, i.e. a time period that has
* Scala API: Schedules a message to be sent once with a delay, i.e. a time period that has
* to pass before the message is sent.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Java & Scala API
* Note: For scheduling within actors `with Timers` should be preferred.
*/
final def scheduleOnce(delay: FiniteDuration, receiver: ActorRef, message: Any)(
implicit
@ -169,13 +437,13 @@ trait Scheduler {
})
/**
* Schedules a message to be sent once with a delay, i.e. a time period that has
* Java API: Schedules a message to be sent once with a delay, i.e. a time period that has
* to pass before the message is sent.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Java API
* Note: For scheduling within actors `AbstractActorWithTimers` should be preferred.
*/
final def scheduleOnce(
delay: java.time.Duration,
@ -188,13 +456,13 @@ trait Scheduler {
}
/**
* Schedules a function to be run once with a delay, i.e. a time period that has
* Scala API: Schedules a function to be run once with a delay, i.e. a time period that has
* to pass before the function is run.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Scala API
* Note: For scheduling within actors `with Timers` should be preferred.
*/
final def scheduleOnce(delay: FiniteDuration)(f: => Unit)(
implicit
@ -202,24 +470,24 @@ trait Scheduler {
scheduleOnce(delay, new Runnable { override def run(): Unit = f })
/**
* Schedules a Runnable to be run once with a delay, i.e. a time period that
* Scala API: Schedules a Runnable to be run once with a delay, i.e. a time period that
* has to pass before the runnable is executed.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Java & Scala API
* Note: For scheduling within actors `with Timers` should be preferred.
*/
def scheduleOnce(delay: FiniteDuration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable
/**
* Schedules a Runnable to be run once with a delay, i.e. a time period that
* Java API: Schedules a Runnable to be run once with a delay, i.e. a time period that
* has to pass before the runnable is executed.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* reach (calculated as: `delay / tickNanos > Int.MaxValue`).
*
* Java & Scala API
* Note: For scheduling within actors `AbstractActorWithTimers` should be preferred.
*/
def scheduleOnce(delay: java.time.Duration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable = {
import JavaDurationConverters._
@ -233,12 +501,10 @@ trait Scheduler {
def maxFrequency: Double
}
//#scheduler
// this one is just here so we can present a nice AbstractScheduler for Java
abstract class AbstractSchedulerBase extends Scheduler
//#cancellable
/**
* Signifies something that can be cancelled
* There is no strict guarantee that the implementation is thread-safe,
@ -262,11 +528,18 @@ trait Cancellable {
*/
def isCancelled: Boolean
}
//#cancellable
object Cancellable {
val alreadyCancelled: Cancellable = new Cancellable {
def cancel(): Boolean = false
def isCancelled: Boolean = true
}
/**
* INTERNAL API
*/
@InternalApi private[akka] val initialNotCancelled: Cancellable = new Cancellable {
def cancel(): Boolean = false
def isCancelled: Boolean = false
}
}

View file

@ -86,26 +86,114 @@ abstract class AbstractActorWithTimers extends AbstractActor with Timers {
@DoNotInherit abstract class TimerScheduler {
/**
* Start a periodic timer that will send `msg` to the `self` actor at
* a fixed `interval`.
* Scala API: Schedules a message to be sent repeatedly to the `self` actor with a
* fixed `delay` between messages.
*
* It will not compensate the delay between messages if scheduling is delayed
* longer than specified for some reason. The delay between sending of subsequent
* messages will always be (at least) the given `delay`.
*
* In the long run, the frequency of messages will generally be slightly lower than
* the reciprocal of the specified `delay`.
*
* Each timer has a key and if a new timer with same key is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startTimerWithFixedDelay(key: Any, msg: Any, delay: FiniteDuration): Unit
/**
* Java API: Schedules a message to be sent repeatedly to the `self` actor with a
* fixed `delay` between messages.
*
* It will not compensate the delay between messages if scheduling is delayed
* longer than specified for some reason. The delay between sending of subsequent
* messages will always be (at least) the given `delay`.
*
* In the long run, the frequency of messages will generally be slightly lower than
* the reciprocal of the specified `delay`.
*
* Each timer has a key and if a new timer with same key is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
final def startTimerWithFixedDelay(key: Any, msg: Any, delay: java.time.Duration): Unit =
startTimerWithFixedDelay(key, msg, delay.asScala)
/**
* Scala API: Schedules a message to be sent repeatedly to the `self` actor with a
* given frequency.
*
* It will compensate the delay for a subsequent message if the sending of previous
* message was delayed more than specified. In such cases, the actual message interval
* will differ from the interval passed to the method.
*
* If the execution is delayed longer than the `interval`, the subsequent message will
* be sent immediately after the prior one. This also has the consequence that after
* long garbage collection pauses or other reasons when the JVM was suspended all
* "missed" messages will be sent when the process wakes up again.
*
* In the long run, the frequency of messages will be exactly the reciprocal of the
* specified `interval`.
*
* Warning: `startTimerAtFixedRate` can result in bursts of scheduled messages after long
* garbage collection pauses, which may in worst case cause undesired load on the system.
* Therefore `startTimerWithFixedDelay` is often preferred.
*
* Each timer has a key and if a new timer with same key is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startTimerAtFixedRate(key: Any, msg: Any, interval: FiniteDuration): Unit
/**
* Java API: Schedules a message to be sent repeatedly to the `self` actor with a
* given frequency.
*
* It will compensate the delay for a subsequent message if the sending of previous
* message was delayed more than specified. In such cases, the actual message interval
* will differ from the interval passed to the method.
*
* If the execution is delayed longer than the `interval`, the subsequent message will
* be sent immediately after the prior one. This also has the consequence that after
* long garbage collection pauses or other reasons when the JVM was suspended all
* "missed" messages will be sent when the process wakes up again.
*
* In the long run, the frequency of messages will be exactly the reciprocal of the
* specified `interval`.
*
* Warning: `startTimerAtFixedRate` can result in bursts of scheduled messages after long
* garbage collection pauses, which may in worst case cause undesired load on the system.
* Therefore `startTimerWithFixedDelay` is often preferred.
*
* Each timer has a key and if a new timer with same key is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
final def startTimerAtFixedRate(key: Any, msg: Any, interval: java.time.Duration): Unit =
startTimerAtFixedRate(key, msg, interval.asScala)
/**
* Deprecated API: See [[TimerScheduler#startTimerWithFixedDelay]] or [[TimerScheduler#startTimerAtFixedRate]].
*/
@deprecated(
"Use startTimerWithFixedDelay or startTimerAtFixedRate instead. This has the same semantics as " +
"startTimerAtFixedRate, but startTimerWithFixedDelay is often preferred.",
since = "2.6.0")
def startPeriodicTimer(key: Any, msg: Any, interval: FiniteDuration): Unit
/**
* Start a periodic timer that will send `msg` to the `self` actor at
* a fixed `interval`.
*
* Each timer has a key and if a new timer with same key is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
* Deprecated API: See [[TimerScheduler#startTimerWithFixedDelay]] or [[TimerScheduler#startTimerAtFixedRate]].
*/
def startPeriodicTimer(key: Any, msg: Any, interval: java.time.Duration): Unit =
@deprecated(
"Use startTimerWithFixedDelay or startTimerAtFixedRate instead. This has the same semantics as " +
"startTimerAtFixedRate, but startTimerWithFixedDelay is often preferred.",
since = "2.6.0")
final def startPeriodicTimer(key: Any, msg: Any, interval: java.time.Duration): Unit =
startPeriodicTimer(key, msg, interval.asScala)
/**
@ -128,7 +216,7 @@ abstract class AbstractActorWithTimers extends AbstractActor with Timers {
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startSingleTimer(key: Any, msg: Any, timeout: java.time.Duration): Unit =
final def startSingleTimer(key: Any, msg: Any, timeout: java.time.Duration): Unit =
startSingleTimer(key, msg, timeout.asScala)
/**

View file

@ -28,6 +28,19 @@ import akka.util.OptionVal
extends TimerMsg
with NoSerializationVerificationNeeded
with NotInfluenceReceiveTimeout
private sealed trait TimerMode {
def repeat: Boolean
}
private case object FixedRateMode extends TimerMode {
override def repeat: Boolean = true
}
private case object FixedDelayMode extends TimerMode {
override def repeat: Boolean = true
}
private case object SingleMode extends TimerMode {
override def repeat: Boolean = false
}
}
/**
@ -44,13 +57,19 @@ import akka.util.OptionVal
timerGen
}
override def startTimerAtFixedRate(key: Any, msg: Any, interval: FiniteDuration): Unit =
startTimer(key, msg, interval, FixedRateMode)
override def startTimerWithFixedDelay(key: Any, msg: Any, delay: FiniteDuration): Unit =
startTimer(key, msg, delay, FixedDelayMode)
override def startPeriodicTimer(key: Any, msg: Any, interval: FiniteDuration): Unit =
startTimer(key, msg, interval, repeat = true)
startTimerAtFixedRate(key, msg, interval)
override def startSingleTimer(key: Any, msg: Any, timeout: FiniteDuration): Unit =
startTimer(key, msg, timeout, repeat = false)
startTimer(key, msg, timeout, SingleMode)
private def startTimer(key: Any, msg: Any, timeout: FiniteDuration, repeat: Boolean): Unit = {
private def startTimer(key: Any, msg: Any, timeout: FiniteDuration, mode: TimerMode): Unit = {
timers.get(key) match {
case Some(t) => cancelTimer(t)
case None =>
@ -63,13 +82,16 @@ import akka.util.OptionVal
else
InfluenceReceiveTimeoutTimerMsg(key, nextGen, this)
val task =
if (repeat)
ctx.system.scheduler.schedule(timeout, timeout, ctx.self, timerMsg)(ctx.dispatcher)
else
val task = mode match {
case SingleMode =>
ctx.system.scheduler.scheduleOnce(timeout, ctx.self, timerMsg)(ctx.dispatcher)
case FixedDelayMode =>
ctx.system.scheduler.scheduleWithFixedDelay(timeout, timeout, ctx.self, timerMsg)(ctx.dispatcher)
case FixedRateMode =>
ctx.system.scheduler.scheduleAtFixedRate(timeout, timeout, ctx.self, timerMsg)(ctx.dispatcher)
}
val nextTimer = Timer(key, msg, repeat, nextGen, task)
val nextTimer = Timer(key, msg, mode.repeat, nextGen, task)
timers = timers.updated(key, nextTimer)
}

View file

@ -38,7 +38,7 @@ class SimpleDnsManager(val ext: DnsExt)
val interval = Duration(
ext.Settings.ResolverConfig.getDuration("cache-cleanup-interval", TimeUnit.MILLISECONDS),
TimeUnit.MILLISECONDS)
system.scheduler.schedule(interval, interval, self, SimpleDnsManager.CacheCleanup)
system.scheduler.scheduleWithFixedDelay(interval, interval, self, SimpleDnsManager.CacheCleanup)
}
override def receive: Receive = {

View file

@ -81,7 +81,7 @@ private[io] final class AsyncDnsManager(
cacheCleanup.foreach { _ =>
val interval =
Duration(resolverConfig.getDuration("cache-cleanup-interval", TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS)
timers.startPeriodicTimer(CacheCleanup, CacheCleanup, interval)
timers.startTimerWithFixedDelay(CacheCleanup, CacheCleanup, interval)
}
}

View file

@ -77,7 +77,7 @@ private[akka] final case class TailChoppingRoutees(
val aIdx = new AtomicInteger()
val size = shuffled.length
val tryWithNext = scheduler.schedule(0.millis, interval) {
val tryWithNext = scheduler.scheduleWithFixedDelay(Duration.Zero, interval) { () =>
val idx = aIdx.getAndIncrement
if (idx < size) {
shuffled(idx) match {

View file

@ -14,27 +14,6 @@ import scala.concurrent.ExecutionContext.Implicits.global
import scala.concurrent.duration._
import scala.concurrent.{ Await, Promise }
/*
[info] Benchmark (ratio) (to) Mode Samples Score Score error Units
[info] a.a.ScheduleBenchmark.multipleScheduleOnce 0.1 4 thrpt 40 397174.273 18707.983 ops/s
[info] a.a.ScheduleBenchmark.multipleScheduleOnce 0.1 16 thrpt 40 89385.115 3198.783 ops/s
[info] a.a.ScheduleBenchmark.multipleScheduleOnce 0.1 64 thrpt 40 26152.329 2291.895 ops/s
[info] a.a.ScheduleBenchmark.multipleScheduleOnce 0.35 4 thrpt 40 383100.418 15052.818 ops/s
[info] a.a.ScheduleBenchmark.multipleScheduleOnce 0.35 16 thrpt 40 83574.143 6612.393 ops/s
[info] a.a.ScheduleBenchmark.multipleScheduleOnce 0.35 64 thrpt 40 20509.715 2814.356 ops/s
[info] a.a.ScheduleBenchmark.multipleScheduleOnce 0.9 4 thrpt 40 367227.500 16169.665 ops/s
[info] a.a.ScheduleBenchmark.multipleScheduleOnce 0.9 16 thrpt 40 72611.445 4086.267 ops/s
[info] a.a.ScheduleBenchmark.multipleScheduleOnce 0.9 64 thrpt 40 7332.554 1087.250 ops/s
[info] a.a.ScheduleBenchmark.oneSchedule 0.1 4 thrpt 40 1040918.731 21830.348 ops/s
[info] a.a.ScheduleBenchmark.oneSchedule 0.1 16 thrpt 40 1036284.894 26962.984 ops/s
[info] a.a.ScheduleBenchmark.oneSchedule 0.1 64 thrpt 40 944350.638 32055.335 ops/s
[info] a.a.ScheduleBenchmark.oneSchedule 0.35 4 thrpt 40 1045371.779 34943.155 ops/s
[info] a.a.ScheduleBenchmark.oneSchedule 0.35 16 thrpt 40 954663.161 18032.730 ops/s
[info] a.a.ScheduleBenchmark.oneSchedule 0.35 64 thrpt 40 739593.387 21132.531 ops/s
[info] a.a.ScheduleBenchmark.oneSchedule 0.9 4 thrpt 40 1046392.800 29542.291 ops/s
[info] a.a.ScheduleBenchmark.oneSchedule 0.9 16 thrpt 40 820986.574 22058.708 ops/s
[info] a.a.ScheduleBenchmark.oneSchedule 0.9 64 thrpt 40 210115.907 14176.402 ops/s
*/
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(2)
@ -71,9 +50,23 @@ class ScheduleBenchmark {
def op(idx: Int) = if (idx == winner) promise.trySuccess(idx) else idx
@Benchmark
def oneSchedule(): Unit = {
def scheduleWithFixedDelay(): Unit = {
val aIdx = new AtomicInteger(1)
val tryWithNext = scheduler.schedule(0.millis, interval) {
val tryWithNext = scheduler.scheduleWithFixedDelay(0.millis, interval) { () =>
val idx = aIdx.getAndIncrement
if (idx <= to) op(idx)
}
promise.future.onComplete {
case _ =>
tryWithNext.cancel()
}
Await.result(promise.future, within)
}
@Benchmark
def scheduleAtFixedRate(): Unit = {
val aIdx = new AtomicInteger(1)
val tryWithNext = scheduler.scheduleAtFixedRate(0.millis, interval) { () =>
val idx = aIdx.getAndIncrement
if (idx <= to) op(idx)
}

View file

@ -161,12 +161,16 @@ private[metrics] class ClusterMetricsCollector extends Actor with ActorLogging {
* Start periodic gossip to random nodes in cluster
*/
val gossipTask =
scheduler.schedule(PeriodicTasksInitialDelay max CollectorGossipInterval, CollectorGossipInterval, self, GossipTick)
scheduler.scheduleWithFixedDelay(
PeriodicTasksInitialDelay max CollectorGossipInterval,
CollectorGossipInterval,
self,
GossipTick)
/**
* Start periodic metrics collection
*/
val sampleTask = scheduler.schedule(
val sampleTask = scheduler.scheduleWithFixedDelay(
PeriodicTasksInitialDelay max CollectorSampleInterval,
CollectorSampleInterval,
self,

View file

@ -200,7 +200,7 @@ private[akka] class Shard(
import context.dispatcher
val passivateIdleTask = if (settings.passivateIdleEntityAfter > Duration.Zero) {
val idleInterval = settings.passivateIdleEntityAfter / 2
Some(context.system.scheduler.schedule(idleInterval, idleInterval, self, PassivateIdleTick))
Some(context.system.scheduler.scheduleWithFixedDelay(idleInterval, idleInterval, self, PassivateIdleTick))
} else {
None
}
@ -546,7 +546,7 @@ private[akka] class RememberEntityStarter(
val tickTask = {
val resendInterval = settings.tuningParameters.retryInterval
context.system.scheduler.schedule(resendInterval, resendInterval, self, Tick)
context.system.scheduler.scheduleWithFixedDelay(resendInterval, resendInterval, self, Tick)
}
def sendStart(ids: Set[ShardRegion.EntityId]): Unit = {

View file

@ -497,7 +497,8 @@ abstract class ShardCoordinator(
var regionTerminationInProgress = Set.empty[ActorRef]
import context.dispatcher
val rebalanceTask = context.system.scheduler.schedule(rebalanceInterval, rebalanceInterval, self, RebalanceTick)
val rebalanceTask =
context.system.scheduler.scheduleWithFixedDelay(rebalanceInterval, rebalanceInterval, self, RebalanceTick)
cluster.subscribe(self, initialStateMode = InitialStateAsEvents, ClusterShuttingDown.getClass)

View file

@ -488,7 +488,7 @@ private[akka] class ShardRegion(
// subscribe to MemberEvent, re-subscribe when restart
override def preStart(): Unit = {
cluster.subscribe(self, classOf[MemberEvent])
timers.startPeriodicTimer(Retry, Retry, retryInterval)
timers.startTimerWithFixedDelay(Retry, Retry, retryInterval)
startRegistration()
if (settings.passivateIdleEntityAfter > Duration.Zero)
log.info(

View file

@ -367,7 +367,8 @@ final class ClusterClient(settings: ClusterClientSettings) extends Actor with Ac
var subscribers = Vector.empty[ActorRef]
import context.dispatcher
val heartbeatTask = context.system.scheduler.schedule(heartbeatInterval, heartbeatInterval, self, HeartbeatTick)
val heartbeatTask =
context.system.scheduler.scheduleWithFixedDelay(heartbeatInterval, heartbeatInterval, self, HeartbeatTick)
var refreshContactsTask: Option[Cancellable] = None
scheduleRefreshContactsTick(establishingGetContactsInterval)
self ! RefreshContactsTick
@ -376,7 +377,8 @@ final class ClusterClient(settings: ClusterClientSettings) extends Actor with Ac
def scheduleRefreshContactsTick(interval: FiniteDuration): Unit = {
refreshContactsTask.foreach { _.cancel() }
refreshContactsTask = Some(context.system.scheduler.schedule(interval, interval, self, RefreshContactsTick))
refreshContactsTask = Some(
context.system.scheduler.scheduleWithFixedDelay(interval, interval, self, RefreshContactsTick))
}
override def postStop(): Unit = {
@ -933,8 +935,11 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
var subscribers = Vector.empty[ActorRef]
val checkDeadlinesTask =
context.system.scheduler.schedule(failureDetectionInterval, failureDetectionInterval, self, CheckDeadlines)(
context.dispatcher)
context.system.scheduler.scheduleWithFixedDelay(
failureDetectionInterval,
failureDetectionInterval,
self,
CheckDeadlines)(context.dispatcher)
override def preStart(): Unit = {
super.preStart()

View file

@ -328,7 +328,7 @@ object DistributedPubSubMediator {
trait TopicLike extends Actor {
import context.dispatcher
val pruneInterval: FiniteDuration = emptyTimeToLive / 2
val pruneTask = context.system.scheduler.schedule(pruneInterval, pruneInterval, self, Prune)
val pruneTask = context.system.scheduler.scheduleWithFixedDelay(pruneInterval, pruneInterval, self, Prune)
var pruneDeadline: Option[Deadline] = None
var subscribers = Set.empty[ActorRef]
@ -552,9 +552,9 @@ class DistributedPubSubMediator(settings: DistributedPubSubSettings)
//Start periodic gossip to random nodes in cluster
import context.dispatcher
val gossipTask = context.system.scheduler.schedule(gossipInterval, gossipInterval, self, GossipTick)
val gossipTask = context.system.scheduler.scheduleWithFixedDelay(gossipInterval, gossipInterval, self, GossipTick)
val pruneInterval: FiniteDuration = removedTimeToLive / 2
val pruneTask = context.system.scheduler.schedule(pruneInterval, pruneInterval, self, Prune)
val pruneTask = context.system.scheduler.scheduleWithFixedDelay(pruneInterval, pruneInterval, self, Prune)
var registry: Map[Address, Bucket] = Map.empty.withDefault(a => Bucket(a, 0L, TreeMap.empty))
var nodes: Set[Address] = Set.empty

View file

@ -569,7 +569,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
// subscribe to cluster changes, re-subscribe when restart
cluster.subscribe(self, ClusterEvent.InitialStateAsEvents, classOf[MemberRemoved], classOf[MemberDowned])
setTimer(CleanupTimer, Cleanup, 1.minute, repeat = true)
startTimerWithFixedDelay(CleanupTimer, Cleanup, 1.minute)
// defer subscription to avoid some jitter when
// starting/joining several nodes at the same time
@ -723,7 +723,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
if (count <= maxHandOverRetries) {
logInfo("Retry [{}], sending HandOverToMe to [{}]", count, previousOldestOption.map(_.address))
previousOldestOption.foreach(node => peer(node.address) ! HandOverToMe)
setTimer(HandOverRetryTimer, HandOverRetry(count + 1), handOverRetryInterval, repeat = false)
startSingleTimer(HandOverRetryTimer, HandOverRetry(count + 1), handOverRetryInterval)
stay()
} else if (previousOldestOption.forall(removed.contains)) {
// can't send HandOverToMe, previousOldest unknown for new node (or restart)
@ -771,7 +771,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
if (result) {
gotoOldest()
} else {
setTimer(LeaseRetryTimer, LeaseRetry, leaseRetryInterval)
startSingleTimer(LeaseRetryTimer, LeaseRetry, leaseRetryInterval)
stay.using(AcquiringLeaseData(leaseRequestInProgress = false, None))
}
case Event(Terminated(ref), AcquiringLeaseData(_, Some(singleton))) if ref == singleton =>
@ -780,7 +780,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
tryAcquireLease()
case Event(AcquireLeaseFailure(t), _) =>
log.error(t, "failed to get lease (will be retried)")
setTimer(LeaseRetryTimer, LeaseRetry, leaseRetryInterval)
startSingleTimer(LeaseRetryTimer, LeaseRetry, leaseRetryInterval)
stay.using(AcquiringLeaseData(leaseRequestInProgress = false, None))
case Event(LeaseRetry, _) =>
// If lease was lost (so previous state was oldest) then we don't try and get the lease
@ -827,11 +827,11 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
case Some(a) =>
// send TakeOver request in case the new oldest doesn't know previous oldest
peer(a.address) ! TakeOverFromMe
setTimer(TakeOverRetryTimer, TakeOverRetry(1), handOverRetryInterval, repeat = false)
startSingleTimer(TakeOverRetryTimer, TakeOverRetry(1), handOverRetryInterval)
goto(WasOldest).using(WasOldestData(singleton, newOldestOption = Some(a)))
case None =>
// new oldest will initiate the hand-over
setTimer(TakeOverRetryTimer, TakeOverRetry(1), handOverRetryInterval, repeat = false)
startSingleTimer(TakeOverRetryTimer, TakeOverRetry(1), handOverRetryInterval)
goto(WasOldest).using(WasOldestData(singleton, newOldestOption = None))
}
}
@ -890,7 +890,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
else
log.debug("Retry [{}], sending TakeOverFromMe to [{}]", count, newOldestOption.map(_.address))
newOldestOption.foreach(node => peer(node.address) ! TakeOverFromMe)
setTimer(TakeOverRetryTimer, TakeOverRetry(count + 1), handOverRetryInterval, repeat = false)
startSingleTimer(TakeOverRetryTimer, TakeOverRetry(count + 1), handOverRetryInterval)
stay
} else
throw new ClusterSingletonManagerIsStuck(s"Expected hand-over to [$newOldestOption] never occurred")
@ -1043,7 +1043,7 @@ class ClusterSingletonManager(singletonProps: Props, terminationMessage: Any, se
}
onTransition {
case _ -> BecomingOldest => setTimer(HandOverRetryTimer, HandOverRetry(1), handOverRetryInterval, repeat = false)
case _ -> BecomingOldest => startSingleTimer(HandOverRetryTimer, HandOverRetry(1), handOverRetryInterval)
}
onTransition {

View file

@ -16,7 +16,6 @@ import akka.cluster.ClusterEvent.CurrentClusterState
import akka.cluster.ClusterEvent.MemberExited
import scala.concurrent.duration._
import scala.language.postfixOps
import com.typesafe.config.Config
import akka.actor.NoSerializationVerificationNeeded
import akka.event.Logging
@ -219,8 +218,11 @@ final class ClusterSingletonProxy(singletonManagerPath: String, settings: Cluste
singleton = None
cancelTimer()
identifyTimer = Some(
context.system.scheduler
.schedule(0 milliseconds, singletonIdentificationInterval, self, ClusterSingletonProxy.TryToIdentifySingleton))
context.system.scheduler.scheduleWithFixedDelay(
Duration.Zero,
singletonIdentificationInterval,
self,
ClusterSingletonProxy.TryToIdentifySingleton))
}
def trackChange(block: () => Unit): Unit = {

View file

@ -128,11 +128,11 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
// also periodic cleanup in case removal from ORMultiMap is skipped due to concurrent update,
// which is possible for OR CRDTs - done with an adapter to leverage the existing NodesRemoved message
timers.startPeriodicTimer("remove-nodes", RemoveTick, setup.settings.pruningInterval)
timers.startTimerWithFixedDelay("remove-nodes", RemoveTick, setup.settings.pruningInterval)
// default tomstone keepalive is 24h (based on prune-gossip-tombstones-after) and keeping the actorrefs
// around isn't very costly so don't prune often
timers.startPeriodicTimer("prune-tombstones", PruneTombstonesTick, setup.keepTombstonesFor / 24)
timers.startTimerWithFixedDelay("prune-tombstones", PruneTombstonesTick, setup.keepTombstonesFor / 24)
behavior(setup, registry, TypedMultiMap.empty[AbstractServiceKey, SubscriptionsKV])
}

View file

@ -164,6 +164,7 @@ class Cluster(val system: ExtendedActorSystem) extends Extension {
override def maxFrequency: Double = systemScheduler.maxFrequency
@silent
override def schedule(initialDelay: FiniteDuration, interval: FiniteDuration, runnable: Runnable)(
implicit executor: ExecutionContext): Cancellable =
systemScheduler.schedule(initialDelay, interval, runnable)

View file

@ -377,17 +377,18 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef, joinConfigCompatCh
import context.dispatcher
// start periodic gossip to random nodes in cluster
val gossipTask = scheduler.schedule(PeriodicTasksInitialDelay.max(GossipInterval), GossipInterval, self, GossipTick)
val gossipTask =
scheduler.scheduleWithFixedDelay(PeriodicTasksInitialDelay.max(GossipInterval), GossipInterval, self, GossipTick)
// start periodic cluster failure detector reaping (moving nodes condemned by the failure detector to unreachable list)
val failureDetectorReaperTask = scheduler.schedule(
val failureDetectorReaperTask = scheduler.scheduleWithFixedDelay(
PeriodicTasksInitialDelay.max(UnreachableNodesReaperInterval),
UnreachableNodesReaperInterval,
self,
ReapUnreachableTick)
// start periodic leader action management (only applies for the current leader)
val leaderActionsTask = scheduler.schedule(
val leaderActionsTask = scheduler.scheduleWithFixedDelay(
PeriodicTasksInitialDelay.max(LeaderActionsInterval),
LeaderActionsInterval,
self,
@ -397,7 +398,7 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef, joinConfigCompatCh
val publishStatsTask: Option[Cancellable] = PublishStatsInterval match {
case Duration.Zero | _: Duration.Infinite => None
case d: FiniteDuration =>
Some(scheduler.schedule(PeriodicTasksInitialDelay.max(d), d, self, PublishStatsTick))
Some(scheduler.scheduleWithFixedDelay(PeriodicTasksInitialDelay.max(d), d, self, PublishStatsTick))
}
override def preStart(): Unit = {
@ -1514,7 +1515,7 @@ private[cluster] final class FirstSeedNodeProcess(
// retry until one ack, or all nack, or timeout
import context.dispatcher
val retryTask = cluster.scheduler.schedule(1.second, 1.second, self, JoinSeedNode)
val retryTask = cluster.scheduler.scheduleWithFixedDelay(1.second, 1.second, self, JoinSeedNode)
self ! JoinSeedNode
override def postStop(): Unit = retryTask.cancel()

View file

@ -104,7 +104,11 @@ private[cluster] final class ClusterHeartbeatSender extends Actor with ActorLogg
// start periodic heartbeat to other nodes in cluster
val heartbeatTask =
scheduler.schedule(PeriodicTasksInitialDelay max HeartbeatInterval, HeartbeatInterval, self, HeartbeatTick)
scheduler.scheduleWithFixedDelay(
PeriodicTasksInitialDelay max HeartbeatInterval,
HeartbeatInterval,
self,
HeartbeatTick)
// used for logging warning if actual tick interval is unexpected (e.g. due to starvation)
private var tickTimestamp = System.nanoTime() + (PeriodicTasksInitialDelay max HeartbeatInterval).toNanos

View file

@ -65,7 +65,7 @@ private[cluster] final class CrossDcHeartbeatSender extends Actor with ActorLogg
immutable.SortedSet.empty)
// start periodic heartbeat to other nodes in cluster
val heartbeatTask = scheduler.schedule(
val heartbeatTask = scheduler.scheduleWithFixedDelay(
PeriodicTasksInitialDelay max HeartbeatInterval,
HeartbeatInterval,
self,

View file

@ -362,7 +362,7 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
}
import context.dispatcher
val checkPhiTask = context.system.scheduler.schedule(1.second, 1.second, self, PhiTick)
val checkPhiTask = context.system.scheduler.scheduleWithFixedDelay(1.second, 1.second, self, PhiTick)
// subscribe to MemberEvent, re-subscribe when restart
override def preStart(): Unit = cluster.subscribe(self, classOf[MemberEvent])
@ -466,7 +466,7 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
var startTime = 0L
import context.dispatcher
val resendTask = context.system.scheduler.schedule(3.seconds, 3.seconds, self, RetryTick)
val resendTask = context.system.scheduler.scheduleWithFixedDelay(3.seconds, 3.seconds, self, RetryTick)
override def postStop(): Unit = {
resendTask.cancel()

View file

@ -1232,14 +1232,19 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
//Start periodic gossip to random nodes in cluster
import context.dispatcher
val gossipTask = context.system.scheduler.schedule(gossipInterval, gossipInterval, self, GossipTick)
val gossipTask = context.system.scheduler.scheduleWithFixedDelay(gossipInterval, gossipInterval, self, GossipTick)
val notifyTask =
context.system.scheduler.schedule(notifySubscribersInterval, notifySubscribersInterval, self, FlushChanges)
context.system.scheduler.scheduleWithFixedDelay(
notifySubscribersInterval,
notifySubscribersInterval,
self,
FlushChanges)
val pruningTask =
if (pruningInterval >= Duration.Zero)
Some(context.system.scheduler.schedule(pruningInterval, pruningInterval, self, RemovedNodePruningTick))
Some(
context.system.scheduler.scheduleWithFixedDelay(pruningInterval, pruningInterval, self, RemovedNodePruningTick))
else None
val clockTask = context.system.scheduler.schedule(gossipInterval, gossipInterval, self, ClockTick)
val clockTask = context.system.scheduler.scheduleWithFixedDelay(gossipInterval, gossipInterval, self, ClockTick)
val serializer = SerializationExtension(context.system).serializerFor(classOf[DataEnvelope])
val maxPruningDisseminationNanos = maxPruningDissemination.toNanos
@ -1291,7 +1296,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
val deltaPropagationInterval = (gossipInterval / deltaPropagationSelector.gossipIntervalDivisor).max(200.millis)
Some(
context.system.scheduler
.schedule(deltaPropagationInterval, deltaPropagationInterval, self, DeltaPropagationTick))
.scheduleWithFixedDelay(deltaPropagationInterval, deltaPropagationInterval, self, DeltaPropagationTick))
} else None
// cluster nodes, doesn't contain selfAddress, doesn't contain joining and weaklyUp

View file

@ -159,7 +159,7 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
.millis
private val readCache = new SmallCache[Read, Array[Byte]](4, cacheTimeToLive, m => readToProto(m).toByteArray)
private val writeCache = new SmallCache[Write, Array[Byte]](4, cacheTimeToLive, m => writeToProto(m).toByteArray)
system.scheduler.schedule(cacheTimeToLive, cacheTimeToLive / 2) {
system.scheduler.scheduleWithFixedDelay(cacheTimeToLive, cacheTimeToLive / 2) { () =>
readCache.evict()
writeCache.evict()
}(system.dispatchers.internalDispatcher)

View file

@ -74,9 +74,9 @@ class LotsOfDataBot extends Actor with ActorLogging {
val isPassive = context.system.settings.config.getBoolean("passive")
var tickTask =
if (isPassive)
context.system.scheduler.schedule(1.seconds, 1.seconds, self, Tick)
context.system.scheduler.scheduleWithFixedDelay(1.seconds, 1.seconds, self, Tick)
else
context.system.scheduler.schedule(20.millis, 20.millis, self, Tick)
context.system.scheduler.scheduleWithFixedDelay(20.millis, 20.millis, self, Tick)
val startTime = System.nanoTime()
var count = 1L
@ -94,7 +94,7 @@ class LotsOfDataBot extends Actor with ActorLogging {
if (count == maxEntries) {
log.info("Reached {} entries", count)
tickTask.cancel()
tickTask = context.system.scheduler.schedule(1.seconds, 1.seconds, self, Tick)
tickTask = context.system.scheduler.scheduleWithFixedDelay(1.seconds, 1.seconds, self, Tick)
}
val key = ORSetKey[String]((count % maxEntries).toString)
if (count <= 100)

View file

@ -903,6 +903,10 @@ Scala
Java
: @@snip [ActorDocTest.java](/akka-docs/src/test/java/jdocs/actor/TimerDocTest.java) { #timers }
The @ref:[Scheduler](scheduler.md#schedule-periodically) documentation describes the difference between
`fixed-delay` and `fixed-rate` scheduling. If you are uncertain of which one to use you should pick
`startTimerWithFixedDelay`.
Each timer has a key and can be replaced or cancelled. It's guaranteed that a message from the
previous incarnation of the timer with the same key is not received, even though it might already
be enqueued in the mailbox when it was cancelled or the new timer was started.

View file

@ -447,15 +447,20 @@ Besides state timeouts, FSM manages timers identified by `String` names.
You may set a timer using
```
setTimer(name, msg, interval, repeat)
startSingleTimer(name, msg, interval)
startTimerWithFixedDelay(name, msg, interval)
```
where `msg` is the message object which will be sent after the duration
`interval` has elapsed. If `repeat` is `true`, then the timer is
scheduled at fixed rate given by the `interval` parameter.
`interval` has elapsed.
Any existing timer with the same name will automatically be canceled before
adding the new timer.
The @ref:[Scheduler](scheduler.md#schedule-periodically) documentation describes the difference between
`fixed-delay` and `fixed-rate` scheduling. If you are uncertain of which one to use you should pick
`startTimerWithFixedDelay`.
Timers may be canceled using
```

View file

@ -211,6 +211,25 @@ A full cluster restart is required to change to Artery.
Classic remoting depends on Netty and Artery UDP depends on Aeron. These are now both optional dependencies that need
to be explicitly added. See @ref[classic remoting](../remoting.md) or [artery remoting](../remoting-artery.md) for instructions.
## Schedule periodically with fixed-delay vs. fixed-rate
The `Scheduler.schedule` method has been deprecated in favor of selecting `scheduleWithFixedDelay` or
`scheduleAtFixedRate`.
The @ref:[Scheduler](../scheduler.md#schedule-periodically) documentation describes the difference between
`fixed-delay` and `fixed-rate` scheduling. If you are uncertain of which one to use you should pick
`startTimerWithFixedDelay`.
The deprecated `schedule` method had the same semantics as `scheduleAtFixedRate`, but since that can result in
bursts of scheduled tasks or messages after long garbage collection pauses and in worst case cause undesired
load on the system `scheduleWithFixedDelay` is often preferred.
For the same reason the following methods have also been deprecated:
* `TimerScheduler.startPeriodicTimer`, replaced by `startTimerWithFixedDelay` or `startTimerAtFixedRate`
* `FSM.setTimer`, replaced by `startSingleTimer`, `startTimerWithFixedDelay` or `startTimerAtFixedRate`
* `PersistentFSM.setTimer`, replaced by `startSingleTimer`, `startTimerWithFixedDelay` or `startTimerAtFixedRate`
## Streams
### StreamRefs
@ -274,7 +293,10 @@ Akka Typed APIs are still marked as [may change](../common/may-change.md) and th
* Factory method `Entity.ofPersistentEntity` is renamed to `Entity.ofEventSourcedEntity` in the Java API for Akka Cluster Sharding Typed.
* New abstract class `EventSourcedEntityWithEnforcedReplies` in Java API for Akka Cluster Sharding Typed and corresponding factory method `Entity.ofEventSourcedEntityWithEnforcedReplies` to ease the creation of `EventSourcedBehavior` with enforced replies.
* New method `EventSourcedEntity.withEnforcedReplies` added to Scala API to ease the creation of `EventSourcedBehavior` with enforced replies.
* `ActorSystem.scheduler` previously gave access to the untyped `akka.actor.Scheduler` but now returns a typed specific `akka.actor.typed.Scheduler`. Additionally `.schedule` has been renamed to `.scheduleAtFixedRate`. Actors that needs to schedule tasks should prefer `Behaviors.withTimers`.
* `ActorSystem.scheduler` previously gave access to the untyped `akka.actor.Scheduler` but now returns a typed specific `akka.actor.typed.Scheduler`.
Additionally `schedule` method has been replaced by `scheduleWithFixedDelay` and `scheduleAtFixedRate`. Actors that needs to schedule tasks should
prefer `Behaviors.withTimers`.
* `TimerScheduler.startPeriodicTimer`, replaced by `startTimerWithFixedDelay` or `startTimerAtFixedRate`
* `Routers.pool` now take a factory function rather than a `Behavior` to protect against accidentally sharing same behavior instance and state across routees.
### Akka Typed Stream API changes

View file

@ -93,10 +93,6 @@ necessary parameters) and then call the method when the message is received.
@@@
## From `akka.actor.ActorSystem`
@@snip [ActorSystem.scala](/akka-actor/src/main/scala/akka/actor/ActorSystem.scala) { #scheduler }
@@@ warning
All scheduled task will be executed when the `ActorSystem` is terminated, i.e.
@ -104,22 +100,60 @@ the task may execute before its timeout.
@@@
## Schedule periodically
Scheduling of recurring tasks or messages can have two different characteristics:
* fixed-delay - The delay between subsequent execution will always be (at least) the given `delay`.
Use `scheduleWithFixedDelay`.
* fixed-rate - The frequency of execution over time will meet the given `interval`. Use `scheduleAtFixedRate`.
If you are uncertain of which one to use you should pick `scheduleWithFixedDelay`.
When using **fixed-delay** it will not compensate the delay between tasks or messages if the execution takes long
time or if scheduling is delayed longer than specified for some reason. The delay between subsequent execution
will always be (at least) the given `delay`. In the long run, the frequency of execution will generally be
slightly lower than the reciprocal of the specified `delay`.
Fixed-delay execution is appropriate for recurring activities that require "smoothness." In other words,
it is appropriate for activities where it is more important to keep the frequency accurate in the short run
than in the long run.
When using **fixed-rate** it will compensate the delay for a subsequent task if the previous tasks took
too long to execute. For example, if the given `interval` is 1000 milliseconds and a task takes 200 milliseconds to
execute the next task will be scheduled to run after 800 milliseconds. In such cases, the actual execution
interval will differ from the interval passed to the `scheduleAtFixedRate` method.
If the execution of the tasks takes longer than the `interval`, the subsequent execution will start immediately
after the prior one completes (there will be no overlap of executions). This also has the consequence that after
long garbage collection pauses or other reasons when the JVM was suspended all "missed" tasks will execute
when the process wakes up again. For example, `scheduleAtFixedRate` with an interval of 1 second and the process
is suspended for 30 seconds will result in 30 tasks (or messages) being executed in rapid succession to catch up.
In the long run, the frequency of execution will be exactly the reciprocal of the specified `interval`.
Fixed-rate execution is appropriate for recurring activities that are sensitive to absolute time
or where the total time to perform a fixed number of executions is important, such as a countdown
timer that ticks once every second for ten seconds.
@@@ warning
`scheduleAtFixedRate` can result in bursts of scheduled tasks or messages after long garbage collection pauses,
which may in worst case cause undesired load on the system. `scheduleWithFixedDelay` is often preferred.
@@@
## The Scheduler interface
The actual scheduler implementation is loaded reflectively upon
`ActorSystem` start-up, which means that it is possible to provide a
different one using the `akka.scheduler.implementation` configuration
property. The referenced class must implement the following interface:
Scala
: @@snip [Scheduler.scala](/akka-actor/src/main/scala/akka/actor/Scheduler.scala) { #scheduler }
Java
: @@snip [AbstractScheduler.java](/akka-actor/src/main/java/akka/actor/AbstractScheduler.java) { #scheduler }
property. The referenced class must implement the @scala[@apidoc[akka.actor.Scheduler]]@java[@apidoc[akka.actor.AbstractScheduler]]
interface.
## The Cancellable interface
Scheduling a task will result in a `Cancellable` (or throw an
Scheduling a task will result in a @apidoc[akka.actor.Cancellable] (or throw an
`IllegalStateException` if attempted after the schedulers shutdown).
This allows you to cancel something that has been scheduled for execution.
@ -131,4 +165,3 @@ scheduled task was canceled or will (eventually) have run.
@@@
@@snip [Scheduler.scala](/akka-actor/src/main/scala/akka/actor/Scheduler.scala) { #cancellable }

View file

@ -244,5 +244,45 @@ This can be used with any type of `Behavior`, including `receive`, `receiveMessa
* The `TimerScheduler` is bound to the lifecycle of the actor that owns it and it's cancelled automatically when the actor is stopped.
* `Behaviors.withTimers` can also be used inside `Behaviors.supervise` and it will automatically cancel the started timers correctly when the actor is restarted, so that the new incarnation will not receive scheduled messages from previous incarnation.
### Schedule periodically
Scheduling of recurring messages can have two different characteristics:
* fixed-delay - The delay between sending subsequent messages will always be (at least) the given `delay`.
Use `startTimerWithFixedDelay`.
* fixed-rate - The frequency of execution over time will meet the given `interval`. Use `startTimerAtFixedRate`.
If you are uncertain of which one to use you should pick `startTimerWithFixedDelay`.
When using **fixed-delay** it will not compensate the delay between messages if the scheduling is delayed longer
than specified for some reason. The delay between sending subsequent messages will always be (at least) the given
`delay`. In the long run, the frequency of messages will generally be slightly lower than the reciprocal of the
specified `delay`.
Fixed-delay execution is appropriate for recurring activities that require "smoothness." In other words,
it is appropriate for activities where it is more important to keep the frequency accurate in the short run
than in the long run.
When using **fixed-rate** it will compensate the delay for a subsequent task if the previous messages were delayed
too long. In such cases, the actual sending interval will differ from the interval passed to the `scheduleAtFixedRate`
method.
If the tasks are delayed longer than the `interval`, the subsequent message will be sent immediately after the
prior one. This also has the consequence that after long garbage collection pauses or other reasons when the JVM
was suspended all "missed" tasks will execute when the process wakes up again. For example, `scheduleAtFixedRate`
with an interval of 1 second and the process is suspended for 30 seconds will result in 30 messages being sent
in rapid succession to catch up. In the long run, the frequency of execution will be exactly the reciprocal of
the specified `interval`.
Fixed-rate execution is appropriate for recurring activities that are sensitive to absolute time
or where the total time to perform a fixed number of executions is important, such as a countdown
timer that ticks once every second for ten seconds.
@@@ warning
`scheduleAtFixedRate` can result in bursts of scheduled messages after long garbage collection pauses,
which may in worst case cause undesired load on the system. `scheduleWithFixedDelay` is often preferred.
@@@

View file

@ -151,7 +151,7 @@ public class FaultHandlingDocSample {
getContext()
.getSystem()
.scheduler()
.schedule(
.scheduleWithFixedDelay(
Duration.ZERO,
Duration.ofSeconds(1L),
getSelf(),

View file

@ -5,8 +5,6 @@
package jdocs.actor;
// #imports1
import akka.actor.Props;
import jdocs.AbstractJavaTest;
import java.time.Duration;
// #imports1
@ -14,6 +12,8 @@ import java.time.Duration;
import akka.actor.Cancellable;
// #imports2
import jdocs.AbstractJavaTest;
import akka.actor.Props;
import akka.actor.AbstractActor;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
@ -35,7 +35,8 @@ public class SchedulerDocTest extends AbstractJavaTest {
// #schedule-one-off-message
system
.scheduler()
.scheduleOnce(Duration.ofMillis(50), testActor, "foo", system.dispatcher(), null);
.scheduleOnce(
Duration.ofMillis(50), testActor, "foo", system.dispatcher(), ActorRef.noSender());
// #schedule-one-off-message
// #schedule-one-off-thunk
@ -76,8 +77,13 @@ public class SchedulerDocTest extends AbstractJavaTest {
Cancellable cancellable =
system
.scheduler()
.schedule(
Duration.ZERO, Duration.ofMillis(50), tickActor, "Tick", system.dispatcher(), null);
.scheduleWithFixedDelay(
Duration.ZERO,
Duration.ofMillis(50),
tickActor,
"Tick",
system.dispatcher(),
ActorRef.noSender());
// This cancels further Ticks to be sent
cancellable.cancel();

View file

@ -33,7 +33,7 @@ public class TimerDocTest {
FirstTick.class,
message -> {
// do something useful here
getTimers().startPeriodicTimer(TICK_KEY, new Tick(), Duration.ofSeconds(1));
getTimers().startTimerWithFixedDelay(TICK_KEY, new Tick(), Duration.ofSeconds(1));
})
.match(
Tick.class,

View file

@ -79,7 +79,10 @@ public class FSMDocTest extends AbstractJavaTest {
// #transition-syntax
onTransition(
matchState(Idle, Active, () -> setTimer("timeout", Tick, Duration.ofSeconds(1L), true))
matchState(
Idle,
Active,
() -> startTimerWithFixedDelay("timeout", Tick, Duration.ofSeconds(1L)))
.state(Active, null, () -> cancelTimer("timeout"))
.state(null, Idle, (f, t) -> log().info("entering Idle from " + f)));
// #transition-syntax

View file

@ -43,7 +43,8 @@ public class StatsSampleClient extends AbstractActor {
getContext()
.getSystem()
.scheduler()
.schedule(interval, interval, getSelf(), "tick", getContext().getDispatcher(), null);
.scheduleWithFixedDelay(
interval, interval, getSelf(), "tick", getContext().getDispatcher(), null);
}
// subscribe to cluster changes, MemberEvent

View file

@ -37,7 +37,7 @@ public class DataBot extends AbstractActor {
getContext()
.getSystem()
.scheduler()
.schedule(
.scheduleWithFixedDelay(
Duration.ofSeconds(5),
Duration.ofSeconds(5),
getSelf(),

View file

@ -79,7 +79,7 @@ public class RecipeGlobalRateLimit extends RecipeTest {
this.replenishTimer =
system
.scheduler()
.schedule(
.scheduleWithFixedDelay(
this.tokenRefreshPeriod,
this.tokenRefreshPeriod,
getSelf(),

View file

@ -116,7 +116,7 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
//#transition-syntax
onTransition {
case Idle -> Active => setTimer("timeout", Tick, 1 second, repeat = true)
case Idle -> Active => startTimerWithFixedDelay("timeout", Tick, 1 second)
case Active -> _ => cancelTimer("timeout")
case x -> Idle => log.info("entering Idle from " + x)
}

View file

@ -97,7 +97,7 @@ class Worker extends Actor with ActorLogging {
def receive = LoggingReceive {
case Start if progressListener.isEmpty =>
progressListener = Some(sender())
context.system.scheduler.schedule(Duration.Zero, 1 second, self, Do)
context.system.scheduler.scheduleWithFixedDelay(Duration.Zero, 1 second, self, Do)
case Do =>
counterService ! Increment(1)

View file

@ -54,7 +54,7 @@ class SchedulerDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
//This will schedule to send the Tick-message
//to the tickActor after 0ms repeating every 50ms
val cancellable =
system.scheduler.schedule(0 milliseconds, 50 milliseconds, tickActor, Tick)
system.scheduler.scheduleWithFixedDelay(Duration.Zero, 50.milliseconds, tickActor, Tick)
//This cancels further Ticks to be sent
cancellable.cancel()

View file

@ -24,7 +24,7 @@ object TimerDocSpec {
def receive = {
case FirstTick =>
// do something useful here
timers.startPeriodicTimer(TickKey, Tick, 1.second)
timers.startTimerWithFixedDelay(TickKey, Tick, 1.second)
case Tick =>
// do something useful here
}

View file

@ -55,7 +55,7 @@ object TransformationFrontend {
val counter = new AtomicInteger
import system.dispatcher
system.scheduler.schedule(2.seconds, 2.seconds) {
system.scheduler.scheduleWithFixedDelay(2.seconds, 2.seconds) { () =>
implicit val timeout = Timeout(5 seconds)
(frontend ? TransformationJob("hello-" + counter.incrementAndGet())).foreach { result =>
println(result)

View file

@ -66,7 +66,7 @@ object DistributedDataDocSpec {
implicit val node = DistributedData(context.system).selfUniqueAddress
import context.dispatcher
val tickTask = context.system.scheduler.schedule(5.seconds, 5.seconds, self, Tick)
val tickTask = context.system.scheduler.scheduleWithFixedDelay(5.seconds, 5.seconds, self, Tick)
val DataKey = ORSetKey[String]("key")

View file

@ -36,7 +36,7 @@ class MyEventsByTagSource(tag: String, offset: Long, refreshInterval: FiniteDura
private val serialization = SerializationExtension(system)
override def preStart(): Unit = {
schedulePeriodically(Continue, refreshInterval)
scheduleWithFixedDelay(Continue, refreshInterval, refreshInterval)
}
override def onPull(): Unit = {

View file

@ -37,9 +37,9 @@ class RecipeGlobalRateLimit extends RecipeSpec {
private var waitQueue = immutable.Queue.empty[ActorRef]
private var permitTokens = maxAvailableTokens
private val replenishTimer = system.scheduler.schedule(
private val replenishTimer = system.scheduler.scheduleWithFixedDelay(
initialDelay = tokenRefreshPeriod,
interval = tokenRefreshPeriod,
delay = tokenRefreshPeriod,
receiver = self,
ReplenishTokens)

View file

@ -154,7 +154,7 @@ class TestKitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
assert(fsm.stateName == 1)
assert(fsm.isTimerActive("test") == false)
fsm.setTimer("test", 12, 10 millis, true)
fsm.startTimerWithFixedDelay("test", 12, 10 millis)
assert(fsm.isTimerActive("test") == true)
fsm.cancelTimer("test")
assert(fsm.isTimerActive("test") == false)

View file

@ -622,7 +622,7 @@ private[akka] class BarrierCoordinator
}
onTransition {
case Idle -> Waiting => setTimer("Timeout", StateTimeout, nextStateData.deadline.timeLeft, false)
case Idle -> Waiting => startSingleTimer("Timeout", StateTimeout, nextStateData.deadline.timeLeft)
case Waiting -> Idle => cancelTimer("Timeout")
}
@ -633,7 +633,7 @@ private[akka] class BarrierCoordinator
val enterDeadline = getDeadline(timeout)
// we only allow the deadlines to get shorter
if (enterDeadline.timeLeft < deadline.timeLeft) {
setTimer("Timeout", StateTimeout, enterDeadline.timeLeft, false)
startSingleTimer("Timeout", StateTimeout, enterDeadline.timeLeft)
handleBarrier(d.copy(arrived = together, deadline = enterDeadline))
} else
handleBarrier(d.copy(arrived = together))

View file

@ -158,7 +158,8 @@ private[akka] class LiveEventsByPersistenceIdPublisher(
import EventsByPersistenceIdPublisher._
val tickTask: Cancellable =
context.system.scheduler.schedule(refreshInterval, refreshInterval, self, Continue)(context.dispatcher)
context.system.scheduler.scheduleWithFixedDelay(refreshInterval, refreshInterval, self, Continue)(
context.dispatcher)
override def postStop(): Unit =
tickTask.cancel()

View file

@ -142,7 +142,8 @@ private[akka] class LiveEventsByTagPublisher(
import EventsByTagPublisher._
val tickTask: Cancellable =
context.system.scheduler.schedule(refreshInterval, refreshInterval, self, Continue)(context.dispatcher)
context.system.scheduler.scheduleWithFixedDelay(refreshInterval, refreshInterval, self, Continue)(
context.dispatcher)
override def postStop(): Unit =
tickTask.cancel()

View file

@ -97,10 +97,10 @@ private[akka] final class BehaviorSetup[C, E, S](
if (snapshot)
context.scheduleOnce(settings.recoveryEventTimeout, context.self, RecoveryTickEvent(snapshot = true))
else
context.system.scheduler.scheduleAtFixedRate(
settings.recoveryEventTimeout,
settings.recoveryEventTimeout,
() => context.self ! RecoveryTickEvent(snapshot = false))
context.system.scheduler.scheduleWithFixedDelay(settings.recoveryEventTimeout, settings.recoveryEventTimeout) {
() =>
context.self ! RecoveryTickEvent(snapshot = false)
}
recoveryTimer = OptionVal.Some(timer)
}

View file

@ -143,7 +143,7 @@ object PersistentActorCompileOnlyTest {
})
Behaviors.withTimers((timers: TimerScheduler[Command]) => {
timers.startPeriodicTimer("swing", MoodSwing, 10.seconds)
timers.startTimerWithFixedDelay("swing", MoodSwing, 10.seconds)
b
})
}

View file

@ -1,3 +1,12 @@
# excludes for 2.6
# Remove deprecated features since 2.5.0 https://github.com/akka/akka/issues/26492
ProblemFilters.exclude[MissingClassProblem]("akka.persistence.UntypedPersistentActor")
ProblemFilters.exclude[MissingClassProblem]("akka.persistence.UntypedPersistentActorWithAtLeastOnceDelivery")
# #26910 scheduleWithFixedDelay vs scheduleAtFixedRate
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.persistence.fsm.PersistentFSM#Timer.apply")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.persistence.fsm.PersistentFSM#Timer.repeat")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.persistence.fsm.PersistentFSM#Timer.copy$default$3")
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.persistence.fsm.PersistentFSM#Timer.this")
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.persistence.fsm.PersistentFSM#Timer.copy")

View file

@ -243,7 +243,7 @@ trait AtLeastOnceDeliveryLike extends Eventsourced {
if (redeliverTask.isEmpty) {
val interval = redeliverInterval / 2
redeliverTask = Some(
context.system.scheduler.schedule(interval, interval, self, RedeliveryTick)(context.dispatcher))
context.system.scheduler.scheduleWithFixedDelay(interval, interval, self, RedeliveryTick)(context.dispatcher))
}
}

View file

@ -709,7 +709,7 @@ private[persistence] trait Eventsourced
// protect against snapshot stalling forever because of journal overloaded and such
val timeoutCancellable = {
import context.dispatcher
context.system.scheduler.schedule(timeout, timeout, self, RecoveryTick(snapshot = false))
context.system.scheduler.scheduleWithFixedDelay(timeout, timeout, self, RecoveryTick(snapshot = false))
}
var eventSeenInInterval = false
var _recoveryRunning = true

View file

@ -293,11 +293,35 @@ object PersistentFSM {
@InternalApi
private[persistence] final case class TimeoutMarker(generation: Long)
/** INTERNAL API */
@InternalApi
private[persistence] sealed trait TimerMode {
def repeat: Boolean
}
/** INTERNAL API */
@InternalApi
private[persistence] case object FixedRateMode extends TimerMode {
override def repeat: Boolean = true
}
/** INTERNAL API */
@InternalApi
private[persistence] case object FixedDelayMode extends TimerMode {
override def repeat: Boolean = true
}
/** INTERNAL API */
@InternalApi
private[persistence] case object SingleMode extends TimerMode {
override def repeat: Boolean = false
}
/**
* INTERNAL API
*/
@InternalApi
private[persistence] final case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int, owner: AnyRef)(
private[persistence] final case class Timer(name: String, msg: Any, mode: TimerMode, generation: Int, owner: AnyRef)(
context: ActorContext)
extends NoSerializationVerificationNeeded {
private var ref: Option[Cancellable] = _
@ -309,9 +333,11 @@ object PersistentFSM {
case m: AutoReceivedMessage => m
case _ => this
}
ref = Some(
if (repeat) scheduler.schedule(timeout, timeout, actor, timerMsg)
else scheduler.scheduleOnce(timeout, actor, timerMsg))
ref = Some(mode match {
case SingleMode => scheduler.scheduleOnce(timeout, actor, timerMsg)
case FixedDelayMode => scheduler.scheduleWithFixedDelay(timeout, timeout, actor, timerMsg)
case FixedRateMode => scheduler.scheduleAtFixedRate(timeout, timeout, actor, timerMsg)
})
}
def cancel(): Unit =
if (ref.isDefined) {

View file

@ -6,12 +6,12 @@ package akka.persistence.fsm
import akka.actor._
import akka.japi.pf.{ FSMTransitionHandlerBuilder, UnitMatch, UnitPFBuilder }
import language.implicitConversions
import scala.collection.mutable
import akka.routing.{ Deafen, Listen, Listeners }
import akka.util.unused
import akka.util.JavaDurationConverters._
import scala.concurrent.duration.FiniteDuration
/**
@ -88,8 +88,8 @@ import scala.concurrent.duration.FiniteDuration
* repeated timers which arrange for the sending of a user-specified message:
*
* <pre>
* setTimer("tock", TockMsg, 1 second, true) // repeating
* setTimer("lifetime", TerminateMsg, 1 hour, false) // single-shot
* startTimerWithFixedDelay("tock", TockMsg, 1 second) // repeating
* startSingleTimer("lifetime", TerminateMsg, 1 hour) // single-shot
* cancelTimer("tock")
* isTimerActive("tock")
* </pre>
@ -200,6 +200,65 @@ trait PersistentFSMBase[S, D, E] extends Actor with Listeners with ActorLogging
final def transform(func: StateFunction): TransformHelper = new TransformHelper(func)
/**
* Schedules a message to be sent repeatedly to the `self` actor with a
* fixed `delay` between messages.
*
* It will not compensate the delay between messages if scheduling is delayed
* longer than specified for some reason. The delay between sending of subsequent
* messages will always be (at least) the given `delay`.
*
* In the long run, the frequency of messages will generally be slightly lower than
* the reciprocal of the specified `delay`.
*
* Each timer has a `name` and if a new timer with same `name` is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startTimerWithFixedDelay(name: String, msg: Any, delay: FiniteDuration): Unit =
startTimer(name, msg, delay, FixedDelayMode)
/**
* Schedules a message to be sent repeatedly to the `self` actor with a
* given frequency.
*
* It will compensate the delay for a subsequent message if the sending of previous
* message was delayed more than specified. In such cases, the actual message interval
* will differ from the interval passed to the method.
*
* If the execution is delayed longer than the `interval`, the subsequent message will
* be sent immediately after the prior one. This also has the consequence that after
* long garbage collection pauses or other reasons when the JVM was suspended all
* "missed" messages will be sent when the process wakes up again.
*
* In the long run, the frequency of messages will be exactly the reciprocal of the
* specified `interval`.
*
* Warning: `startTimerAtFixedRate` can result in bursts of scheduled messages after long
* garbage collection pauses, which may in worst case cause undesired load on the system.
* Therefore `startTimerWithFixedDelay` is often preferred.
*
* Each timer has a `name` and if a new timer with same `name` is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startTimerAtFixedRate(name: String, msg: Any, interval: FiniteDuration): Unit =
startTimer(name, msg, interval, FixedRateMode)
/**
* Start a timer that will send `msg` once to the `self` actor after
* the given `delay`.
*
* Each timer has a `name` and if a new timer with same `name` is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startSingleTimer(name: String, msg: Any, delay: FiniteDuration): Unit =
startTimer(name, msg, delay, SingleMode)
/**
* Schedule named timer to deliver message after given delay, possibly repeating.
* Any existing timer with the same name will automatically be canceled before
@ -209,13 +268,23 @@ trait PersistentFSMBase[S, D, E] extends Actor with Listeners with ActorLogging
* @param timeout delay of first message delivery and between subsequent messages
* @param repeat send once if false, scheduleAtFixedRate if true
*/
@deprecated(
"Use startSingleTimer, startTimerWithFixedDelay or startTimerAtFixedRate instead. This has the same semantics as " +
"startTimerAtFixedRate, but startTimerWithFixedDelay is often preferred.",
since = "2.6.0")
final def setTimer(name: String, msg: Any, timeout: FiniteDuration, repeat: Boolean = false): Unit = {
// repeat => FixedRateMode for compatibility
val mode = if (repeat) FixedRateMode else SingleMode
startTimer(name, msg, timeout, mode)
}
private def startTimer(name: String, msg: Any, timeout: FiniteDuration, mode: TimerMode): Unit = {
if (debugEvent)
log.debug("setting " + (if (repeat) "repeating " else "") + "timer '" + name + "'/" + timeout + ": " + msg)
log.debug("setting " + (if (mode.repeat) "repeating " else "") + "timer '" + name + "'/" + timeout + ": " + msg)
if (timers contains name) {
timers(name).cancel
}
val timer = Timer(name, msg, repeat, timerGen.next, this)(context)
val timer = Timer(name, msg, mode, timerGen.next, this)(context)
timer.schedule(self, timeout)
timers(name) = timer
}
@ -416,14 +485,14 @@ trait PersistentFSMBase[S, D, E] extends Actor with Listeners with ActorLogging
if (generation == gen) {
processMsg(StateTimeout, "state timeout")
}
case t @ Timer(name, msg, repeat, gen, owner) =>
case t @ Timer(name, msg, mode, gen, owner) =>
if ((owner eq this) && (timers contains name) && (timers(name).generation == gen)) {
if (timeoutFuture.isDefined) {
timeoutFuture.get.cancel()
timeoutFuture = None
}
generation += 1
if (!repeat) {
if (!mode.repeat) {
timers -= name
}
processMsg(msg, t)
@ -1018,6 +1087,65 @@ abstract class AbstractPersistentFSMBase[S, D, E] extends PersistentFSMBase[S, D
*/
final def goTo(nextStateName: S): State = goto(nextStateName)
/**
* Schedules a message to be sent repeatedly to the `self` actor with a
* fixed `delay` between messages.
*
* It will not compensate the delay between messages if scheduling is delayed
* longer than specified for some reason. The delay between sending of subsequent
* messages will always be (at least) the given `delay`.
*
* In the long run, the frequency of messages will generally be slightly lower than
* the reciprocal of the specified `delay`.
*
* Each timer has a `name` and if a new timer with same `name` is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startTimerWithFixedDelay(name: String, msg: Any, delay: java.time.Duration): Unit =
startTimerWithFixedDelay(name, msg, delay.asScala)
/**
* Schedules a message to be sent repeatedly to the `self` actor with a
* given frequency.
*
* It will compensate the delay for a subsequent message if the sending of previous
* message was delayed more than specified. In such cases, the actual message interval
* will differ from the interval passed to the method.
*
* If the execution is delayed longer than the `interval`, the subsequent message will
* be sent immediately after the prior one. This also has the consequence that after
* long garbage collection pauses or other reasons when the JVM was suspended all
* "missed" messages will be sent when the process wakes up again.
*
* In the long run, the frequency of messages will be exactly the reciprocal of the
* specified `interval`.
*
* Warning: `startTimerAtFixedRate` can result in bursts of scheduled messages after long
* garbage collection pauses, which may in worst case cause undesired load on the system.
* Therefore `startTimerWithFixedDelay` is often preferred.
*
* Each timer has a `name` and if a new timer with same `name` is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startTimerAtFixedRate(name: String, msg: Any, interval: java.time.Duration): Unit =
startTimerAtFixedRate(name, msg, interval.asScala)
/**
* Start a timer that will send `msg` once to the `self` actor after
* the given `delay`.
*
* Each timer has a `name` and if a new timer with same `name` is started
* the previous is cancelled and it's guaranteed that a message from the
* previous timer is not received, even though it might already be enqueued
* in the mailbox when the new timer is started.
*/
def startSingleTimer(name: String, msg: Any, delay: java.time.Duration): Unit =
startSingleTimer(name, msg, delay.asScala)
/**
* Schedule named timer to deliver message after given delay, possibly repeating.
* Any existing timer with the same name will automatically be canceled before
@ -1026,6 +1154,7 @@ abstract class AbstractPersistentFSMBase[S, D, E] extends PersistentFSMBase[S, D
* @param msg message to be delivered
* @param timeout delay of first message delivery and between subsequent messages
*/
@deprecated("Use startSingleTimer instead.", since = "2.6.0")
final def setTimer(name: String, msg: Any, timeout: FiniteDuration): Unit =
setTimer(name, msg, timeout, false)

View file

@ -257,7 +257,7 @@ private[remote] class ReliableDeliverySupervisor(
import ReliableDeliverySupervisor._
import context.dispatcher
val autoResendTimer = context.system.scheduler.schedule(
val autoResendTimer = context.system.scheduler.scheduleWithFixedDelay(
settings.SysResendTimeout,
settings.SysResendTimeout,
self,
@ -672,7 +672,7 @@ private[remote] class EndpointWriter(
val ackIdleTimer = {
val interval = settings.SysMsgAckTimeout / 2
context.system.scheduler.schedule(interval, interval, self, AckIdleCheckTimer)
context.system.scheduler.scheduleWithFixedDelay(interval, interval, self, AckIdleCheckTimer)
}
override def preStart(): Unit = {

View file

@ -129,9 +129,9 @@ private[akka] class RemoteWatcher(
var unreachable: Set[Address] = Set.empty
var addressUids: Map[Address, Long] = Map.empty
val heartbeatTask = scheduler.schedule(heartbeatInterval, heartbeatInterval, self, HeartbeatTick)
val heartbeatTask = scheduler.scheduleWithFixedDelay(heartbeatInterval, heartbeatInterval, self, HeartbeatTick)
val failureDetectorReaperTask =
scheduler.schedule(unreachableReaperInterval, unreachableReaperInterval, self, ReapUnreachableTick)
scheduler.scheduleWithFixedDelay(unreachableReaperInterval, unreachableReaperInterval, self, ReapUnreachableTick)
override def postStop(): Unit = {
super.postStop()

View file

@ -499,7 +499,7 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter)
val pruneInterval: FiniteDuration = (settings.RetryGateClosedFor * 2).max(1.second).min(10.seconds)
val pruneTimerCancellable: Cancellable =
context.system.scheduler.schedule(pruneInterval, pruneInterval, self, Prune)
context.system.scheduler.scheduleWithFixedDelay(pruneInterval, pruneInterval, self, Prune)
var pendingReadHandoffs = Map[ActorRef, AkkaProtocolHandle]()
var stashedInbound = Map[ActorRef, Vector[InboundAssociation]]()

View file

@ -503,7 +503,7 @@ private[remote] abstract class ArteryTransport(_system: ExtendedActorSystem, _pr
private def startRemoveQuarantinedAssociationTask(): Unit = {
val removeAfter = settings.Advanced.RemoveQuarantinedAssociationAfter
val interval = removeAfter / 2
system.scheduler.schedule(removeAfter, interval) {
system.scheduler.scheduleWithFixedDelay(removeAfter, interval) { () =>
if (!isShutdown)
associationRegistry.removeUnusedQuarantined(removeAfter)
}(system.dispatchers.internalDispatcher)

View file

@ -601,44 +601,45 @@ private[remote] class Association(
val QuarantineIdleOutboundAfter = settings.Advanced.QuarantineIdleOutboundAfter
val interval = StopIdleOutboundAfter / 2
val initialDelay = settings.Advanced.Tcp.ConnectionTimeout.max(StopIdleOutboundAfter) + 1.second
val task = transport.system.scheduler.schedule(initialDelay, interval) {
val lastUsedDurationNanos = System.nanoTime() - associationState.lastUsedTimestamp.get
if (lastUsedDurationNanos >= QuarantineIdleOutboundAfter.toNanos && !associationState.isQuarantined()) {
// If idle longer than quarantine-idle-outbound-after and the low frequency HandshakeReq
// doesn't get through it will be quarantined to cleanup lingering associations to crashed systems.
quarantine(s"Idle longer than quarantine-idle-outbound-after [${QuarantineIdleOutboundAfter.pretty}]")
} else if (lastUsedDurationNanos >= StopIdleOutboundAfter.toNanos) {
streamMatValues.get.foreach {
case (queueIndex, OutboundStreamMatValues(streamKillSwitch, _, stopping)) =>
if (isStreamActive(queueIndex) && stopping.isEmpty) {
if (queueIndex != ControlQueueIndex) {
streamKillSwitch match {
case OptionVal.Some(k) =>
// for non-control streams we can stop the entire stream
log.info("Stopping idle outbound stream [{}] to [{}]", queueIndex, remoteAddress)
flightRecorder.loFreq(Transport_StopIdleOutbound, s"$remoteAddress - $queueIndex")
setStopReason(queueIndex, OutboundStreamStopIdleSignal)
clearStreamKillSwitch(queueIndex, k)
k.abort(OutboundStreamStopIdleSignal)
case OptionVal.None => // already aborted
}
val task =
transport.system.scheduler.scheduleWithFixedDelay(initialDelay, interval) { () =>
val lastUsedDurationNanos = System.nanoTime() - associationState.lastUsedTimestamp.get
if (lastUsedDurationNanos >= QuarantineIdleOutboundAfter.toNanos && !associationState.isQuarantined()) {
// If idle longer than quarantine-idle-outbound-after and the low frequency HandshakeReq
// doesn't get through it will be quarantined to cleanup lingering associations to crashed systems.
quarantine(s"Idle longer than quarantine-idle-outbound-after [${QuarantineIdleOutboundAfter.pretty}]")
} else if (lastUsedDurationNanos >= StopIdleOutboundAfter.toNanos) {
streamMatValues.get.foreach {
case (queueIndex, OutboundStreamMatValues(streamKillSwitch, _, stopping)) =>
if (isStreamActive(queueIndex) && stopping.isEmpty) {
if (queueIndex != ControlQueueIndex) {
streamKillSwitch match {
case OptionVal.Some(k) =>
// for non-control streams we can stop the entire stream
log.info("Stopping idle outbound stream [{}] to [{}]", queueIndex, remoteAddress)
flightRecorder.loFreq(Transport_StopIdleOutbound, s"$remoteAddress - $queueIndex")
setStopReason(queueIndex, OutboundStreamStopIdleSignal)
clearStreamKillSwitch(queueIndex, k)
k.abort(OutboundStreamStopIdleSignal)
case OptionVal.None => // already aborted
}
} else {
// only stop the transport parts of the stream because SystemMessageDelivery stage has
// state (seqno) and system messages might be sent at the same time
associationState.controlIdleKillSwitch match {
case OptionVal.Some(killSwitch) =>
log.info("Stopping idle outbound control stream to [{}]", remoteAddress)
flightRecorder.loFreq(Transport_StopIdleOutbound, s"$remoteAddress - $queueIndex")
setControlIdleKillSwitch(OptionVal.None)
killSwitch.abort(OutboundStreamStopIdleSignal)
case OptionVal.None => // already stopped
} else {
// only stop the transport parts of the stream because SystemMessageDelivery stage has
// state (seqno) and system messages might be sent at the same time
associationState.controlIdleKillSwitch match {
case OptionVal.Some(killSwitch) =>
log.info("Stopping idle outbound control stream to [{}]", remoteAddress)
flightRecorder.loFreq(Transport_StopIdleOutbound, s"$remoteAddress - $queueIndex")
setControlIdleKillSwitch(OptionVal.None)
killSwitch.abort(OutboundStreamStopIdleSignal)
case OptionVal.None => // already stopped
}
}
}
}
}
}
}
}(transport.system.dispatcher)
}(transport.system.dispatcher)
if (!idleTimer.compareAndSet(None, Some(task))) {
// another thread did same thing and won

View file

@ -371,16 +371,17 @@ private[remote] class Decoder(
override protected def logSource = classOf[Decoder]
override def preStart(): Unit = {
schedulePeriodically(Tick, 1.seconds)
val tickDelay = 1.seconds
scheduleWithFixedDelay(Tick, tickDelay, tickDelay)
if (settings.Advanced.Compression.Enabled) {
settings.Advanced.Compression.ActorRefs.AdvertisementInterval match {
case d: FiniteDuration => schedulePeriodicallyWithInitialDelay(AdvertiseActorRefsCompressionTable, d, d)
case d: FiniteDuration => scheduleWithFixedDelay(AdvertiseActorRefsCompressionTable, d, d)
case _ => // not advertising actor ref compressions
}
settings.Advanced.Compression.Manifests.AdvertisementInterval match {
case d: FiniteDuration =>
schedulePeriodicallyWithInitialDelay(AdvertiseClassManifestsCompressionTable, d, d)
scheduleWithFixedDelay(AdvertiseClassManifestsCompressionTable, d, d)
case _ => // not advertising class manifest compressions
}
}

View file

@ -75,7 +75,7 @@ private[remote] class OutboundHandshake(
override def preStart(): Unit = {
scheduleOnce(HandshakeTimeout, timeout)
livenessProbeInterval match {
case d: FiniteDuration => schedulePeriodically(LivenessProbeTick, d)
case d: FiniteDuration => scheduleWithFixedDelay(LivenessProbeTick, d, d)
case _ => // only used in control stream
}
}
@ -122,7 +122,7 @@ private[remote] class OutboundHandshake(
} else {
// will pull when handshake reply is received (uniqueRemoteAddress completed)
handshakeState = ReqInProgress
schedulePeriodically(HandshakeRetryTick, retryInterval)
scheduleWithFixedDelay(HandshakeRetryTick, retryInterval, retryInterval)
// The InboundHandshake stage will complete the uniqueRemoteAddress future
// when it receives the HandshakeRsp reply

View file

@ -256,7 +256,7 @@ private[remote] class ArteryAeronUdpTransport(_system: ExtendedActorSystem, _pro
aeronErrorLog = new AeronErrorLog(new File(aeronDir, CncFileDescriptor.CNC_FILE), log)
val lastTimestamp = new AtomicLong(0L)
implicit val ec = system.dispatchers.internalDispatcher
aeronErrorLogTask = system.scheduler.schedule(3.seconds, 5.seconds) {
aeronErrorLogTask = system.scheduler.scheduleWithFixedDelay(3.seconds, 5.seconds) { () =>
if (!isShutdown) {
val newLastTimestamp = aeronErrorLog.logErrors(log, lastTimestamp.get)
lastTimestamp.set(newLastTimestamp + 1)
@ -266,7 +266,7 @@ private[remote] class ArteryAeronUdpTransport(_system: ExtendedActorSystem, _pro
private def startAeronCounterLog(): Unit = {
implicit val ec = system.dispatchers.internalDispatcher
aeronCounterTask = system.scheduler.schedule(5.seconds, 5.seconds) {
aeronCounterTask = system.scheduler.scheduleWithFixedDelay(5.seconds, 5.seconds) { () =>
if (!isShutdown && log.isDebugEnabled) {
aeron.countersReader.forEach(new MetaData() {
def accept(counterId: Int, typeId: Int, keyBuffer: DirectBuffer, label: String): Unit = {

View file

@ -408,11 +408,10 @@ private[remote] class ProtocolStateActor(
} else {
// Underlying transport was busy -- Associate could not be sent
setTimer(
startSingleTimer(
"associate-retry",
Handle(wrappedHandle),
RARP(context.system).provider.remoteSettings.BackoffPeriod,
repeat = false)
RARP(context.system).provider.remoteSettings.BackoffPeriod)
stay()
}
@ -594,11 +593,11 @@ private[remote] class ProtocolStateActor(
}
private def initHeartbeatTimer(): Unit = {
setTimer("heartbeat-timer", HeartbeatTimer, settings.TransportHeartBeatInterval, repeat = true)
startTimerWithFixedDelay("heartbeat-timer", HeartbeatTimer, settings.TransportHeartBeatInterval)
}
private def initHandshakeTimer(): Unit = {
setTimer(handshakeTimerKey, HandshakeTimer, settings.HandshakeTimeout, repeat = false)
startSingleTimer(handshakeTimerKey, HandshakeTimer, settings.HandshakeTimeout)
}
private def handleTimers(wrappedHandle: AssociationHandle): State = {

View file

@ -570,7 +570,7 @@ private[transport] class ThrottledAssociation(
def scheduleDequeue(delay: FiniteDuration): Unit = inboundThrottleMode match {
case Blackhole => // Do nothing
case _ if delay <= Duration.Zero => self ! Dequeue
case _ => setTimer(DequeueTimerName, Dequeue, delay, repeat = false)
case _ => startSingleTimer(DequeueTimerName, Dequeue, delay)
}
}

View file

@ -72,7 +72,7 @@ object AkkaProtocolStressTest {
// the proper ordering.
if (seq > limit * 0.5) {
controller ! ((maxSeq, losses))
context.system.scheduler.schedule(1.second, 1.second, self, ResendFinal)
context.system.scheduler.scheduleWithFixedDelay(1.second, 1.second, self, ResendFinal)
context.become(done)
}
} else {

View file

@ -149,11 +149,11 @@ abstract class SystemMessageDeliveryStressTest(msg: String, cfg: String)
Await.result(transportB.managementCommand(One(addressA, Drop(0.1, 0.1))), 3.seconds.dilated)
// Schedule peridodic disassociates
systemA.scheduler.schedule(3.second, 8.seconds) {
systemA.scheduler.scheduleWithFixedDelay(3.second, 8.seconds) { () =>
transportA.managementCommand(ForceDisassociateExplicitly(addressB, reason = AssociationHandle.Unknown))
}
systemB.scheduler.schedule(7.seconds, 8.seconds) {
systemB.scheduler.scheduleWithFixedDelay(7.seconds, 8.seconds) { () =>
transportB.managementCommand(ForceDisassociateExplicitly(addressA, reason = AssociationHandle.Unknown))
}

View file

@ -79,7 +79,7 @@ class GraphStageTimersSpec extends StreamSpec {
probe ! TestCancelTimerAck
scheduleOnce("TestCancelTimer", 500.milli.dilated)
case TestRepeatedTimer =>
schedulePeriodically("TestRepeatedTimer", 100.millis.dilated)
scheduleWithFixedDelay("TestRepeatedTimer", 100.millis.dilated, 100.millis.dilated)
}
}
}
@ -156,7 +156,7 @@ class GraphStageTimersSpec extends StreamSpec {
override def createLogic(inheritedAttributes: Attributes) = new TimerGraphStageLogic(shape) {
var tickCount = 0
override def preStart(): Unit = schedulePeriodically("tick", 100.millis)
override def preStart(): Unit = scheduleWithFixedDelay("tick", 100.millis, 100.millis)
setHandler(out, new OutHandler {
override def onPull() = () // Do nothing

View file

@ -105,3 +105,7 @@ ProblemFilters.exclude[MissingClassProblem]("akka.stream.impl.SourceQueueAdapter
# Remove deprecated features since 2.5.0 https://github.com/akka/akka/issues/26492
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.ActorMaterializerSettings.withAutoFusing")
# #26910 scheduleWithFixedDelay vs scheduleAtFixedRate
# Adding methods to Materializer is not compatible but we don't support other Materializer implementations
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.Materializer.scheduleAtFixedRate")
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.Materializer.scheduleWithFixedDelay")

View file

@ -71,6 +71,65 @@ abstract class Materializer {
*/
def scheduleOnce(delay: FiniteDuration, task: Runnable): Cancellable
/**
* Interface for operators that need timer services for their functionality.
*
* Schedules a `Runnable` to be run repeatedly with an initial delay and
* a fixed `delay` between subsequent executions.
*
* It will not compensate the delay between tasks if the execution takes a long time or if
* scheduling is delayed longer than specified for some reason. The delay between subsequent
* execution will always be (at least) the given `delay`. In the long run, the
* frequency of execution will generally be slightly lower than the reciprocal of the specified
* `delay`.
*
* If the `Runnable` throws an exception the repeated scheduling is aborted,
* i.e. the function will not be invoked any more.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* supported by the `Scheduler`.
*
* @return A [[akka.actor.Cancellable]] that allows cancelling the timer. Cancelling is best effort, if the event
* has been already enqueued it will not have an effect.
*/
def scheduleWithFixedDelay(initialDelay: FiniteDuration, delay: FiniteDuration, task: Runnable): Cancellable
/**
* Interface for operators that need timer services for their functionality.
*
* Schedules a `Runnable` 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)`
* and `interval=Duration(100, TimeUnit.MILLISECONDS)`.
*
* It will compensate the delay for a subsequent task if the previous tasks took
* too long to execute. In such cases, the actual execution interval will differ from
* the interval passed to the method.
*
* If the execution of the tasks takes longer than the `interval`, the subsequent
* execution will start immediately after the prior one completes (there will be
* no overlap of executions). This also has the consequence that after long garbage
* collection pauses or other reasons when the JVM was suspended all "missed" tasks
* will execute when the process wakes up again.
*
* In the long run, the frequency of execution will be exactly the reciprocal of the
* specified `interval`.
*
* Warning: `scheduleAtFixedRate` can result in bursts of scheduled tasks after long
* garbage collection pauses, which may in worst case cause undesired load on the system.
* Therefore `scheduleWithFixedDelay` is often preferred.
*
* If the `Runnable` throws an exception the repeated scheduling is aborted,
* i.e. the function will not be invoked any more.
*
* @throws IllegalArgumentException if the given delays exceed the maximum
* supported by the `Scheduler`.
*
* @return A [[akka.actor.Cancellable]] that allows cancelling the timer. Cancelling is best effort, if the event
* has been already enqueued it will not have an effect.
*/
def scheduleAtFixedRate(initialDelay: FiniteDuration, interval: FiniteDuration, task: Runnable): Cancellable
/**
* Interface for operators that need timer services for their functionality. Schedules a
* repeated task with the given interval between invocations.
@ -78,6 +137,10 @@ abstract class Materializer {
* @return A [[akka.actor.Cancellable]] that allows cancelling the timer. Cancelling is best effort, if the event
* has been already enqueued it will not have an effect.
*/
@deprecated(
"Use scheduleWithFixedDelay or scheduleAtFixedRate instead. This has the same semantics as " +
"scheduleAtFixedRate, but scheduleWithFixedDelay is often preferred.",
since = "2.6.0")
def schedulePeriodically(initialDelay: FiniteDuration, interval: FiniteDuration, task: Runnable): Cancellable
}
@ -102,6 +165,18 @@ private[akka] object NoMaterializer extends Materializer {
def schedulePeriodically(initialDelay: FiniteDuration, interval: FiniteDuration, task: Runnable): Cancellable =
throw new UnsupportedOperationException("NoMaterializer cannot schedule a repeated event")
override def scheduleWithFixedDelay(
initialDelay: FiniteDuration,
delay: FiniteDuration,
task: Runnable): Cancellable =
throw new UnsupportedOperationException("NoMaterializer cannot scheduleWithFixedDelay")
override def scheduleAtFixedRate(
initialDelay: FiniteDuration,
interval: FiniteDuration,
task: Runnable): Cancellable =
throw new UnsupportedOperationException("NoMaterializer cannot scheduleAtFixedRate")
}
/**

View file

@ -131,11 +131,23 @@ private[akka] class SubFusingActorMaterializerImpl(
override def scheduleOnce(delay: FiniteDuration, task: Runnable): Cancellable = delegate.scheduleOnce(delay, task)
override def scheduleWithFixedDelay(
initialDelay: FiniteDuration,
delay: FiniteDuration,
task: Runnable): Cancellable =
delegate.scheduleWithFixedDelay(initialDelay, delay, task)
override def scheduleAtFixedRate(
initialDelay: FiniteDuration,
interval: FiniteDuration,
task: Runnable): Cancellable =
delegate.scheduleAtFixedRate(initialDelay, interval, task)
override def schedulePeriodically(
initialDelay: FiniteDuration,
interval: FiniteDuration,
task: Runnable): Cancellable =
delegate.schedulePeriodically(initialDelay, interval, task)
scheduleAtFixedRate(initialDelay, interval, task)
override def withNamePrefix(name: String): SubFusingActorMaterializerImpl =
new SubFusingActorMaterializerImpl(delegate.withNamePrefix(name), registerShell)

View file

@ -417,11 +417,23 @@ private final case class SavedIslandData(
override lazy val executionContext: ExecutionContextExecutor = dispatchers.lookup(settings.dispatcher)
override def scheduleWithFixedDelay(
initialDelay: FiniteDuration,
delay: FiniteDuration,
task: Runnable): Cancellable =
system.scheduler.scheduleWithFixedDelay(initialDelay, delay)(task)(executionContext)
override def scheduleAtFixedRate(
initialDelay: FiniteDuration,
interval: FiniteDuration,
task: Runnable): Cancellable =
system.scheduler.scheduleAtFixedRate(initialDelay, interval)(task)(executionContext)
override def schedulePeriodically(
initialDelay: FiniteDuration,
interval: FiniteDuration,
task: Runnable): Cancellable =
system.scheduler.schedule(initialDelay, interval, task)(executionContext)
system.scheduler.scheduleAtFixedRate(initialDelay, interval)(task)(executionContext)
override def scheduleOnce(delay: FiniteDuration, task: Runnable): Cancellable =
system.scheduler.scheduleOnce(delay, task)(executionContext)

View file

@ -109,7 +109,8 @@ import scala.concurrent.duration.{ Duration, FiniteDuration }
if (nextDeadline - System.nanoTime < 0)
failStage(new TimeoutException(s"No elements passed in the last $timeout."))
override def preStart(): Unit = schedulePeriodically(GraphStageLogicTimer, timeoutCheckInterval(timeout))
override def preStart(): Unit =
scheduleWithFixedDelay(GraphStageLogicTimer, timeoutCheckInterval(timeout), timeoutCheckInterval(timeout))
}
override def toString = "IdleTimeout"
@ -141,7 +142,8 @@ import scala.concurrent.duration.{ Duration, FiniteDuration }
if (waitingDemand && (nextDeadline - System.nanoTime < 0))
failStage(new TimeoutException(s"No demand signalled in the last $timeout."))
override def preStart(): Unit = schedulePeriodically(GraphStageLogicTimer, timeoutCheckInterval(timeout))
override def preStart(): Unit =
scheduleWithFixedDelay(GraphStageLogicTimer, timeoutCheckInterval(timeout), timeoutCheckInterval(timeout))
}
override def toString = "BackpressureTimeout"
@ -169,7 +171,8 @@ import scala.concurrent.duration.{ Duration, FiniteDuration }
if (nextDeadline - System.nanoTime < 0)
failStage(new TimeoutException(s"No elements passed in the last $timeout."))
override def preStart(): Unit = schedulePeriodically(GraphStageLogicTimer, timeoutCheckInterval(timeout))
override def preStart(): Unit =
scheduleWithFixedDelay(GraphStageLogicTimer, timeoutCheckInterval(timeout), timeoutCheckInterval(timeout))
class IdleBidiHandler[P](in: Inlet[P], out: Outlet[P]) extends InHandler with OutHandler {
override def onPush(): Unit = {

View file

@ -234,7 +234,7 @@ import scala.concurrent.{ Future, Promise }
if (cancelled.get)
completeStage()
else
schedulePeriodicallyWithInitialDelay("TickTimer", initialDelay, interval)
scheduleWithFixedDelay("TickTimer", initialDelay, interval)
}
setHandler(out, eagerTerminateOutput)

View file

@ -1605,7 +1605,7 @@ private[stream] object Collect {
private var hasElements = false
override def preStart() = {
schedulePeriodically(GroupedWeightedWithin.groupedWeightedWithinTimer, interval)
scheduleWithFixedDelay(GroupedWeightedWithin.groupedWeightedWithinTimer, interval, interval)
pull(in)
}
@ -1631,7 +1631,7 @@ private[stream] object Collect {
// we also have to pull if downstream hasn't yet requested an element.
pull(in)
} else {
schedulePeriodically(GroupedWeightedWithin.groupedWeightedWithinTimer, interval)
scheduleWithFixedDelay(GroupedWeightedWithin.groupedWeightedWithinTimer, interval, interval)
emitGroup()
}
}
@ -1645,7 +1645,7 @@ private[stream] object Collect {
pending = elem
pendingWeight = cost
}
schedulePeriodically(GroupedWeightedWithin.groupedWeightedWithinTimer, interval)
scheduleWithFixedDelay(GroupedWeightedWithin.groupedWeightedWithinTimer, interval, interval)
tryCloseGroup()
}
}

View file

@ -1561,38 +1561,6 @@ abstract class TimerGraphStageLogic(_shape: Shape) extends GraphStageLogic(_shap
}
}
/**
* Schedule timer to call [[#onTimer]] periodically with the given interval after the specified
* initial delay.
* Any existing timer with the same key will automatically be canceled before
* adding the new timer.
*/
final protected def schedulePeriodicallyWithInitialDelay(
timerKey: Any,
initialDelay: FiniteDuration,
interval: FiniteDuration): Unit = {
cancelTimer(timerKey)
val id = timerIdGen.next()
val task = interpreter.materializer.schedulePeriodically(initialDelay, interval, new Runnable {
def run() = getTimerAsyncCallback.invoke(Scheduled(timerKey, id, repeating = true))
})
keyToTimers(timerKey) = Timer(id, task)
}
/**
* Schedule timer to call [[#onTimer]] periodically with the given interval after the specified
* initial delay.
* Any existing timer with the same key will automatically be canceled before
* adding the new timer.
*/
final protected def schedulePeriodicallyWithInitialDelay(
timerKey: Any,
initialDelay: java.time.Duration,
interval: java.time.Duration): Unit = {
import akka.util.JavaDurationConverters._
schedulePeriodicallyWithInitialDelay(timerKey, initialDelay.asScala, interval.asScala)
}
/**
* Schedule timer to call [[#onTimer]] after given delay.
* Any existing timer with the same key will automatically be canceled before
@ -1617,6 +1585,130 @@ abstract class TimerGraphStageLogic(_shape: Shape) extends GraphStageLogic(_shap
scheduleOnce(timerKey, delay.asScala)
}
/**
* Schedule timer to call [[#onTimer]] periodically with the given `delay` after the specified
* initial delay.
* Any existing timer with the same key will automatically be canceled before
* adding the new timer.
*/
final protected def scheduleWithFixedDelay(
timerKey: Any,
initialDelay: FiniteDuration,
delay: FiniteDuration): Unit = {
cancelTimer(timerKey)
val id = timerIdGen.next()
val task = interpreter.materializer.scheduleWithFixedDelay(initialDelay, delay, new Runnable {
def run() = getTimerAsyncCallback.invoke(Scheduled(timerKey, id, repeating = true))
})
keyToTimers(timerKey) = Timer(id, task)
}
/**
* Schedule timer to call [[#onTimer]] periodically with the given `delay` after the specified
* initial delay.
* Any existing timer with the same key will automatically be canceled before
* adding the new timer.
*/
final protected def scheduleWithFixedDelay(
timerKey: Any,
initialDelay: java.time.Duration,
interval: java.time.Duration): Unit = {
import akka.util.JavaDurationConverters._
scheduleWithFixedDelay(timerKey, initialDelay.asScala, interval.asScala)
}
/**
* Schedule timer to call [[#onTimer]] periodically with the given `interval` after the specified
* initial delay.
* Any existing timer with the same key will automatically be canceled before
* adding the new timer.
*/
final protected def scheduleAtFixedRate(
timerKey: Any,
initialDelay: FiniteDuration,
interval: FiniteDuration): Unit = {
cancelTimer(timerKey)
val id = timerIdGen.next()
val task = interpreter.materializer.scheduleAtFixedRate(initialDelay, interval, new Runnable {
def run() = getTimerAsyncCallback.invoke(Scheduled(timerKey, id, repeating = true))
})
keyToTimers(timerKey) = Timer(id, task)
}
/**
* Schedule timer to call [[#onTimer]] periodically with the given `interval` after the specified
* initial delay.
* Any existing timer with the same key will automatically be canceled before
* adding the new timer.
*/
final protected def scheduleAtFixedRate(
timerKey: Any,
initialDelay: java.time.Duration,
interval: java.time.Duration): Unit = {
import akka.util.JavaDurationConverters._
scheduleAtFixedRate(timerKey, initialDelay.asScala, interval.asScala)
}
/**
* Schedule timer to call [[#onTimer]] periodically with the given interval after the specified
* initial delay.
* Any existing timer with the same key will automatically be canceled before
* adding the new timer.
*/
@deprecated(
"Use scheduleWithFixedDelay or scheduleAtFixedRate instead. This has the same semantics as " +
"scheduleAtFixedRate, but scheduleWithFixedDelay is often preferred.",
since = "2.6.0")
final protected def schedulePeriodicallyWithInitialDelay(
timerKey: Any,
initialDelay: FiniteDuration,
interval: FiniteDuration): Unit =
scheduleAtFixedRate(timerKey, initialDelay, interval)
/**
* Schedule timer to call [[#onTimer]] periodically with the given interval after the specified
* initial delay.
* Any existing timer with the same key will automatically be canceled before
* adding the new timer.
*/
@deprecated(
"Use scheduleWithFixedDelay or scheduleAtFixedRate instead. This has the same semantics as " +
"scheduleAtFixedRate, but scheduleWithFixedDelay is often preferred.",
since = "2.6.0")
final protected def schedulePeriodicallyWithInitialDelay(
timerKey: Any,
initialDelay: java.time.Duration,
interval: java.time.Duration): Unit = {
import akka.util.JavaDurationConverters._
schedulePeriodicallyWithInitialDelay(timerKey, initialDelay.asScala, interval.asScala)
}
/**
* Schedule timer to call [[#onTimer]] periodically with the given interval.
* Any existing timer with the same key will automatically be canceled before
* adding the new timer.
*/
@deprecated(
"Use scheduleWithFixedDelay or scheduleAtFixedRate instead. This has the same semantics as " +
"scheduleAtFixedRate, but scheduleWithFixedDelay is often preferred.",
since = "2.6.0")
final protected def schedulePeriodically(timerKey: Any, interval: FiniteDuration): Unit =
schedulePeriodicallyWithInitialDelay(timerKey, interval, interval)
/**
* Schedule timer to call [[#onTimer]] periodically with the given interval.
* Any existing timer with the same key will automatically be canceled before
* adding the new timer.
*/
@deprecated(
"Use scheduleWithFixedDelay or scheduleAtFixedRate instead. This has the same semantics as " +
"scheduleAtFixedRate, but scheduleWithFixedDelay is often preferred.",
since = "2.6.0")
final protected def schedulePeriodically(timerKey: Any, interval: java.time.Duration): Unit = {
import akka.util.JavaDurationConverters._
schedulePeriodically(timerKey, interval.asScala)
}
/**
* Cancel timer, ensuring that the [[#onTimer]] is not subsequently called.
*
@ -1635,23 +1727,6 @@ abstract class TimerGraphStageLogic(_shape: Shape) extends GraphStageLogic(_shap
*/
final protected def isTimerActive(timerKey: Any): Boolean = keyToTimers contains timerKey
/**
* Schedule timer to call [[#onTimer]] periodically with the given interval.
* Any existing timer with the same key will automatically be canceled before
* adding the new timer.
*/
final protected def schedulePeriodically(timerKey: Any, interval: FiniteDuration): Unit =
schedulePeriodicallyWithInitialDelay(timerKey, interval, interval)
/**
* Schedule timer to call [[#onTimer]] periodically with the given interval.
* Any existing timer with the same key will automatically be canceled before
* adding the new timer.
*/
final protected def schedulePeriodically(timerKey: Any, interval: java.time.Duration): Unit = {
import akka.util.JavaDurationConverters._
schedulePeriodically(timerKey, interval.asScala)
}
}
/** Java API: [[GraphStageLogic]] with [[StageLogging]]. */

View file

@ -62,9 +62,31 @@ class TestFSMRef[S, D, T <: Actor](system: ActorSystem, props: Props, supervisor
fsm.applyState(FSM.State(stateName, stateData, Option(timeout), stopReason))
}
/**
* Proxy for [[akka.actor.FSM#startTimerWithFixedDelay]].
*/
def startTimerWithFixedDelay(name: String, msg: Any, delay: FiniteDuration): Unit =
fsm.startTimerWithFixedDelay(name, msg, delay)
/**
* Proxy for [[akka.actor.FSM#startTimerAtFixedRate]].
*/
def startTimerAtFixedRate(name: String, msg: Any, interval: FiniteDuration): Unit =
fsm.startTimerAtFixedRate(name, msg, interval)
/**
* Proxy for [[akka.actor.FSM#startSingleTimer]].
*/
def startSingleTimer(name: String, msg: Any, delay: FiniteDuration): Unit =
fsm.startSingleTimer(name, msg, delay)
/**
* Proxy for [[akka.actor.FSM#setTimer]].
*/
@deprecated(
"Use startTimerWithFixedDelay or startTimerAtFixedRate instead. This has the same semantics as " +
"startTimerAtFixedRate, but startTimerWithFixedDelay is often preferred.",
since = "2.6.0")
def setTimer(name: String, msg: Any, timeout: FiniteDuration, repeat: Boolean = false): Unit = {
fsm.setTimer(name, msg, timeout, repeat)
}

View file

@ -49,7 +49,7 @@ class TestFSMRefSpec extends AkkaSpec {
}
}, "test-fsm-ref-2")
fsm.isTimerActive("test") should ===(false)
fsm.setTimer("test", 12, 10 millis, true)
fsm.startTimerWithFixedDelay("test", 12, 10 millis)
fsm.isTimerActive("test") should ===(true)
fsm.cancelTimer("test")
fsm.isTimerActive("test") should ===(false)