From 10d32fceb9d15b027aefa22b5af59423dce86cf0 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 27 May 2019 11:53:26 +0200 Subject: [PATCH] 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 --- .../scaladsl/ManualTimerExampleSpec.scala | 6 +- .../test/scala/akka/actor/FSMActorSpec.scala | 4 +- .../test/scala/akka/actor/FSMTimingSpec.scala | 14 +- .../scala/akka/actor/ReceiveTimeoutSpec.scala | 14 +- .../test/scala/akka/actor/SchedulerSpec.scala | 1041 +++++++++-------- .../akka/actor/SupervisorHierarchySpec.scala | 4 +- .../src/test/scala/akka/actor/TimerSpec.scala | 19 +- .../actor/typed/javadsl/ActorCompile.java | 2 +- .../akka/actor/typed/SchedulerSpec.scala | 3 +- .../scala/akka/actor/typed/TimerSpec.scala | 24 +- .../scala/akka/actor/typed/Scheduler.scala | 126 +- .../typed/internal/TimerSchedulerImpl.scala | 48 +- .../internal/adapter/SchedulerAdapter.scala | 18 +- .../actor/typed/javadsl/TimerScheduler.scala | 47 +- .../actor/typed/scaladsl/TimerScheduler.scala | 47 +- .../java/akka/actor/AbstractScheduler.java | 2 + .../mima-filters/2.5.x.backwards.excludes | 13 + .../main/scala/akka/actor/AbstractFSM.scala | 71 +- .../main/scala/akka/actor/ActorSystem.scala | 3 +- .../src/main/scala/akka/actor/FSM.scala | 111 +- .../actor/LightArrayRevolverScheduler.scala | 10 +- .../src/main/scala/akka/actor/Scheduler.scala | 423 +++++-- .../src/main/scala/akka/actor/Timers.scala | 110 +- .../actor/dungeon/TimerSchedulerImpl.scala | 38 +- .../main/scala/akka/io/SimpleDnsManager.scala | 2 +- .../io/dns/internal/AsyncDnsManager.scala | 2 +- .../scala/akka/routing/TailChopping.scala | 2 +- .../scala/akka/actor/ScheduleBenchmark.scala | 39 +- .../metrics/ClusterMetricsCollector.scala | 8 +- .../scala/akka/cluster/sharding/Shard.scala | 4 +- .../cluster/sharding/ShardCoordinator.scala | 3 +- .../akka/cluster/sharding/ShardRegion.scala | 2 +- .../akka/cluster/client/ClusterClient.scala | 13 +- .../pubsub/DistributedPubSubMediator.scala | 6 +- .../singleton/ClusterSingletonManager.scala | 16 +- .../singleton/ClusterSingletonProxy.scala | 8 +- .../receptionist/ClusterReceptionist.scala | 4 +- .../src/main/scala/akka/cluster/Cluster.scala | 1 + .../scala/akka/cluster/ClusterDaemon.scala | 11 +- .../scala/akka/cluster/ClusterHeartbeat.scala | 6 +- .../cluster/CrossDcClusterHeartbeat.scala | 2 +- .../scala/akka/cluster/StressSpec.scala | 4 +- .../scala/akka/cluster/ddata/Replicator.scala | 15 +- .../ReplicatorMessageSerializer.scala | 2 +- .../akka/cluster/ddata/LotsOfDataBot.scala | 6 +- akka-docs/src/main/paradox/actors.md | 4 + akka-docs/src/main/paradox/fsm.md | 11 +- .../project/migration-guide-2.5.x-2.6.x.md | 24 +- akka-docs/src/main/paradox/scheduler.md | 59 +- .../paradox/typed/interaction-patterns.md | 40 + .../jdocs/actor/FaultHandlingDocSample.java | 2 +- .../java/jdocs/actor/SchedulerDocTest.java | 16 +- .../test/java/jdocs/actor/TimerDocTest.java | 2 +- .../test/java/jdocs/actor/fsm/FSMDocTest.java | 5 +- .../java/jdocs/cluster/StatsSampleClient.java | 3 +- .../src/test/java/jdocs/ddata/DataBot.java | 2 +- .../cookbook/RecipeGlobalRateLimit.java | 2 +- .../test/scala/docs/actor/FSMDocSpec.scala | 2 +- .../docs/actor/FaultHandlingDocSample.scala | 2 +- .../scala/docs/actor/SchedulerDocSpec.scala | 2 +- .../test/scala/docs/actor/TimerDocSpec.scala | 2 +- .../docs/cluster/TransformationFrontend.scala | 2 +- .../docs/ddata/DistributedDataDocSpec.scala | 2 +- .../query/MyEventsByTagSource.scala | 2 +- .../cookbook/RecipeGlobalRateLimit.scala | 4 +- .../scala/docs/testkit/TestkitDocSpec.scala | 2 +- .../akka/remote/testconductor/Conductor.scala | 4 +- .../EventsByPersistenceIdPublisher.scala | 3 +- .../leveldb/EventsByTagPublisher.scala | 3 +- .../typed/internal/BehaviorSetup.scala | 8 +- .../PersistentActorCompileOnlyTest.scala | 2 +- .../mima-filters/2.5.x.backwards.excludes | 9 + .../persistence/AtLeastOnceDelivery.scala | 2 +- .../scala/akka/persistence/Eventsourced.scala | 2 +- .../akka/persistence/fsm/PersistentFSM.scala | 34 +- .../persistence/fsm/PersistentFSMBase.scala | 145 ++- .../src/main/scala/akka/remote/Endpoint.scala | 4 +- .../scala/akka/remote/RemoteWatcher.scala | 4 +- .../src/main/scala/akka/remote/Remoting.scala | 2 +- .../akka/remote/artery/ArteryTransport.scala | 2 +- .../akka/remote/artery/Association.scala | 69 +- .../scala/akka/remote/artery/Codecs.scala | 7 +- .../scala/akka/remote/artery/Handshake.scala | 4 +- .../aeron/ArteryAeronUdpTransport.scala | 4 +- .../transport/AkkaProtocolTransport.scala | 9 +- .../transport/ThrottlerTransportAdapter.scala | 2 +- .../transport/AkkaProtocolStressTest.scala | 2 +- .../SystemMessageDeliveryStressTest.scala | 4 +- .../scaladsl/GraphStageTimersSpec.scala | 4 +- .../mima-filters/2.5.x.backwards.excludes | 4 + .../main/scala/akka/stream/Materializer.scala | 75 ++ .../stream/impl/ActorMaterializerImpl.scala | 14 +- .../impl/PhasedFusingActorMaterializer.scala | 14 +- .../main/scala/akka/stream/impl/Timers.scala | 9 +- .../akka/stream/impl/fusing/GraphStages.scala | 2 +- .../scala/akka/stream/impl/fusing/Ops.scala | 6 +- .../scala/akka/stream/stage/GraphStage.scala | 173 ++- .../main/scala/akka/testkit/TestFSMRef.scala | 22 + .../scala/akka/testkit/TestFSMRefSpec.scala | 2 +- 99 files changed, 2285 insertions(+), 909 deletions(-) diff --git a/akka-actor-testkit-typed/src/test/scala/docs/akka/actor/testkit/typed/scaladsl/ManualTimerExampleSpec.scala b/akka-actor-testkit-typed/src/test/scala/docs/akka/actor/testkit/typed/scaladsl/ManualTimerExampleSpec.scala index c187c3170c..67f39c59f6 100644 --- a/akka-actor-testkit-typed/src/test/scala/docs/akka/actor/testkit/typed/scaladsl/ManualTimerExampleSpec.scala +++ b/akka-actor-testkit-typed/src/test/scala/docs/akka/actor/testkit/typed/scaladsl/ManualTimerExampleSpec.scala @@ -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 } diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala index 0786cee148..6e9dd8b0f7 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala @@ -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) { diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala index c2729ccb27..162b3a1cac 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala @@ -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, _) => diff --git a/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala index 0f3eabc021..bb3a786e6c 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala @@ -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 = { diff --git a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala index 6592551b87..64900df506 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala @@ -30,6 +30,7 @@ object SchedulerSpec { akka.scheduler.ticks-per-wheel = 32 akka.actor.serialize-messages = off """).withFallback(AkkaSpec.testConf) + } trait SchedulerSpec extends BeforeAndAfterEach with DefaultTimeout with ImplicitSender { this: AkkaSpec => @@ -37,277 +38,379 @@ trait SchedulerSpec extends BeforeAndAfterEach with DefaultTimeout with Implicit def collectCancellable(c: Cancellable): Cancellable - "A Scheduler" must { + abstract class ScheduleAdapter { + def schedule(initialDelay: FiniteDuration, delay: FiniteDuration, runnable: Runnable)( + implicit executor: ExecutionContext): Cancellable - "schedule more than once" taggedAs TimingTest in { - case object Tick - case object Tock + def schedule(initialDelay: FiniteDuration, delay: FiniteDuration, receiver: ActorRef, message: Any)( + implicit + executor: ExecutionContext): Cancellable + } - val tickActor, tickActor2 = system.actorOf(Props(new Actor { - var ticks = 0 - def receive = { - case Tick => - if (ticks < 3) { - sender() ! Tock - ticks += 1 + class ScheduleWithFixedDelayAdapter extends ScheduleAdapter { + def schedule(initialDelay: FiniteDuration, delay: FiniteDuration, runnable: Runnable)( + implicit executor: ExecutionContext): Cancellable = + system.scheduler.scheduleWithFixedDelay(initialDelay, delay)(runnable) + + def schedule(initialDelay: FiniteDuration, delay: FiniteDuration, receiver: ActorRef, message: Any)( + implicit + executor: ExecutionContext): Cancellable = + system.scheduler.scheduleWithFixedDelay(initialDelay, delay, receiver, message) + + override def toString: String = "scheduleWithFixedDelay" + } + + class ScheduleAtFixedRateAdapter extends ScheduleAdapter { + def schedule(initialDelay: FiniteDuration, delay: FiniteDuration, runnable: Runnable)( + implicit executor: ExecutionContext): Cancellable = + system.scheduler.scheduleAtFixedRate(initialDelay, delay)(runnable) + + def schedule(initialDelay: FiniteDuration, delay: FiniteDuration, receiver: ActorRef, message: Any)( + implicit + executor: ExecutionContext): Cancellable = + system.scheduler.scheduleAtFixedRate(initialDelay, delay, receiver, message) + + override def toString: String = "scheduleAtFixedRate" + } + + "A Scheduler" when { + + "using scheduleOnce" must { + "schedule once" taggedAs TimingTest in { + case object Tick + val countDownLatch = new CountDownLatch(3) + val tickActor = system.actorOf(Props(new Actor { + def receive = { case Tick => countDownLatch.countDown() } + })) + + // run after 300 millisec + collectCancellable(system.scheduler.scheduleOnce(300.millis, tickActor, Tick)) + collectCancellable(system.scheduler.scheduleOnce(300.millis)(countDownLatch.countDown())) + + // should not be run immediately + assert(countDownLatch.await(100, TimeUnit.MILLISECONDS) == false) + countDownLatch.getCount should ===(3L) + + // after 1 second the wait should fail + assert(countDownLatch.await(2, TimeUnit.SECONDS) == false) + // should still be 1 left + countDownLatch.getCount should ===(1L) + } + + /** + * ticket #372 + */ + "be cancellable" taggedAs TimingTest in { + for (_ <- 1 to 10) system.scheduler.scheduleOnce(1 second, testActor, "fail").cancel() + + expectNoMessage(2 seconds) + } + + "be canceled if cancel is performed before execution" taggedAs TimingTest in { + val task = collectCancellable(system.scheduler.scheduleOnce(10 seconds)(())) + task.cancel() should ===(true) + task.isCancelled should ===(true) + task.cancel() should ===(false) + task.isCancelled should ===(true) + } + + "not be canceled if cancel is performed after execution" taggedAs TimingTest in { + val latch = TestLatch(1) + val task = collectCancellable(system.scheduler.scheduleOnce(10.millis)(latch.countDown())) + Await.ready(latch, remainingOrDefault) + task.cancel() should ===(false) + task.isCancelled should ===(false) + task.cancel() should ===(false) + task.isCancelled should ===(false) + } + + "never fire prematurely" taggedAs TimingTest in { + val ticks = new TestLatch(300) + + final case class Msg(ts: Long) + + val actor = system.actorOf(Props(new Actor { + def receive = { + case Msg(ts) => + val now = System.nanoTime + // Make sure that no message has been dispatched before the scheduled time (10ms) has occurred + if (now < ts) throw new RuntimeException("Interval is too small: " + (now - ts)) + ticks.countDown() + } + })) + + (1 to 300).foreach { _ => + collectCancellable(system.scheduler.scheduleOnce(20.millis, actor, Msg(System.nanoTime))) + Thread.sleep(5) + } + + Await.ready(ticks, 3 seconds) + } + + "handle timeouts equal to multiple of wheel period" taggedAs TimingTest in { + val timeout = 3200.millis + val barrier = TestLatch() + import system.dispatcher + val job = system.scheduler.scheduleOnce(timeout)(barrier.countDown()) + try { + Await.ready(barrier, 5000.millis) + } finally { + job.cancel() + } + } + + "survive being stressed without cancellation" taggedAs TimingTest in { + val r = ThreadLocalRandom.current() + val N = 100000 + for (_ <- 1 to N) { + val next = r.nextInt(3000) + val now = System.nanoTime + system.scheduler.scheduleOnce(next.millis) { + val stop = System.nanoTime + testActor ! (stop - now - next * 1000000L) + } + } + val latencies = within(10.seconds) { + for (i <- 1 to N) + yield try expectMsgType[Long] + catch { + case NonFatal(e) => throw new Exception(s"failed expecting the $i-th latency", e) } } - })) - // run every 50 milliseconds - collectCancellable(system.scheduler.schedule(0 milliseconds, 50 milliseconds, tickActor, Tick)) - - // after max 1 second it should be executed at least the 3 times already - expectMsg(Tock) - expectMsg(Tock) - expectMsg(Tock) - expectNoMessage(500 millis) - - collectCancellable(system.scheduler.schedule(0 milliseconds, 50 milliseconds)(tickActor2 ! Tick)) - - // after max 1 second it should be executed at least the 3 times already - expectMsg(Tock) - expectMsg(Tock) - expectMsg(Tock) - expectNoMessage(500 millis) - } - - "stop continuous scheduling if the receiving actor has been terminated" taggedAs TimingTest in { - val actor = system.actorOf(Props(new Actor { def receive = { case x => sender() ! x } })) - - // run immediately and then every 100 milliseconds - collectCancellable(system.scheduler.schedule(0 milliseconds, 100 milliseconds, actor, "msg")) - expectMsg("msg") - - // stop the actor and, hence, the continuous messaging from happening - system.stop(actor) - - expectNoMessage(500 millis) - } - - "stop continuous scheduling if the task throws exception" taggedAs TimingTest in { - val count = new AtomicInteger(0) - collectCancellable(system.scheduler.schedule(0 milliseconds, 20.millis) { - val c = count.incrementAndGet() - testActor ! c - if (c == 3) throw new RuntimeException("TEST") with NoStackTrace - }) - expectMsg(1) - expectMsg(2) - expectMsg(3) - expectNoMessage(500 millis) - } - - "schedule once" taggedAs TimingTest in { - case object Tick - val countDownLatch = new CountDownLatch(3) - val tickActor = system.actorOf(Props(new Actor { - def receive = { case Tick => countDownLatch.countDown() } - })) - - // run after 300 millisec - collectCancellable(system.scheduler.scheduleOnce(300 milliseconds, tickActor, Tick)) - collectCancellable(system.scheduler.scheduleOnce(300 milliseconds)(countDownLatch.countDown())) - - // should not be run immediately - assert(countDownLatch.await(100, TimeUnit.MILLISECONDS) == false) - countDownLatch.getCount should ===(3L) - - // after 1 second the wait should fail - assert(countDownLatch.await(2, TimeUnit.SECONDS) == false) - // should still be 1 left - countDownLatch.getCount should ===(1L) - } - - /** - * ticket #372 - */ - "be cancellable" taggedAs TimingTest in { - for (_ <- 1 to 10) system.scheduler.scheduleOnce(1 second, testActor, "fail").cancel() - - expectNoMessage(2 seconds) - } - - "be cancellable during initial delay" taggedAs TimingTest in { - val ticks = new AtomicInteger - - val initialDelay = 200.milliseconds.dilated - val delay = 10.milliseconds.dilated - val timeout = collectCancellable(system.scheduler.schedule(initialDelay, delay) { - ticks.incrementAndGet() - }) - Thread.sleep(10.milliseconds.dilated.toMillis) - timeout.cancel() - Thread.sleep((initialDelay + 100.milliseconds.dilated).toMillis) - - ticks.get should ===(0) - } - - "be cancellable after initial delay" taggedAs TimingTest in { - val ticks = new AtomicInteger - - val initialDelay = 90.milliseconds.dilated - val delay = 500.milliseconds.dilated - val timeout = collectCancellable(system.scheduler.schedule(initialDelay, delay) { - ticks.incrementAndGet() - }) - Thread.sleep((initialDelay + 200.milliseconds.dilated).toMillis) - timeout.cancel() - Thread.sleep((delay + 100.milliseconds.dilated).toMillis) - - ticks.get should ===(1) - } - - "be canceled if cancel is performed before execution" taggedAs TimingTest in { - val task = collectCancellable(system.scheduler.scheduleOnce(10 seconds)(())) - task.cancel() should ===(true) - task.isCancelled should ===(true) - task.cancel() should ===(false) - task.isCancelled should ===(true) - } - - "not be canceled if cancel is performed after execution" taggedAs TimingTest in { - val latch = TestLatch(1) - val task = collectCancellable(system.scheduler.scheduleOnce(10 millis)(latch.countDown())) - Await.ready(latch, remainingOrDefault) - task.cancel() should ===(false) - task.isCancelled should ===(false) - task.cancel() should ===(false) - task.isCancelled should ===(false) - } - - /** - * ticket #307 - */ - "pick up schedule after actor restart" taggedAs TimingTest in { - - object Ping - object Crash - - val restartLatch = new TestLatch - val pingLatch = new TestLatch(6) - - val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(3, 1 second)(List(classOf[Exception]))))) - val props = Props(new Actor { - def receive = { - case Ping => pingLatch.countDown() - case Crash => throw new Exception("CRASH") - } - - override def postRestart(reason: Throwable) = restartLatch.open - }) - val actor = Await.result((supervisor ? props).mapTo[ActorRef], timeout.duration) - - collectCancellable(system.scheduler.schedule(500 milliseconds, 500 milliseconds, actor, Ping)) - // appx 2 pings before crash - EventFilter[Exception]("CRASH", occurrences = 1).intercept { - collectCancellable(system.scheduler.scheduleOnce(1000 milliseconds, actor, Crash)) - } - - Await.ready(restartLatch, 2 seconds) - // should be enough time for the ping countdown to recover and reach 6 pings - Await.ready(pingLatch, 5 seconds) - } - - "never fire prematurely" taggedAs TimingTest in { - val ticks = new TestLatch(300) - - final case class Msg(ts: Long) - - val actor = system.actorOf(Props(new Actor { - def receive = { - case Msg(ts) => - val now = System.nanoTime - // Make sure that no message has been dispatched before the scheduled time (10ms) has occurred - if (now < ts) throw new RuntimeException("Interval is too small: " + (now - ts)) - ticks.countDown() - } - })) - - (1 to 300).foreach { _ => - collectCancellable(system.scheduler.scheduleOnce(20 milliseconds, actor, Msg(System.nanoTime))) - Thread.sleep(5) - } - - Await.ready(ticks, 3 seconds) - } - - "schedule with different initial delay and frequency" taggedAs TimingTest in { - val ticks = new TestLatch(3) - - case object Msg - - val actor = system.actorOf(Props(new Actor { - def receive = { case Msg => ticks.countDown() } - })) - - val startTime = System.nanoTime() - collectCancellable(system.scheduler.schedule(1 second, 300 milliseconds, actor, Msg)) - Await.ready(ticks, 3 seconds) - - // LARS is a bit more aggressive in scheduling recurring tasks at the right - // frequency and may execute them a little earlier; the actual expected timing - // is 1599ms on a fast machine or 1699ms on a loaded one (plus some room for jenkins) - (System.nanoTime() - startTime).nanos.toMillis should ===(1750L +- 250) - } - - "adjust for scheduler inaccuracy" taggedAs TimingTest in { - val startTime = System.nanoTime - val n = 200 - val latch = new TestLatch(n) - system.scheduler.schedule(25.millis, 25.millis) { latch.countDown() } - Await.ready(latch, 6.seconds) - // Rate - n * 1000.0 / (System.nanoTime - startTime).nanos.toMillis should ===(40.0 +- 4) - } - - "not be affected by long running task" taggedAs TimingTest in { - val n = 22 - val latch = new TestLatch(n) - val startTime = System.nanoTime - system.scheduler.schedule(225.millis, 225.millis) { - Thread.sleep(100) - latch.countDown() - } - Await.ready(latch, 6.seconds) - // Rate - n * 1000.0 / (System.nanoTime - startTime).nanos.toMillis should ===(4.4 +- 0.5) - } - - "handle timeouts equal to multiple of wheel period" taggedAs TimingTest in { - val timeout = 3200 milliseconds - val barrier = TestLatch() - import system.dispatcher - val job = system.scheduler.scheduleOnce(timeout)(barrier.countDown()) - try { - Await.ready(barrier, 5000 milliseconds) - } finally { - job.cancel() - } - } - - "survive being stressed without cancellation" taggedAs TimingTest in { - val r = ThreadLocalRandom.current() - val N = 100000 - for (_ <- 1 to N) { - val next = r.nextInt(3000) - val now = System.nanoTime - system.scheduler.scheduleOnce(next.millis) { - val stop = System.nanoTime - testActor ! (stop - now - next * 1000000L) + val histogram = latencies.groupBy(_ / 100000000L) + for (k <- histogram.keys.toSeq.sorted) { + system.log.info(f"${k * 100}%3d: ${histogram(k).size}") } } - val latencies = within(10.seconds) { - for (i <- 1 to N) - yield try expectMsgType[Long] - catch { - case NonFatal(e) => throw new Exception(s"failed expecting the $i-th latency", e) + } + + // same tests for fixedDelay and fixedRate + List(new ScheduleWithFixedDelayAdapter, new ScheduleAtFixedRateAdapter).foreach { scheduleAdapter => + s"using $scheduleAdapter" must { + + "schedule more than once" taggedAs TimingTest in { + case object Tick + case object Tock + + val tickActor, tickActor2 = system.actorOf(Props(new Actor { + var ticks = 0 + + def receive = { + case Tick => + if (ticks < 3) { + sender() ! Tock + ticks += 1 + } + } + })) + // run every 50 milliseconds + scheduleAdapter.schedule(Duration.Zero, 50.millis, tickActor, Tick) + + // after max 1 second it should be executed at least the 3 times already + expectMsg(Tock) + expectMsg(Tock) + expectMsg(Tock) + expectNoMessage(500.millis) + + collectCancellable(scheduleAdapter.schedule(Duration.Zero, 50.millis, () => tickActor2 ! Tick)) + + // after max 1 second it should be executed at least the 3 times already + expectMsg(Tock) + expectMsg(Tock) + expectMsg(Tock) + expectNoMessage(500.millis) + } + + "stop continuous scheduling if the receiving actor has been terminated" taggedAs TimingTest in { + val actor = system.actorOf(Props(new Actor { + def receive = { + case x => sender() ! x + } + })) + + // run immediately and then every 100 milliseconds + collectCancellable(scheduleAdapter.schedule(Duration.Zero, 100.millis, actor, "msg")) + expectMsg("msg") + + // stop the actor and, hence, the continuous messaging from happening + system.stop(actor) + + expectNoMessage(500.millis) + } + + "stop continuous scheduling if the task throws exception" taggedAs TimingTest in { + EventFilter[Exception]("TEST", occurrences = 1).intercept { + val count = new AtomicInteger(0) + collectCancellable(scheduleAdapter.schedule(Duration.Zero, 20.millis, () => { + val c = count.incrementAndGet() + testActor ! c + if (c == 3) throw new RuntimeException("TEST") with NoStackTrace + })) + expectMsg(1) + expectMsg(2) + expectMsg(3) + expectNoMessage(200.millis) } - } - val histogram = latencies.groupBy(_ / 100000000L) - for (k <- histogram.keys.toSeq.sorted) { - system.log.info(f"${k * 100}%3d: ${histogram(k).size}") + } + + "stop continuous scheduling if the task throws IllegalStateException" taggedAs TimingTest in { + // when first throws + EventFilter[Exception]("TEST-1", occurrences = 1).intercept { + val count1 = new AtomicInteger(0) + collectCancellable(scheduleAdapter.schedule(Duration.Zero, 20.millis, () => { + val c = count1.incrementAndGet() + if (c == 1) + throw new IllegalStateException("TEST-1") with NoStackTrace + else + testActor ! c + })) + expectNoMessage(200.millis) + } + + // when later + EventFilter[Exception]("TEST-3", occurrences = 1).intercept { + val count2 = new AtomicInteger(0) + collectCancellable(scheduleAdapter.schedule(Duration.Zero, 20.millis, () => { + val c = count2.incrementAndGet() + testActor ! c + if (c == 3) throw new IllegalStateException("TEST-3") with NoStackTrace + })) + expectMsg(1) + expectMsg(2) + expectMsg(3) + expectNoMessage(200.millis) + } + } + + "be cancellable during initial delay" taggedAs TimingTest in { + val ticks = new AtomicInteger + + val initialDelay = 200.millis.dilated + val delay = 10.millis.dilated + val timeout = collectCancellable(scheduleAdapter.schedule(initialDelay, delay, () => { + ticks.incrementAndGet() + })) + Thread.sleep(10.millis.dilated.toMillis) + timeout.cancel() + Thread.sleep((initialDelay + 100.millis.dilated).toMillis) + + ticks.get should ===(0) + } + + "be cancellable after initial delay" taggedAs TimingTest in { + val ticks = new AtomicInteger + + val initialDelay = 90.millis.dilated + val delay = 500.millis.dilated + val timeout = collectCancellable(scheduleAdapter.schedule(initialDelay, delay, () => { + ticks.incrementAndGet() + })) + Thread.sleep((initialDelay + 200.millis.dilated).toMillis) + timeout.cancel() + Thread.sleep((delay + 100.millis.dilated).toMillis) + + ticks.get should ===(1) + } + + /** + * ticket #307 + */ + "pick up schedule after actor restart" taggedAs TimingTest in { + + object Ping + object Crash + + val restartLatch = new TestLatch + val pingLatch = new TestLatch(6) + + val supervisor = + system.actorOf(Props(new Supervisor(AllForOneStrategy(3, 1 second)(List(classOf[Exception]))))) + val props = Props(new Actor { + def receive = { + case Ping => pingLatch.countDown() + case Crash => throw new Exception("CRASH") + } + + override def postRestart(reason: Throwable) = restartLatch.open + }) + val actor = Await.result((supervisor ? props).mapTo[ActorRef], timeout.duration) + + collectCancellable(scheduleAdapter.schedule(500.millis, 500.millis, actor, Ping)) + // appx 2 pings before crash + EventFilter[Exception]("CRASH", occurrences = 1).intercept { + collectCancellable(system.scheduler.scheduleOnce(1000.millis, actor, Crash)) + } + + Await.ready(restartLatch, 2 seconds) + // should be enough time for the ping countdown to recover and reach 6 pings + Await.ready(pingLatch, 5 seconds) + } + + "schedule with different initial delay and frequency" taggedAs TimingTest in { + val ticks = new TestLatch(3) + + case object Msg + + val actor = system.actorOf(Props(new Actor { + def receive = { case Msg => ticks.countDown() } + })) + + val startTime = System.nanoTime() + collectCancellable(scheduleAdapter.schedule(1 second, 300.millis, actor, Msg)) + Await.ready(ticks, 3 seconds) + + // LARS is a bit more aggressive in scheduling recurring tasks at the right + // frequency and may execute them a little earlier; the actual expected timing + // is 1599ms on a fast machine or 1699ms on a loaded one (plus some room for jenkins) + (System.nanoTime() - startTime).nanos.toMillis should ===(1750L +- 250) + } } } + + "using scheduleAtFixedRate" must { + + "adjust for scheduler inaccuracy" taggedAs TimingTest in { + val startTime = System.nanoTime + val n = 200 + val latch = new TestLatch(n) + system.scheduler.scheduleAtFixedRate(25.millis, 25.millis) { () => + latch.countDown() + } + Await.ready(latch, 6.seconds) + // Rate + n * 1000.0 / (System.nanoTime - startTime).nanos.toMillis should ===(40.0 +- 4) + } + + "not be affected by long running task" taggedAs TimingTest in { + val n = 22 + val latch = new TestLatch(n) + val startTime = System.nanoTime + system.scheduler.scheduleAtFixedRate(225.millis, 225.millis) { () => + Thread.sleep(100) + latch.countDown() + } + Await.ready(latch, 6.seconds) + // Rate + n * 1000.0 / (System.nanoTime - startTime).nanos.toMillis should ===(4.4 +- 0.5) + } + } + + "using scheduleWithFixedDelay" must { + + "keep delay after task completed" taggedAs TimingTest in { + val n = 12 + val latch = new TestLatch(n) + val startTime = System.nanoTime + system.scheduler.scheduleWithFixedDelay(125.millis, 125.millis) { () => + Thread.sleep(100) + latch.countDown() + } + Await.ready(latch, 6.seconds) + // Rate + n * 1000.0 / (System.nanoTime - startTime).nanos.toMillis should ===(4.4 +- 0.5) + } + } + } } @@ -317,224 +420,234 @@ class LightArrayRevolverSchedulerSpec extends AkkaSpec(SchedulerSpec.testConfRev def tickDuration = system.scheduler.asInstanceOf[LightArrayRevolverScheduler].TickDuration - "A LightArrayRevolverScheduler" must { + "A LightArrayRevolverScheduler" when { - "reject tasks scheduled too far into the future" taggedAs TimingTest in { - val maxDelay = tickDuration * Int.MaxValue - import system.dispatcher - system.scheduler.scheduleOnce(maxDelay, testActor, "OK") - intercept[IllegalArgumentException] { - system.scheduler.scheduleOnce(maxDelay + tickDuration, testActor, "Too far") - } - } + "using scheduleOnce" must { - "reject periodic tasks scheduled too far into the future" taggedAs TimingTest in { - val maxDelay = tickDuration * Int.MaxValue - import system.dispatcher - system.scheduler.schedule(maxDelay, 1.second, testActor, "OK") - intercept[IllegalArgumentException] { - system.scheduler.schedule(maxDelay + tickDuration, 1.second, testActor, "Too far") - } - } - - "reject periodic tasks scheduled with too long interval" taggedAs TimingTest in { - val maxDelay = tickDuration * Int.MaxValue - import system.dispatcher - system.scheduler.schedule(100.millis, maxDelay, testActor, "OK") - expectMsg("OK") - intercept[IllegalArgumentException] { - system.scheduler.schedule(100.millis, maxDelay + tickDuration, testActor, "Too long") - } - expectNoMessage(1.second) - } - - "survive being stressed with cancellation" taggedAs TimingTest in { - import system.dispatcher - val r = ThreadLocalRandom.current - val N = 1000000 - val tasks = for (_ <- 1 to N) yield { - val next = r.nextInt(3000) - val now = System.nanoTime - system.scheduler.scheduleOnce(next.millis) { - val stop = System.nanoTime - testActor ! (stop - now - next * 1000000L) - } - } - // get somewhat into the middle of things - Thread.sleep(500) - val cancellations = for (t <- tasks) yield { - t.cancel() - if (t.isCancelled) 1 else 0 - } - val cancelled = cancellations.sum - println(cancelled) - val latencies = within(10.seconds) { - for (i <- 1 to (N - cancelled)) - yield try expectMsgType[Long] - catch { - case NonFatal(e) => throw new Exception(s"failed expecting the $i-th latency", e) - } - } - val histogram = latencies.groupBy(_ / 100000000L) - for (k <- histogram.keys.toSeq.sorted) { - system.log.info(f"${k * 100}%3d: ${histogram(k).size}") - } - expectNoMessage(1.second) - } - - "survive vicious enqueueing" taggedAs TimingTest in { - withScheduler(config = ConfigFactory.parseString("akka.scheduler.ticks-per-wheel=2")) { (sched, driver) => - import driver._ + "reject tasks scheduled too far into the future" taggedAs TimingTest in { + val maxDelay = tickDuration * Int.MaxValue import system.dispatcher - val counter = new AtomicInteger - val terminated = Future { - var rounds = 0 - while (Try(sched.scheduleOnce(Duration.Zero)(())(localEC)).isSuccess) { - Thread.sleep(1) - driver.wakeUp(step) - rounds += 1 - } - rounds + system.scheduler.scheduleOnce(maxDelay, testActor, "OK") + intercept[IllegalArgumentException] { + system.scheduler.scheduleOnce(maxDelay + tickDuration, testActor, "Too far") } - def delay = if (ThreadLocalRandom.current.nextBoolean) step * 2 else step + } + + "survive being stressed with cancellation" taggedAs TimingTest in { + import system.dispatcher + val r = ThreadLocalRandom.current val N = 1000000 - (1 to N).foreach(_ => sched.scheduleOnce(delay)(counter.incrementAndGet())) - sched.close() - Await.result(terminated, 3.seconds.dilated) should be > 10 - awaitCond(counter.get == N) + val tasks = for (_ <- 1 to N) yield { + val next = r.nextInt(3000) + val now = System.nanoTime + system.scheduler.scheduleOnce(next.millis) { + val stop = System.nanoTime + testActor ! (stop - now - next * 1000000L) + } + } + // get somewhat into the middle of things + Thread.sleep(500) + val cancellations = for (t <- tasks) yield { + t.cancel() + if (t.isCancelled) 1 else 0 + } + val cancelled = cancellations.sum + println(cancelled) + val latencies = within(10.seconds) { + for (i <- 1 to (N - cancelled)) + yield try expectMsgType[Long] + catch { + case NonFatal(e) => throw new Exception(s"failed expecting the $i-th latency", e) + } + } + val histogram = latencies.groupBy(_ / 100000000L) + for (k <- histogram.keys.toSeq.sorted) { + system.log.info(f"${k * 100}%3d: ${histogram(k).size}") + } + expectNoMessage(1.second) } - } - "execute multiple jobs at once when expiring multiple buckets" taggedAs TimingTest in { - withScheduler() { (sched, driver) => - implicit def ec = localEC - import driver._ - val start = step / 2 - (0 to 3).foreach(i => sched.scheduleOnce(start + step * i, testActor, "hello")) - expectNoMessage(step) - wakeUp(step) - expectWait(step) - wakeUp(step * 4 + step / 2) - expectWait(step / 2) - (0 to 3).foreach(_ => expectMsg(Duration.Zero, "hello")) + "survive vicious enqueueing" taggedAs TimingTest in { + withScheduler(config = ConfigFactory.parseString("akka.scheduler.ticks-per-wheel=2")) { (sched, driver) => + import driver._ + import system.dispatcher + val counter = new AtomicInteger + val terminated = Future { + var rounds = 0 + while (Try(sched.scheduleOnce(Duration.Zero)(())(localEC)).isSuccess) { + Thread.sleep(1) + driver.wakeUp(step) + rounds += 1 + } + rounds + } + def delay = if (ThreadLocalRandom.current.nextBoolean) step * 2 else step + val N = 1000000 + (1 to N).foreach(_ => sched.scheduleOnce(delay)(counter.incrementAndGet())) + sched.close() + Await.result(terminated, 3.seconds.dilated) should be > 10 + awaitCond(counter.get == N) + } } - } - "properly defer jobs even when the timer thread oversleeps" taggedAs TimingTest in { - withScheduler() { (sched, driver) => - implicit def ec = localEC - import driver._ - sched.scheduleOnce(step * 3, probe.ref, "hello") - wakeUp(step * 5) - expectWait(step) - wakeUp(step * 2) - expectWait(step) - wakeUp(step) - probe.expectMsg("hello") - expectWait(step) - } - } - - "correctly wrap around wheel rounds" taggedAs TimingTest in { - withScheduler(config = ConfigFactory.parseString("akka.scheduler.ticks-per-wheel=2")) { (sched, driver) => - implicit def ec = localEC - import driver._ - val start = step / 2 - (0 to 3).foreach(i => sched.scheduleOnce(start + step * i, probe.ref, "hello")) - probe.expectNoMessage(step) - wakeUp(step) - expectWait(step) - // the following are no for-comp to see which iteration fails - wakeUp(step) - probe.expectMsg("hello") - expectWait(step) - wakeUp(step) - probe.expectMsg("hello") - expectWait(step) - wakeUp(step) - probe.expectMsg("hello") - expectWait(step) - wakeUp(step) - probe.expectMsg("hello") - expectWait(step) - wakeUp(step) - expectWait(step) - } - } - - "correctly execute jobs when clock wraps around" taggedAs TimingTest in { - withScheduler(Long.MaxValue - 200000000L) { (sched, driver) => - implicit def ec = localEC - import driver._ - val start = step / 2 - (0 to 3).foreach(i => sched.scheduleOnce(start + step * i, testActor, "hello")) - expectNoMessage(step) - wakeUp(step) - expectWait(step) - // the following are no for-comp to see which iteration fails - wakeUp(step) - expectMsg("hello") - expectWait(step) - wakeUp(step) - expectMsg("hello") - expectWait(step) - wakeUp(step) - expectMsg("hello") - expectWait(step) - wakeUp(step) - expectMsg("hello") - expectWait(step) - wakeUp(step) - expectWait(step) - } - } - - "correctly wrap around ticks" taggedAs TimingTest in { - val numEvents = 40 - val targetTicks = Int.MaxValue - numEvents + 20 - - withScheduler(_startTick = Int.MaxValue - 100) { (sched, driver) => - implicit def ec = localEC - import driver._ - - val start = step / 2 - - wakeUp(step * targetTicks) - probe.expectMsgType[Long] - - val nums = 0 until numEvents - nums.foreach(i => sched.scheduleOnce(start + step * i, testActor, "hello-" + i)) - expectNoMessage(step) - wakeUp(step) - expectWait(step) - - nums.foreach { i => + "execute multiple jobs at once when expiring multiple buckets" taggedAs TimingTest in { + withScheduler() { (sched, driver) => + implicit def ec = localEC + import driver._ + val start = step / 2 + (0 to 3).foreach(i => sched.scheduleOnce(start + step * i, testActor, "hello")) + expectNoMessage(step) wakeUp(step) - expectMsg("hello-" + i) + expectWait(step) + wakeUp(step * 4 + step / 2) + expectWait(step / 2) + (0 to 3).foreach(_ => expectMsg(Duration.Zero, "hello")) + } + } + + "properly defer jobs even when the timer thread oversleeps" taggedAs TimingTest in { + withScheduler() { (sched, driver) => + implicit def ec = localEC + import driver._ + sched.scheduleOnce(step * 3, probe.ref, "hello") + wakeUp(step * 5) + expectWait(step) + wakeUp(step * 2) + expectWait(step) + wakeUp(step) + probe.expectMsg("hello") expectWait(step) } } - } - "reliably reject jobs when shutting down" taggedAs TimingTest in { - withScheduler() { (sched, driver) => - import system.dispatcher - val counter = new AtomicInteger - Future { Thread.sleep(5); driver.close(); sched.close() } - val headroom = 200 - var overrun = headroom - val cap = 1000000 - val (success, failure) = Iterator - .continually(Try(sched.scheduleOnce(100.millis)(counter.incrementAndGet()))) - .take(cap) - .takeWhile(_.isSuccess || { overrun -= 1; overrun >= 0 }) - .partition(_.isSuccess) - val s = success.size - s should be < cap - awaitCond(s == counter.get, message = s"$s was not ${counter.get}") - failure.size should ===(headroom) + "correctly wrap around wheel rounds" taggedAs TimingTest in { + withScheduler(config = ConfigFactory.parseString("akka.scheduler.ticks-per-wheel=2")) { (sched, driver) => + implicit def ec = localEC + import driver._ + val start = step / 2 + (0 to 3).foreach(i => sched.scheduleOnce(start + step * i, probe.ref, "hello")) + probe.expectNoMessage(step) + wakeUp(step) + expectWait(step) + // the following are no for-comp to see which iteration fails + wakeUp(step) + probe.expectMsg("hello") + expectWait(step) + wakeUp(step) + probe.expectMsg("hello") + expectWait(step) + wakeUp(step) + probe.expectMsg("hello") + expectWait(step) + wakeUp(step) + probe.expectMsg("hello") + expectWait(step) + wakeUp(step) + expectWait(step) + } + } + + "correctly execute jobs when clock wraps around" taggedAs TimingTest in { + withScheduler(Long.MaxValue - 200000000L) { (sched, driver) => + implicit def ec = localEC + import driver._ + val start = step / 2 + (0 to 3).foreach(i => sched.scheduleOnce(start + step * i, testActor, "hello")) + expectNoMessage(step) + wakeUp(step) + expectWait(step) + // the following are no for-comp to see which iteration fails + wakeUp(step) + expectMsg("hello") + expectWait(step) + wakeUp(step) + expectMsg("hello") + expectWait(step) + wakeUp(step) + expectMsg("hello") + expectWait(step) + wakeUp(step) + expectMsg("hello") + expectWait(step) + wakeUp(step) + expectWait(step) + } + } + + "correctly wrap around ticks" taggedAs TimingTest in { + val numEvents = 40 + val targetTicks = Int.MaxValue - numEvents + 20 + + withScheduler(_startTick = Int.MaxValue - 100) { (sched, driver) => + implicit def ec = localEC + import driver._ + + val start = step / 2 + + wakeUp(step * targetTicks) + probe.expectMsgType[Long] + + val nums = 0 until numEvents + nums.foreach(i => sched.scheduleOnce(start + step * i, testActor, "hello-" + i)) + expectNoMessage(step) + wakeUp(step) + expectWait(step) + + nums.foreach { i => + wakeUp(step) + expectMsg("hello-" + i) + expectWait(step) + } + } + } + + "reliably reject jobs when shutting down" taggedAs TimingTest in { + withScheduler() { (sched, driver) => + import system.dispatcher + val counter = new AtomicInteger + Future { Thread.sleep(5); driver.close(); sched.close() } + val headroom = 200 + var overrun = headroom + val cap = 1000000 + val (success, failure) = Iterator + .continually(Try(sched.scheduleOnce(100.millis)(counter.incrementAndGet()))) + .take(cap) + .takeWhile(_.isSuccess || { overrun -= 1; overrun >= 0 }) + .partition(_.isSuccess) + val s = success.size + s should be < cap + awaitCond(s == counter.get, message = s"$s was not ${counter.get}") + failure.size should ===(headroom) + } } } + + // same tests for fixedDelay and fixedRate + List(new ScheduleWithFixedDelayAdapter, new ScheduleAtFixedRateAdapter).foreach { scheduleAdapter => + s"using $scheduleAdapter" must { + + "reject periodic tasks scheduled too far into the future" taggedAs TimingTest in { + val maxDelay = tickDuration * Int.MaxValue + import system.dispatcher + scheduleAdapter.schedule(maxDelay, 1.second, testActor, "OK") + intercept[IllegalArgumentException] { + scheduleAdapter.schedule(maxDelay + tickDuration, 1.second, testActor, "Too far") + } + } + + "reject periodic tasks scheduled with too long interval" taggedAs TimingTest in { + val maxDelay = tickDuration * Int.MaxValue + import system.dispatcher + scheduleAdapter.schedule(100.millis, maxDelay, testActor, "OK") + expectMsg("OK") + intercept[IllegalArgumentException] { + scheduleAdapter.schedule(100.millis, maxDelay + tickDuration, testActor, "Too long") + } + expectNoMessage(1.second) + } + } + } + } trait Driver { diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index e4c027de15..0b64969541 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -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 diff --git a/akka-actor-tests/src/test/scala/akka/actor/TimerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/TimerSpec.scala index d3c175cd44..35314a968f 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TimerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TimerSpec.scala @@ -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) { diff --git a/akka-actor-typed-tests/src/test/java/akka/actor/typed/javadsl/ActorCompile.java b/akka-actor-typed-tests/src/test/java/akka/actor/typed/javadsl/ActorCompile.java index 5464b4856b..1653579f11 100644 --- a/akka-actor-typed-tests/src/test/java/akka/actor/typed/javadsl/ActorCompile.java +++ b/akka-actor-typed-tests/src/test/java/akka/actor/typed/javadsl/ActorCompile.java @@ -88,7 +88,7 @@ public class ActorCompile { Behavior 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(); }); } diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/SchedulerSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/SchedulerSpec.scala index 7aaa1bc1ed..8b33dc60da 100644 --- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/SchedulerSpec.scala +++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/SchedulerSpec.scala @@ -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!")) } diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/TimerSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/TimerSpec.scala index b2d9688a65..c9f1b5e0c9 100644 --- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/TimerSpec.scala +++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/TimerSpec.scala @@ -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")) } diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/Scheduler.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/Scheduler.scala index 29124fd696..6fe46e7f0a 100644 --- a/akka-actor-typed/src/main/scala/akka/actor/typed/Scheduler.scala +++ b/akka-actor-typed/src/main/scala/akka/actor/typed/Scheduler.scala @@ -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, diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/TimerSchedulerImpl.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/TimerSchedulerImpl.scala index bc439ee6b7..6a6bcef980 100644 --- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/TimerSchedulerImpl.scala +++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/TimerSchedulerImpl.scala @@ -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) } diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/SchedulerAdapter.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/SchedulerAdapter.scala index 2c0ec44d06..3ced2e33ab 100644 --- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/SchedulerAdapter.scala +++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/SchedulerAdapter.scala @@ -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) + } diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/TimerScheduler.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/TimerScheduler.scala index fc053ff593..4715888510 100644 --- a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/TimerScheduler.scala +++ b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/TimerScheduler.scala @@ -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 /** diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/TimerScheduler.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/TimerScheduler.scala index c3d42acc3f..9ff8c2ba51 100644 --- a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/TimerScheduler.scala +++ b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/TimerScheduler.scala @@ -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 /** diff --git a/akka-actor/src/main/java/akka/actor/AbstractScheduler.java b/akka-actor/src/main/java/akka/actor/AbstractScheduler.java index 0816098184..7e2cfe2724 100644 --- a/akka-actor/src/main/java/akka/actor/AbstractScheduler.java +++ b/akka-actor/src/main/java/akka/actor/AbstractScheduler.java @@ -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 diff --git a/akka-actor/src/main/mima-filters/2.5.x.backwards.excludes b/akka-actor/src/main/mima-filters/2.5.x.backwards.excludes index 4b7f83a48f..91109ef715 100644 --- a/akka-actor/src/main/mima-filters/2.5.x.backwards.excludes +++ b/akka-actor/src/main/mima-filters/2.5.x.backwards.excludes @@ -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") diff --git a/akka-actor/src/main/scala/akka/actor/AbstractFSM.scala b/akka-actor/src/main/scala/akka/actor/AbstractFSM.scala index 7e8eaff730..091452f32a 100644 --- a/akka-actor/src/main/scala/akka/actor/AbstractFSM.scala +++ b/akka-actor/src/main/scala/akka/actor/AbstractFSM.scala @@ -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) } diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index f1dad0cb39..3db297f0b4 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -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 diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index 8c5afa9762..9ab69c9d4e 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -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) diff --git a/akka-actor/src/main/scala/akka/actor/LightArrayRevolverScheduler.scala b/akka-actor/src/main/scala/akka/actor/LightArrayRevolverScheduler.scala index 3d14962a12..25c522f5a1 100644 --- a/akka-actor/src/main/scala/akka/actor/LightArrayRevolverScheduler.scala +++ b/akka-actor/src/main/scala/akka/actor/LightArrayRevolverScheduler.scala @@ -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 => diff --git a/akka-actor/src/main/scala/akka/actor/Scheduler.scala b/akka-actor/src/main/scala/akka/actor/Scheduler.scala index 3322676eaa..1c94429790 100644 --- a/akka-actor/src/main/scala/akka/actor/Scheduler.scala +++ b/akka-actor/src/main/scala/akka/actor/Scheduler.scala @@ -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 + } } diff --git a/akka-actor/src/main/scala/akka/actor/Timers.scala b/akka-actor/src/main/scala/akka/actor/Timers.scala index 669f465fd3..3b342936f8 100644 --- a/akka-actor/src/main/scala/akka/actor/Timers.scala +++ b/akka-actor/src/main/scala/akka/actor/Timers.scala @@ -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) /** diff --git a/akka-actor/src/main/scala/akka/actor/dungeon/TimerSchedulerImpl.scala b/akka-actor/src/main/scala/akka/actor/dungeon/TimerSchedulerImpl.scala index 9bec32d715..6143d8f3b2 100644 --- a/akka-actor/src/main/scala/akka/actor/dungeon/TimerSchedulerImpl.scala +++ b/akka-actor/src/main/scala/akka/actor/dungeon/TimerSchedulerImpl.scala @@ -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) } diff --git a/akka-actor/src/main/scala/akka/io/SimpleDnsManager.scala b/akka-actor/src/main/scala/akka/io/SimpleDnsManager.scala index f971cb5139..2f28cf465a 100644 --- a/akka-actor/src/main/scala/akka/io/SimpleDnsManager.scala +++ b/akka-actor/src/main/scala/akka/io/SimpleDnsManager.scala @@ -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 = { diff --git a/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsManager.scala b/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsManager.scala index f0806503dc..53b1666c45 100644 --- a/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsManager.scala +++ b/akka-actor/src/main/scala/akka/io/dns/internal/AsyncDnsManager.scala @@ -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) } } diff --git a/akka-actor/src/main/scala/akka/routing/TailChopping.scala b/akka-actor/src/main/scala/akka/routing/TailChopping.scala index 1cab177e8b..fa805ae5c3 100644 --- a/akka-actor/src/main/scala/akka/routing/TailChopping.scala +++ b/akka-actor/src/main/scala/akka/routing/TailChopping.scala @@ -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 { diff --git a/akka-bench-jmh/src/main/scala/akka/actor/ScheduleBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/actor/ScheduleBenchmark.scala index 99ab061c1f..2996321296 100644 --- a/akka-bench-jmh/src/main/scala/akka/actor/ScheduleBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/actor/ScheduleBenchmark.scala @@ -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) } diff --git a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsCollector.scala b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsCollector.scala index 963ad25271..d3bf8a88bb 100644 --- a/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsCollector.scala +++ b/akka-cluster-metrics/src/main/scala/akka/cluster/metrics/ClusterMetricsCollector.scala @@ -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, diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/Shard.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/Shard.scala index 6ff768975f..7dddc87b31 100644 --- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/Shard.scala +++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/Shard.scala @@ -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 = { diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardCoordinator.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardCoordinator.scala index 78990f62f1..a86558612d 100644 --- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardCoordinator.scala +++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardCoordinator.scala @@ -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) diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardRegion.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardRegion.scala index 629f0cc82c..f712506192 100644 --- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardRegion.scala +++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ShardRegion.scala @@ -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( diff --git a/akka-cluster-tools/src/main/scala/akka/cluster/client/ClusterClient.scala b/akka-cluster-tools/src/main/scala/akka/cluster/client/ClusterClient.scala index 125ecf4529..21e2e15af9 100644 --- a/akka-cluster-tools/src/main/scala/akka/cluster/client/ClusterClient.scala +++ b/akka-cluster-tools/src/main/scala/akka/cluster/client/ClusterClient.scala @@ -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() diff --git a/akka-cluster-tools/src/main/scala/akka/cluster/pubsub/DistributedPubSubMediator.scala b/akka-cluster-tools/src/main/scala/akka/cluster/pubsub/DistributedPubSubMediator.scala index 1ac22a52a7..ef8808b171 100644 --- a/akka-cluster-tools/src/main/scala/akka/cluster/pubsub/DistributedPubSubMediator.scala +++ b/akka-cluster-tools/src/main/scala/akka/cluster/pubsub/DistributedPubSubMediator.scala @@ -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 diff --git a/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonManager.scala b/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonManager.scala index e0268524dc..554701b27a 100644 --- a/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonManager.scala +++ b/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonManager.scala @@ -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 { diff --git a/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonProxy.scala b/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonProxy.scala index e17c7d7a64..40418bf346 100644 --- a/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonProxy.scala +++ b/akka-cluster-tools/src/main/scala/akka/cluster/singleton/ClusterSingletonProxy.scala @@ -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 = { diff --git a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionist.scala b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionist.scala index b94ebce145..f9f7e86f8e 100644 --- a/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionist.scala +++ b/akka-cluster-typed/src/main/scala/akka/cluster/typed/internal/receptionist/ClusterReceptionist.scala @@ -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]) } diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 7b108ca927..5cdb489270 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -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) diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala index 840e55c39c..9608d18b72 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala @@ -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() diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterHeartbeat.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterHeartbeat.scala index ac3208638a..4801326966 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterHeartbeat.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterHeartbeat.scala @@ -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 diff --git a/akka-cluster/src/main/scala/akka/cluster/CrossDcClusterHeartbeat.scala b/akka-cluster/src/main/scala/akka/cluster/CrossDcClusterHeartbeat.scala index ab499d217c..f5bee0184e 100644 --- a/akka-cluster/src/main/scala/akka/cluster/CrossDcClusterHeartbeat.scala +++ b/akka-cluster/src/main/scala/akka/cluster/CrossDcClusterHeartbeat.scala @@ -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, diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala index aee4b0d748..2bf7699ab3 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala @@ -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() diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/Replicator.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/Replicator.scala index cdd1a1b1ab..68fbddfe5f 100644 --- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/Replicator.scala +++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/Replicator.scala @@ -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 diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala index 6e0f949c4f..c1caa5c896 100644 --- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala +++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala @@ -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) diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/LotsOfDataBot.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/LotsOfDataBot.scala index 6b34abeed5..e3ca634ddd 100644 --- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/LotsOfDataBot.scala +++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/LotsOfDataBot.scala @@ -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) diff --git a/akka-docs/src/main/paradox/actors.md b/akka-docs/src/main/paradox/actors.md index 613e8c83f3..551b0389f5 100644 --- a/akka-docs/src/main/paradox/actors.md +++ b/akka-docs/src/main/paradox/actors.md @@ -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. diff --git a/akka-docs/src/main/paradox/fsm.md b/akka-docs/src/main/paradox/fsm.md index 631729f609..3f3261788f 100644 --- a/akka-docs/src/main/paradox/fsm.md +++ b/akka-docs/src/main/paradox/fsm.md @@ -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 ``` diff --git a/akka-docs/src/main/paradox/project/migration-guide-2.5.x-2.6.x.md b/akka-docs/src/main/paradox/project/migration-guide-2.5.x-2.6.x.md index f9b70efe9c..077b5d347a 100644 --- a/akka-docs/src/main/paradox/project/migration-guide-2.5.x-2.6.x.md +++ b/akka-docs/src/main/paradox/project/migration-guide-2.5.x-2.6.x.md @@ -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 diff --git a/akka-docs/src/main/paradox/scheduler.md b/akka-docs/src/main/paradox/scheduler.md index 651efb5450..b9d3502ab4 100644 --- a/akka-docs/src/main/paradox/scheduler.md +++ b/akka-docs/src/main/paradox/scheduler.md @@ -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 } diff --git a/akka-docs/src/main/paradox/typed/interaction-patterns.md b/akka-docs/src/main/paradox/typed/interaction-patterns.md index d2b7a507a9..2b32058b39 100644 --- a/akka-docs/src/main/paradox/typed/interaction-patterns.md +++ b/akka-docs/src/main/paradox/typed/interaction-patterns.md @@ -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. + +@@@ diff --git a/akka-docs/src/test/java/jdocs/actor/FaultHandlingDocSample.java b/akka-docs/src/test/java/jdocs/actor/FaultHandlingDocSample.java index c16eff95d0..aae650bfb1 100644 --- a/akka-docs/src/test/java/jdocs/actor/FaultHandlingDocSample.java +++ b/akka-docs/src/test/java/jdocs/actor/FaultHandlingDocSample.java @@ -151,7 +151,7 @@ public class FaultHandlingDocSample { getContext() .getSystem() .scheduler() - .schedule( + .scheduleWithFixedDelay( Duration.ZERO, Duration.ofSeconds(1L), getSelf(), diff --git a/akka-docs/src/test/java/jdocs/actor/SchedulerDocTest.java b/akka-docs/src/test/java/jdocs/actor/SchedulerDocTest.java index e5cfda9520..ef77be5bc4 100644 --- a/akka-docs/src/test/java/jdocs/actor/SchedulerDocTest.java +++ b/akka-docs/src/test/java/jdocs/actor/SchedulerDocTest.java @@ -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(); diff --git a/akka-docs/src/test/java/jdocs/actor/TimerDocTest.java b/akka-docs/src/test/java/jdocs/actor/TimerDocTest.java index 704f900d91..e443407d2a 100644 --- a/akka-docs/src/test/java/jdocs/actor/TimerDocTest.java +++ b/akka-docs/src/test/java/jdocs/actor/TimerDocTest.java @@ -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, diff --git a/akka-docs/src/test/java/jdocs/actor/fsm/FSMDocTest.java b/akka-docs/src/test/java/jdocs/actor/fsm/FSMDocTest.java index f9183bee81..b2c89753e0 100644 --- a/akka-docs/src/test/java/jdocs/actor/fsm/FSMDocTest.java +++ b/akka-docs/src/test/java/jdocs/actor/fsm/FSMDocTest.java @@ -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 diff --git a/akka-docs/src/test/java/jdocs/cluster/StatsSampleClient.java b/akka-docs/src/test/java/jdocs/cluster/StatsSampleClient.java index cd70fcc256..946a3a7835 100644 --- a/akka-docs/src/test/java/jdocs/cluster/StatsSampleClient.java +++ b/akka-docs/src/test/java/jdocs/cluster/StatsSampleClient.java @@ -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 diff --git a/akka-docs/src/test/java/jdocs/ddata/DataBot.java b/akka-docs/src/test/java/jdocs/ddata/DataBot.java index 6f7a145a68..04b7b54d7d 100644 --- a/akka-docs/src/test/java/jdocs/ddata/DataBot.java +++ b/akka-docs/src/test/java/jdocs/ddata/DataBot.java @@ -37,7 +37,7 @@ public class DataBot extends AbstractActor { getContext() .getSystem() .scheduler() - .schedule( + .scheduleWithFixedDelay( Duration.ofSeconds(5), Duration.ofSeconds(5), getSelf(), diff --git a/akka-docs/src/test/java/jdocs/stream/javadsl/cookbook/RecipeGlobalRateLimit.java b/akka-docs/src/test/java/jdocs/stream/javadsl/cookbook/RecipeGlobalRateLimit.java index 5d91d14c59..d0bf71e726 100644 --- a/akka-docs/src/test/java/jdocs/stream/javadsl/cookbook/RecipeGlobalRateLimit.java +++ b/akka-docs/src/test/java/jdocs/stream/javadsl/cookbook/RecipeGlobalRateLimit.java @@ -79,7 +79,7 @@ public class RecipeGlobalRateLimit extends RecipeTest { this.replenishTimer = system .scheduler() - .schedule( + .scheduleWithFixedDelay( this.tokenRefreshPeriod, this.tokenRefreshPeriod, getSelf(), diff --git a/akka-docs/src/test/scala/docs/actor/FSMDocSpec.scala b/akka-docs/src/test/scala/docs/actor/FSMDocSpec.scala index b647903034..17067f5846 100644 --- a/akka-docs/src/test/scala/docs/actor/FSMDocSpec.scala +++ b/akka-docs/src/test/scala/docs/actor/FSMDocSpec.scala @@ -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) } diff --git a/akka-docs/src/test/scala/docs/actor/FaultHandlingDocSample.scala b/akka-docs/src/test/scala/docs/actor/FaultHandlingDocSample.scala index 32f68d3f56..98c15b610c 100644 --- a/akka-docs/src/test/scala/docs/actor/FaultHandlingDocSample.scala +++ b/akka-docs/src/test/scala/docs/actor/FaultHandlingDocSample.scala @@ -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) diff --git a/akka-docs/src/test/scala/docs/actor/SchedulerDocSpec.scala b/akka-docs/src/test/scala/docs/actor/SchedulerDocSpec.scala index e3ae5353f2..bebb5d9209 100644 --- a/akka-docs/src/test/scala/docs/actor/SchedulerDocSpec.scala +++ b/akka-docs/src/test/scala/docs/actor/SchedulerDocSpec.scala @@ -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() diff --git a/akka-docs/src/test/scala/docs/actor/TimerDocSpec.scala b/akka-docs/src/test/scala/docs/actor/TimerDocSpec.scala index 2c8780926d..18ad21a4ab 100644 --- a/akka-docs/src/test/scala/docs/actor/TimerDocSpec.scala +++ b/akka-docs/src/test/scala/docs/actor/TimerDocSpec.scala @@ -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 } diff --git a/akka-docs/src/test/scala/docs/cluster/TransformationFrontend.scala b/akka-docs/src/test/scala/docs/cluster/TransformationFrontend.scala index e960110121..143c9a41a1 100644 --- a/akka-docs/src/test/scala/docs/cluster/TransformationFrontend.scala +++ b/akka-docs/src/test/scala/docs/cluster/TransformationFrontend.scala @@ -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) diff --git a/akka-docs/src/test/scala/docs/ddata/DistributedDataDocSpec.scala b/akka-docs/src/test/scala/docs/ddata/DistributedDataDocSpec.scala index e758c485d3..6ce507a5cd 100644 --- a/akka-docs/src/test/scala/docs/ddata/DistributedDataDocSpec.scala +++ b/akka-docs/src/test/scala/docs/ddata/DistributedDataDocSpec.scala @@ -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") diff --git a/akka-docs/src/test/scala/docs/persistence/query/MyEventsByTagSource.scala b/akka-docs/src/test/scala/docs/persistence/query/MyEventsByTagSource.scala index 7eb4deb8b9..1491707242 100644 --- a/akka-docs/src/test/scala/docs/persistence/query/MyEventsByTagSource.scala +++ b/akka-docs/src/test/scala/docs/persistence/query/MyEventsByTagSource.scala @@ -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 = { diff --git a/akka-docs/src/test/scala/docs/stream/cookbook/RecipeGlobalRateLimit.scala b/akka-docs/src/test/scala/docs/stream/cookbook/RecipeGlobalRateLimit.scala index 2569f6657a..cad1f7d561 100644 --- a/akka-docs/src/test/scala/docs/stream/cookbook/RecipeGlobalRateLimit.scala +++ b/akka-docs/src/test/scala/docs/stream/cookbook/RecipeGlobalRateLimit.scala @@ -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) diff --git a/akka-docs/src/test/scala/docs/testkit/TestkitDocSpec.scala b/akka-docs/src/test/scala/docs/testkit/TestkitDocSpec.scala index d86bc2b373..69323204c4 100644 --- a/akka-docs/src/test/scala/docs/testkit/TestkitDocSpec.scala +++ b/akka-docs/src/test/scala/docs/testkit/TestkitDocSpec.scala @@ -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) diff --git a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Conductor.scala b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Conductor.scala index c48e3f1dfe..258e5258ef 100644 --- a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Conductor.scala +++ b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Conductor.scala @@ -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)) diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdPublisher.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdPublisher.scala index de4cfca0d5..fec9e8fa18 100644 --- a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdPublisher.scala +++ b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdPublisher.scala @@ -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() diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagPublisher.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagPublisher.scala index 3a79430d7b..f3a9021b9a 100644 --- a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagPublisher.scala +++ b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagPublisher.scala @@ -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() diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/BehaviorSetup.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/BehaviorSetup.scala index af795d2917..2713e86da3 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/BehaviorSetup.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/BehaviorSetup.scala @@ -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) } diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PersistentActorCompileOnlyTest.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PersistentActorCompileOnlyTest.scala index 1029963c63..9dd9340448 100644 --- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PersistentActorCompileOnlyTest.scala +++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PersistentActorCompileOnlyTest.scala @@ -143,7 +143,7 @@ object PersistentActorCompileOnlyTest { }) Behaviors.withTimers((timers: TimerScheduler[Command]) => { - timers.startPeriodicTimer("swing", MoodSwing, 10.seconds) + timers.startTimerWithFixedDelay("swing", MoodSwing, 10.seconds) b }) } diff --git a/akka-persistence/src/main/mima-filters/2.5.x.backwards.excludes b/akka-persistence/src/main/mima-filters/2.5.x.backwards.excludes index 518f46d8ab..6dc291c225 100644 --- a/akka-persistence/src/main/mima-filters/2.5.x.backwards.excludes +++ b/akka-persistence/src/main/mima-filters/2.5.x.backwards.excludes @@ -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") diff --git a/akka-persistence/src/main/scala/akka/persistence/AtLeastOnceDelivery.scala b/akka-persistence/src/main/scala/akka/persistence/AtLeastOnceDelivery.scala index 03cafaeceb..16f2f00df3 100644 --- a/akka-persistence/src/main/scala/akka/persistence/AtLeastOnceDelivery.scala +++ b/akka-persistence/src/main/scala/akka/persistence/AtLeastOnceDelivery.scala @@ -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)) } } diff --git a/akka-persistence/src/main/scala/akka/persistence/Eventsourced.scala b/akka-persistence/src/main/scala/akka/persistence/Eventsourced.scala index 36f50f3f22..40da60bbe2 100644 --- a/akka-persistence/src/main/scala/akka/persistence/Eventsourced.scala +++ b/akka-persistence/src/main/scala/akka/persistence/Eventsourced.scala @@ -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 diff --git a/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSM.scala b/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSM.scala index c6536b90a4..90158ee6c0 100644 --- a/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSM.scala +++ b/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSM.scala @@ -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) { diff --git a/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSMBase.scala b/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSMBase.scala index 743567464b..7482d8a2bd 100644 --- a/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSMBase.scala +++ b/akka-persistence/src/main/scala/akka/persistence/fsm/PersistentFSMBase.scala @@ -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: * *
- *   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")
  * 
@@ -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) diff --git a/akka-remote/src/main/scala/akka/remote/Endpoint.scala b/akka-remote/src/main/scala/akka/remote/Endpoint.scala index e24b2fcfca..0af9ff21f2 100644 --- a/akka-remote/src/main/scala/akka/remote/Endpoint.scala +++ b/akka-remote/src/main/scala/akka/remote/Endpoint.scala @@ -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 = { diff --git a/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala b/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala index 81a8fbd4a2..42b93dba54 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala @@ -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() diff --git a/akka-remote/src/main/scala/akka/remote/Remoting.scala b/akka-remote/src/main/scala/akka/remote/Remoting.scala index 92c9706577..23c6bf1c20 100644 --- a/akka-remote/src/main/scala/akka/remote/Remoting.scala +++ b/akka-remote/src/main/scala/akka/remote/Remoting.scala @@ -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]]() diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index f88d7e8794..ea17952644 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -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) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index d00b9549ca..31438bb1a8 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -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 diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 48dc03ba41..c8186a55af 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -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 } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index 6afb1d14db..fdd9b48364 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -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 diff --git a/akka-remote/src/main/scala/akka/remote/artery/aeron/ArteryAeronUdpTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/aeron/ArteryAeronUdpTransport.scala index aa855d13d3..d3273dd7a2 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/aeron/ArteryAeronUdpTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/aeron/ArteryAeronUdpTransport.scala @@ -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 = { diff --git a/akka-remote/src/main/scala/akka/remote/transport/AkkaProtocolTransport.scala b/akka-remote/src/main/scala/akka/remote/transport/AkkaProtocolTransport.scala index 75efd89d68..fa2a59fc46 100644 --- a/akka-remote/src/main/scala/akka/remote/transport/AkkaProtocolTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/transport/AkkaProtocolTransport.scala @@ -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 = { diff --git a/akka-remote/src/main/scala/akka/remote/transport/ThrottlerTransportAdapter.scala b/akka-remote/src/main/scala/akka/remote/transport/ThrottlerTransportAdapter.scala index a58887d0f6..134a8bb15b 100644 --- a/akka-remote/src/main/scala/akka/remote/transport/ThrottlerTransportAdapter.scala +++ b/akka-remote/src/main/scala/akka/remote/transport/ThrottlerTransportAdapter.scala @@ -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) } } diff --git a/akka-remote/src/test/scala/akka/remote/classic/transport/AkkaProtocolStressTest.scala b/akka-remote/src/test/scala/akka/remote/classic/transport/AkkaProtocolStressTest.scala index d08afd979d..3f649c74e2 100644 --- a/akka-remote/src/test/scala/akka/remote/classic/transport/AkkaProtocolStressTest.scala +++ b/akka-remote/src/test/scala/akka/remote/classic/transport/AkkaProtocolStressTest.scala @@ -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 { diff --git a/akka-remote/src/test/scala/akka/remote/classic/transport/SystemMessageDeliveryStressTest.scala b/akka-remote/src/test/scala/akka/remote/classic/transport/SystemMessageDeliveryStressTest.scala index a8df71000d..79140fca67 100644 --- a/akka-remote/src/test/scala/akka/remote/classic/transport/SystemMessageDeliveryStressTest.scala +++ b/akka-remote/src/test/scala/akka/remote/classic/transport/SystemMessageDeliveryStressTest.scala @@ -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)) } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala index fc873d13e7..b48cfed8f1 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala @@ -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 diff --git a/akka-stream/src/main/mima-filters/2.5.x.backwards.excludes b/akka-stream/src/main/mima-filters/2.5.x.backwards.excludes index 856e3a971c..99322a572f 100644 --- a/akka-stream/src/main/mima-filters/2.5.x.backwards.excludes +++ b/akka-stream/src/main/mima-filters/2.5.x.backwards.excludes @@ -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") diff --git a/akka-stream/src/main/scala/akka/stream/Materializer.scala b/akka-stream/src/main/scala/akka/stream/Materializer.scala index 5b4e9c5e5c..8083ab556c 100644 --- a/akka-stream/src/main/scala/akka/stream/Materializer.scala +++ b/akka-stream/src/main/scala/akka/stream/Materializer.scala @@ -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") } /** diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorMaterializerImpl.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorMaterializerImpl.scala index 2a7246d8b7..4c5e6e5509 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/ActorMaterializerImpl.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/ActorMaterializerImpl.scala @@ -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) diff --git a/akka-stream/src/main/scala/akka/stream/impl/PhasedFusingActorMaterializer.scala b/akka-stream/src/main/scala/akka/stream/impl/PhasedFusingActorMaterializer.scala index 4bdf8c8bb0..aafa688b69 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/PhasedFusingActorMaterializer.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/PhasedFusingActorMaterializer.scala @@ -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) diff --git a/akka-stream/src/main/scala/akka/stream/impl/Timers.scala b/akka-stream/src/main/scala/akka/stream/impl/Timers.scala index 12ebedb097..bd20c2e49b 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/Timers.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/Timers.scala @@ -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 = { diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphStages.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphStages.scala index d6f59772c8..b3ad7a2fb6 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphStages.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphStages.scala @@ -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) diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala index f63fc58087..111b361e89 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala @@ -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() } } diff --git a/akka-stream/src/main/scala/akka/stream/stage/GraphStage.scala b/akka-stream/src/main/scala/akka/stream/stage/GraphStage.scala index 61160b9f34..4852aa0f0c 100644 --- a/akka-stream/src/main/scala/akka/stream/stage/GraphStage.scala +++ b/akka-stream/src/main/scala/akka/stream/stage/GraphStage.scala @@ -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]]. */ diff --git a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala index f886bbbaf0..d616cd314c 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala @@ -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) } diff --git a/akka-testkit/src/test/scala/akka/testkit/TestFSMRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestFSMRefSpec.scala index cb8d797c18..d28246734a 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestFSMRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestFSMRefSpec.scala @@ -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)