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:
parent
72cfc2485e
commit
10d32fceb9
99 changed files with 2285 additions and 909 deletions
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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, _) =>
|
||||
|
|
|
|||
|
|
@ -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
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
});
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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!"))
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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"))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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,
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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 =>
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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,
|
||||
|
|
|
|||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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(
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -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])
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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,
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
```
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 scheduler’s 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 }
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
||||
@@@
|
||||
|
||||
|
||||
|
|
|
|||
|
|
@ -151,7 +151,7 @@ public class FaultHandlingDocSample {
|
|||
getContext()
|
||||
.getSystem()
|
||||
.scheduler()
|
||||
.schedule(
|
||||
.scheduleWithFixedDelay(
|
||||
Duration.ZERO,
|
||||
Duration.ofSeconds(1L),
|
||||
getSelf(),
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
|
|
|
|||
|
|
@ -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,
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -37,7 +37,7 @@ public class DataBot extends AbstractActor {
|
|||
getContext()
|
||||
.getSystem()
|
||||
.scheduler()
|
||||
.schedule(
|
||||
.scheduleWithFixedDelay(
|
||||
Duration.ofSeconds(5),
|
||||
Duration.ofSeconds(5),
|
||||
getSelf(),
|
||||
|
|
|
|||
|
|
@ -79,7 +79,7 @@ public class RecipeGlobalRateLimit extends RecipeTest {
|
|||
this.replenishTimer =
|
||||
system
|
||||
.scheduler()
|
||||
.schedule(
|
||||
.scheduleWithFixedDelay(
|
||||
this.tokenRefreshPeriod,
|
||||
this.tokenRefreshPeriod,
|
||||
getSelf(),
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
|
||||
|
|
|
|||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -143,7 +143,7 @@ object PersistentActorCompileOnlyTest {
|
|||
})
|
||||
|
||||
Behaviors.withTimers((timers: TimerScheduler[Command]) => {
|
||||
timers.startPeriodicTimer("swing", MoodSwing, 10.seconds)
|
||||
timers.startTimerWithFixedDelay("swing", MoodSwing, 10.seconds)
|
||||
b
|
||||
})
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
|
|
|
|||
|
|
@ -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]]()
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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]]. */
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue