2018-10-29 17:19:37 +08:00
|
|
|
/*
|
2018-01-04 17:26:29 +00:00
|
|
|
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
|
2013-01-23 21:02:41 +01:00
|
|
|
*/
|
|
|
|
|
|
2010-10-26 12:49:25 +02:00
|
|
|
package akka.actor
|
2010-08-24 23:21:28 +02:00
|
|
|
|
2012-06-21 16:09:14 +02:00
|
|
|
import language.postfixOps
|
2013-01-14 23:21:51 +01:00
|
|
|
import java.io.Closeable
|
|
|
|
|
import java.util.concurrent._
|
2013-02-04 09:39:45 +01:00
|
|
|
import atomic.{ AtomicReference, AtomicInteger }
|
2013-01-14 23:21:51 +01:00
|
|
|
import scala.concurrent.{ future, Await, ExecutionContext }
|
2012-09-21 14:50:06 +02:00
|
|
|
import scala.concurrent.duration._
|
2015-09-25 08:39:02 +02:00
|
|
|
import java.util.concurrent.ThreadLocalRandom
|
2013-01-14 23:21:51 +01:00
|
|
|
import scala.util.Try
|
|
|
|
|
import scala.util.control.NonFatal
|
|
|
|
|
import org.scalatest.BeforeAndAfterEach
|
|
|
|
|
import com.typesafe.config.{ Config, ConfigFactory }
|
2012-01-18 10:18:51 +01:00
|
|
|
import akka.pattern.ask
|
2013-01-14 23:21:51 +01:00
|
|
|
import akka.testkit._
|
2015-09-10 17:03:18 +02:00
|
|
|
import scala.util.control.NoStackTrace
|
2013-01-10 10:08:23 -08:00
|
|
|
|
|
|
|
|
object SchedulerSpec {
|
2013-12-11 11:53:02 +01:00
|
|
|
val testConfRevolver = ConfigFactory.parseString("""
|
|
|
|
|
akka.scheduler.implementation = akka.actor.LightArrayRevolverScheduler
|
2013-01-10 10:08:23 -08:00
|
|
|
akka.scheduler.ticks-per-wheel = 32
|
2013-08-23 14:39:21 +02:00
|
|
|
akka.actor.serialize-messages = off
|
2013-01-10 10:08:23 -08:00
|
|
|
""").withFallback(AkkaSpec.testConf)
|
|
|
|
|
}
|
2010-08-24 23:21:28 +02:00
|
|
|
|
2013-01-14 23:21:51 +01:00
|
|
|
trait SchedulerSpec extends BeforeAndAfterEach with DefaultTimeout with ImplicitSender { this: AkkaSpec ⇒
|
2012-08-08 15:57:30 +02:00
|
|
|
import system.dispatcher
|
2011-07-12 12:17:32 +02:00
|
|
|
|
2013-01-14 23:21:51 +01:00
|
|
|
def collectCancellable(c: Cancellable): Cancellable
|
2010-08-24 23:21:28 +02:00
|
|
|
|
2011-10-11 16:05:48 +02:00
|
|
|
"A Scheduler" must {
|
2010-08-24 23:21:28 +02:00
|
|
|
|
2013-01-14 23:21:51 +01:00
|
|
|
"schedule more than once" taggedAs TimingTest in {
|
2011-10-11 16:05:48 +02:00
|
|
|
case object Tick
|
2012-12-18 00:51:11 +01:00
|
|
|
case object Tock
|
|
|
|
|
|
|
|
|
|
val tickActor, tickActor2 = system.actorOf(Props(new Actor {
|
|
|
|
|
var ticks = 0
|
|
|
|
|
def receive = {
|
|
|
|
|
case Tick ⇒
|
|
|
|
|
if (ticks < 3) {
|
2014-01-16 15:16:35 +01:00
|
|
|
sender() ! Tock
|
2012-12-18 00:51:11 +01:00
|
|
|
ticks += 1
|
|
|
|
|
}
|
|
|
|
|
}
|
2011-12-13 14:09:40 +01:00
|
|
|
}))
|
2011-11-26 15:24:13 +01:00
|
|
|
// run every 50 milliseconds
|
2011-12-02 17:13:46 +01:00
|
|
|
collectCancellable(system.scheduler.schedule(0 milliseconds, 50 milliseconds, tickActor, Tick))
|
2010-08-24 23:21:28 +02:00
|
|
|
|
2011-10-11 16:05:48 +02:00
|
|
|
// after max 1 second it should be executed at least the 3 times already
|
2012-12-18 00:51:11 +01:00
|
|
|
expectMsg(Tock)
|
|
|
|
|
expectMsg(Tock)
|
|
|
|
|
expectMsg(Tock)
|
|
|
|
|
expectNoMsg(500 millis)
|
2010-08-24 23:21:28 +02:00
|
|
|
|
2012-12-18 00:51:11 +01:00
|
|
|
collectCancellable(system.scheduler.schedule(0 milliseconds, 50 milliseconds)(tickActor2 ! Tick))
|
2010-08-24 23:21:28 +02:00
|
|
|
|
2011-10-11 16:05:48 +02:00
|
|
|
// after max 1 second it should be executed at least the 3 times already
|
2012-12-18 00:51:11 +01:00
|
|
|
expectMsg(Tock)
|
|
|
|
|
expectMsg(Tock)
|
|
|
|
|
expectMsg(Tock)
|
|
|
|
|
expectNoMsg(500 millis)
|
2011-10-11 16:05:48 +02:00
|
|
|
}
|
2010-08-24 23:21:28 +02:00
|
|
|
|
2012-05-28 18:37:41 +02:00
|
|
|
"stop continuous scheduling if the receiving actor has been terminated" taggedAs TimingTest in {
|
2014-01-16 15:16:35 +01:00
|
|
|
val actor = system.actorOf(Props(new Actor { def receive = { case x ⇒ sender() ! x } }))
|
2012-02-03 14:52:10 +01:00
|
|
|
|
2011-11-25 17:56:14 +01:00
|
|
|
// run immediately and then every 100 milliseconds
|
2012-02-03 14:45:58 +01:00
|
|
|
collectCancellable(system.scheduler.schedule(0 milliseconds, 100 milliseconds, actor, "msg"))
|
2012-05-13 23:22:41 +02:00
|
|
|
expectMsg("msg")
|
2011-11-25 17:56:14 +01:00
|
|
|
|
|
|
|
|
// stop the actor and, hence, the continuous messaging from happening
|
2012-12-18 00:51:11 +01:00
|
|
|
system stop actor
|
2011-11-25 17:56:14 +01:00
|
|
|
|
2012-12-18 00:51:11 +01:00
|
|
|
expectNoMsg(500 millis)
|
2011-11-25 17:56:14 +01:00
|
|
|
}
|
|
|
|
|
|
2015-09-10 17:03:18 +02:00
|
|
|
"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)
|
|
|
|
|
expectNoMsg(500 millis)
|
|
|
|
|
}
|
|
|
|
|
|
2013-01-14 23:21:51 +01:00
|
|
|
"schedule once" taggedAs TimingTest in {
|
2011-10-11 16:05:48 +02:00
|
|
|
case object Tick
|
|
|
|
|
val countDownLatch = new CountDownLatch(3)
|
2011-12-13 14:09:40 +01:00
|
|
|
val tickActor = system.actorOf(Props(new Actor {
|
2011-10-11 16:05:48 +02:00
|
|
|
def receive = { case Tick ⇒ countDownLatch.countDown() }
|
2011-12-13 14:09:40 +01:00
|
|
|
}))
|
2011-11-09 15:25:14 +01:00
|
|
|
|
2011-12-02 17:13:46 +01:00
|
|
|
// 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)
|
2018-11-09 09:07:21 +01:00
|
|
|
countDownLatch.getCount should ===(3L)
|
2011-10-11 16:05:48 +02:00
|
|
|
|
|
|
|
|
// after 1 second the wait should fail
|
2012-02-03 14:45:58 +01:00
|
|
|
assert(countDownLatch.await(2, TimeUnit.SECONDS) == false)
|
2011-10-11 16:05:48 +02:00
|
|
|
// should still be 1 left
|
2018-11-09 09:07:21 +01:00
|
|
|
countDownLatch.getCount should ===(1L)
|
2010-08-24 23:21:28 +02:00
|
|
|
}
|
|
|
|
|
|
2011-10-11 16:05:48 +02:00
|
|
|
/**
|
|
|
|
|
* ticket #372
|
|
|
|
|
*/
|
2013-01-14 23:21:51 +01:00
|
|
|
"be cancellable" taggedAs TimingTest in {
|
2012-12-18 00:51:11 +01:00
|
|
|
for (_ ← 1 to 10) system.scheduler.scheduleOnce(1 second, testActor, "fail").cancel()
|
2011-11-09 15:25:14 +01:00
|
|
|
|
2012-12-18 00:51:11 +01:00
|
|
|
expectNoMsg(2 seconds)
|
2011-10-11 16:05:48 +02:00
|
|
|
}
|
2010-08-24 23:21:28 +02:00
|
|
|
|
2012-02-03 14:45:58 +01:00
|
|
|
"be cancellable during initial delay" taggedAs TimingTest in {
|
2012-01-13 16:38:33 +01:00
|
|
|
val ticks = new AtomicInteger
|
|
|
|
|
|
|
|
|
|
val initialDelay = 200.milliseconds.dilated
|
|
|
|
|
val delay = 10.milliseconds.dilated
|
|
|
|
|
val timeout = collectCancellable(system.scheduler.schedule(initialDelay, delay) {
|
|
|
|
|
ticks.incrementAndGet()
|
|
|
|
|
})
|
2012-06-29 13:33:20 +02:00
|
|
|
Thread.sleep(10.milliseconds.dilated.toMillis)
|
2012-01-13 16:38:33 +01:00
|
|
|
timeout.cancel()
|
2012-06-29 13:33:20 +02:00
|
|
|
Thread.sleep((initialDelay + 100.milliseconds.dilated).toMillis)
|
2012-01-13 16:38:33 +01:00
|
|
|
|
2015-01-16 11:09:59 +01:00
|
|
|
ticks.get should ===(0)
|
2012-01-13 16:38:33 +01:00
|
|
|
}
|
|
|
|
|
|
2012-02-03 14:45:58 +01:00
|
|
|
"be cancellable after initial delay" taggedAs TimingTest in {
|
2012-01-13 16:38:33 +01:00
|
|
|
val ticks = new AtomicInteger
|
|
|
|
|
|
2013-01-14 23:21:51 +01:00
|
|
|
val initialDelay = 90.milliseconds.dilated
|
|
|
|
|
val delay = 500.milliseconds.dilated
|
2012-01-13 16:38:33 +01:00
|
|
|
val timeout = collectCancellable(system.scheduler.schedule(initialDelay, delay) {
|
|
|
|
|
ticks.incrementAndGet()
|
|
|
|
|
})
|
2013-01-14 23:21:51 +01:00
|
|
|
Thread.sleep((initialDelay + 200.milliseconds.dilated).toMillis)
|
2012-01-13 16:38:33 +01:00
|
|
|
timeout.cancel()
|
2012-06-29 13:33:20 +02:00
|
|
|
Thread.sleep((delay + 100.milliseconds.dilated).toMillis)
|
2012-01-13 16:38:33 +01:00
|
|
|
|
2015-01-16 11:09:59 +01:00
|
|
|
ticks.get should ===(1)
|
2012-01-13 16:38:33 +01:00
|
|
|
}
|
|
|
|
|
|
2016-11-29 08:33:36 +01:00
|
|
|
"be canceled if cancel is performed before execution" taggedAs TimingTest in {
|
2013-03-28 23:45:48 +01:00
|
|
|
val task = collectCancellable(system.scheduler.scheduleOnce(10 seconds)(()))
|
2015-01-16 11:09:59 +01:00
|
|
|
task.cancel() should ===(true)
|
|
|
|
|
task.isCancelled should ===(true)
|
|
|
|
|
task.cancel() should ===(false)
|
|
|
|
|
task.isCancelled should ===(true)
|
2013-02-20 12:40:40 +01:00
|
|
|
}
|
|
|
|
|
|
2016-11-29 08:33:36 +01:00
|
|
|
"not be canceled if cancel is performed after execution" taggedAs TimingTest in {
|
2013-02-20 12:40:40 +01:00
|
|
|
val latch = TestLatch(1)
|
|
|
|
|
val task = collectCancellable(system.scheduler.scheduleOnce(10 millis)(latch.countDown()))
|
2014-03-11 11:23:12 +01:00
|
|
|
Await.ready(latch, remainingOrDefault)
|
2015-01-16 11:09:59 +01:00
|
|
|
task.cancel() should ===(false)
|
|
|
|
|
task.isCancelled should ===(false)
|
|
|
|
|
task.cancel() should ===(false)
|
|
|
|
|
task.isCancelled should ===(false)
|
2013-02-20 12:40:40 +01:00
|
|
|
}
|
|
|
|
|
|
2011-10-11 16:05:48 +02:00
|
|
|
/**
|
|
|
|
|
* ticket #307
|
|
|
|
|
*/
|
2013-01-14 23:21:51 +01:00
|
|
|
"pick up schedule after actor restart" taggedAs TimingTest in {
|
2011-12-05 20:01:42 +01:00
|
|
|
|
2011-10-11 16:05:48 +02:00
|
|
|
object Ping
|
|
|
|
|
object Crash
|
|
|
|
|
|
2011-12-14 12:39:27 +01:00
|
|
|
val restartLatch = new TestLatch
|
2011-12-19 15:05:33 +01:00
|
|
|
val pingLatch = new TestLatch(6)
|
2011-10-11 16:05:48 +02:00
|
|
|
|
2012-01-24 10:35:09 +01:00
|
|
|
val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(3, 1 second)(List(classOf[Exception])))))
|
2011-10-18 15:39:26 +02:00
|
|
|
val props = Props(new Actor {
|
2011-10-11 16:05:48 +02:00
|
|
|
def receive = {
|
|
|
|
|
case Ping ⇒ pingLatch.countDown()
|
|
|
|
|
case Crash ⇒ throw new Exception("CRASH")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def postRestart(reason: Throwable) = restartLatch.open
|
2011-10-18 15:39:26 +02:00
|
|
|
})
|
2011-12-12 22:50:08 +01:00
|
|
|
val actor = Await.result((supervisor ? props).mapTo[ActorRef], timeout.duration)
|
2011-10-11 16:05:48 +02:00
|
|
|
|
2011-12-02 17:13:46 +01:00
|
|
|
collectCancellable(system.scheduler.schedule(500 milliseconds, 500 milliseconds, actor, Ping))
|
2011-10-11 16:05:48 +02:00
|
|
|
// appx 2 pings before crash
|
2011-11-11 18:41:43 +01:00
|
|
|
EventFilter[Exception]("CRASH", occurrences = 1) intercept {
|
2011-12-02 17:13:46 +01:00
|
|
|
collectCancellable(system.scheduler.scheduleOnce(1000 milliseconds, actor, Crash))
|
2011-11-11 18:41:43 +01:00
|
|
|
}
|
2011-10-11 16:05:48 +02:00
|
|
|
|
2011-12-19 15:05:33 +01:00
|
|
|
Await.ready(restartLatch, 2 seconds)
|
2011-10-11 16:05:48 +02:00
|
|
|
// should be enough time for the ping countdown to recover and reach 6 pings
|
2011-12-20 10:58:17 +01:00
|
|
|
Await.ready(pingLatch, 5 seconds)
|
2011-10-11 16:05:48 +02:00
|
|
|
}
|
2011-11-22 15:26:21 +01:00
|
|
|
|
2013-01-14 23:21:51 +01:00
|
|
|
"never fire prematurely" taggedAs TimingTest in {
|
2011-12-19 15:05:33 +01:00
|
|
|
val ticks = new TestLatch(300)
|
2011-11-22 15:26:21 +01:00
|
|
|
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class Msg(ts: Long)
|
2011-11-22 15:26:21 +01:00
|
|
|
|
2011-12-13 14:09:40 +01:00
|
|
|
val actor = system.actorOf(Props(new Actor {
|
2011-11-22 15:26:21 +01:00
|
|
|
def receive = {
|
|
|
|
|
case Msg(ts) ⇒
|
2011-11-24 14:30:03 +01:00
|
|
|
val now = System.nanoTime
|
2012-01-24 08:37:01 +01:00
|
|
|
// Make sure that no message has been dispatched before the scheduled time (10ms) has occurred
|
2013-06-26 17:01:29 +02:00
|
|
|
if (now < ts) throw new RuntimeException("Interval is too small: " + (now - ts))
|
2011-11-22 15:26:21 +01:00
|
|
|
ticks.countDown()
|
|
|
|
|
}
|
2011-12-13 14:09:40 +01:00
|
|
|
}))
|
2011-11-22 15:26:21 +01:00
|
|
|
|
2011-11-24 09:54:08 +01:00
|
|
|
(1 to 300).foreach { i ⇒
|
2013-06-03 22:09:45 +02:00
|
|
|
collectCancellable(system.scheduler.scheduleOnce(20 milliseconds, actor, Msg(System.nanoTime)))
|
2011-11-22 15:26:21 +01:00
|
|
|
Thread.sleep(5)
|
|
|
|
|
}
|
|
|
|
|
|
2011-12-19 15:05:33 +01:00
|
|
|
Await.ready(ticks, 3 seconds)
|
2011-11-22 15:26:21 +01:00
|
|
|
}
|
2011-11-23 15:15:44 +01:00
|
|
|
|
2012-02-03 14:45:58 +01:00
|
|
|
"schedule with different initial delay and frequency" taggedAs TimingTest in {
|
2011-12-19 15:05:33 +01:00
|
|
|
val ticks = new TestLatch(3)
|
2011-11-23 15:15:44 +01:00
|
|
|
|
|
|
|
|
case object Msg
|
|
|
|
|
|
2011-12-13 14:09:40 +01:00
|
|
|
val actor = system.actorOf(Props(new Actor {
|
2012-12-18 00:51:11 +01:00
|
|
|
def receive = { case Msg ⇒ ticks.countDown() }
|
2011-12-13 14:09:40 +01:00
|
|
|
}))
|
2011-11-23 15:15:44 +01:00
|
|
|
|
|
|
|
|
val startTime = System.nanoTime()
|
2012-12-18 00:51:11 +01:00
|
|
|
collectCancellable(system.scheduler.schedule(1 second, 300 milliseconds, actor, Msg))
|
2011-12-19 15:05:33 +01:00
|
|
|
Await.ready(ticks, 3 seconds)
|
2011-11-23 15:15:44 +01:00
|
|
|
|
2013-01-25 13:10:14 +01:00
|
|
|
// LARS is a bit more aggressive in scheduling recurring tasks at the right
|
2013-02-20 12:40:40 +01:00
|
|
|
// frequency and may execute them a little earlier; the actual expected timing
|
2013-01-25 13:10:14 +01:00
|
|
|
// is 1599ms on a fast machine or 1699ms on a loaded one (plus some room for jenkins)
|
2015-01-16 11:09:59 +01:00
|
|
|
(System.nanoTime() - startTime).nanos.toMillis should ===(1750L +- 250)
|
2011-11-23 15:15:44 +01:00
|
|
|
}
|
2012-10-08 12:17:40 +02:00
|
|
|
|
|
|
|
|
"adjust for scheduler inaccuracy" taggedAs TimingTest in {
|
|
|
|
|
val startTime = System.nanoTime
|
2013-06-03 22:09:45 +02:00
|
|
|
val n = 200
|
2012-10-08 12:17:40 +02:00
|
|
|
val latch = new TestLatch(n)
|
2013-06-03 22:09:45 +02:00
|
|
|
system.scheduler.schedule(25.millis, 25.millis) { latch.countDown() }
|
2012-10-08 12:17:40 +02:00
|
|
|
Await.ready(latch, 6.seconds)
|
2012-12-18 00:51:11 +01:00
|
|
|
// Rate
|
2015-01-16 11:09:59 +01:00
|
|
|
n * 1000.0 / (System.nanoTime - startTime).nanos.toMillis should ===(40.0 +- 4)
|
2012-10-08 12:17:40 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"not be affected by long running task" taggedAs TimingTest in {
|
|
|
|
|
val n = 22
|
|
|
|
|
val latch = new TestLatch(n)
|
2015-08-10 12:35:41 +02:00
|
|
|
val startTime = System.nanoTime
|
2012-10-08 12:17:40 +02:00
|
|
|
system.scheduler.schedule(225.millis, 225.millis) {
|
2015-08-10 12:35:41 +02:00
|
|
|
Thread.sleep(100)
|
2012-10-08 12:17:40 +02:00
|
|
|
latch.countDown()
|
|
|
|
|
}
|
|
|
|
|
Await.ready(latch, 6.seconds)
|
2012-12-18 00:51:11 +01:00
|
|
|
// Rate
|
2015-08-10 12:35:41 +02:00
|
|
|
n * 1000.0 / (System.nanoTime - startTime).nanos.toMillis should ===(4.4 +- 0.5)
|
2012-10-08 12:17:40 +02:00
|
|
|
}
|
2013-01-14 23:21:51 +01:00
|
|
|
|
2013-01-24 21:11:07 +01:00
|
|
|
"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()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-01-14 23:21:51 +01:00
|
|
|
"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)
|
|
|
|
|
}
|
|
|
|
|
}
|
2013-02-26 11:43:29 +01:00
|
|
|
val latencies = within(10.seconds) {
|
2013-01-14 23:21:51 +01:00
|
|
|
for (i ← 1 to N) 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}")
|
|
|
|
|
}
|
|
|
|
|
}
|
2010-08-24 23:21:28 +02:00
|
|
|
}
|
2013-01-24 21:11:07 +01:00
|
|
|
}
|
|
|
|
|
|
2013-01-14 23:21:51 +01:00
|
|
|
class LightArrayRevolverSchedulerSpec extends AkkaSpec(SchedulerSpec.testConfRevolver) with SchedulerSpec {
|
|
|
|
|
|
|
|
|
|
def collectCancellable(c: Cancellable): Cancellable = c
|
|
|
|
|
|
2013-11-11 11:19:25 +01:00
|
|
|
def tickDuration = system.scheduler.asInstanceOf[LightArrayRevolverScheduler].TickDuration
|
|
|
|
|
|
2013-01-14 23:21:51 +01:00
|
|
|
"A LightArrayRevolverScheduler" must {
|
|
|
|
|
|
2016-11-29 08:33:36 +01:00
|
|
|
"reject tasks scheduled too far into the future" taggedAs TimingTest in {
|
2013-11-11 11:19:25 +01:00
|
|
|
val maxDelay = tickDuration * Int.MaxValue
|
|
|
|
|
import system.dispatcher
|
2016-10-18 15:38:52 +05:00
|
|
|
system.scheduler.scheduleOnce(maxDelay, testActor, "OK")
|
2013-11-11 11:19:25 +01:00
|
|
|
intercept[IllegalArgumentException] {
|
2016-10-18 15:38:52 +05:00
|
|
|
system.scheduler.scheduleOnce(maxDelay + tickDuration, testActor, "Too far")
|
2013-11-11 11:19:25 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-11-29 08:33:36 +01:00
|
|
|
"reject periodic tasks scheduled too far into the future" taggedAs TimingTest in {
|
2013-11-11 11:19:25 +01:00
|
|
|
val maxDelay = tickDuration * Int.MaxValue
|
|
|
|
|
import system.dispatcher
|
2016-10-18 15:38:52 +05:00
|
|
|
system.scheduler.schedule(maxDelay, 1.second, testActor, "OK")
|
2013-11-11 11:19:25 +01:00
|
|
|
intercept[IllegalArgumentException] {
|
2016-10-18 15:38:52 +05:00
|
|
|
system.scheduler.schedule(maxDelay + tickDuration, 1.second, testActor, "Too far")
|
2013-11-11 11:19:25 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-11-29 08:33:36 +01:00
|
|
|
"reject periodic tasks scheduled with too long interval" taggedAs TimingTest in {
|
2013-11-11 11:19:25 +01:00
|
|
|
val maxDelay = tickDuration * Int.MaxValue
|
|
|
|
|
import system.dispatcher
|
2016-10-18 15:38:52 +05:00
|
|
|
system.scheduler.schedule(100.millis, maxDelay, testActor, "OK")
|
2013-11-11 11:19:25 +01:00
|
|
|
expectMsg("OK")
|
|
|
|
|
intercept[IllegalArgumentException] {
|
2016-10-18 15:38:52 +05:00
|
|
|
system.scheduler.schedule(100.millis, maxDelay + tickDuration, testActor, "Too long")
|
2013-11-11 11:19:25 +01:00
|
|
|
}
|
|
|
|
|
expectNoMsg(1.second)
|
|
|
|
|
}
|
|
|
|
|
|
2013-01-14 23:21:51 +01:00
|
|
|
"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)
|
2013-02-26 11:43:29 +01:00
|
|
|
val latencies = within(10.seconds) {
|
2013-01-14 23:21:51 +01:00
|
|
|
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}")
|
|
|
|
|
}
|
|
|
|
|
expectNoMsg(1.second)
|
|
|
|
|
}
|
|
|
|
|
|
2016-11-29 08:33:36 +01:00
|
|
|
"survive vicious enqueueing" taggedAs TimingTest in {
|
2013-01-14 23:21:51 +01:00
|
|
|
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()
|
2013-12-17 14:25:56 +01:00
|
|
|
Await.result(terminated, 3.seconds.dilated) should be > 10
|
2013-01-14 23:21:51 +01:00
|
|
|
awaitCond(counter.get == N)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-11-29 08:33:36 +01:00
|
|
|
"execute multiple jobs at once when expiring multiple buckets" taggedAs TimingTest in {
|
2013-01-14 23:21:51 +01:00
|
|
|
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"))
|
|
|
|
|
expectNoMsg(step)
|
|
|
|
|
wakeUp(step)
|
|
|
|
|
expectWait(step)
|
|
|
|
|
wakeUp(step * 4 + step / 2)
|
|
|
|
|
expectWait(step / 2)
|
|
|
|
|
(0 to 3) foreach (_ ⇒ expectMsg(Duration.Zero, "hello"))
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-11-29 08:33:36 +01:00
|
|
|
"properly defer jobs even when the timer thread oversleeps" taggedAs TimingTest in {
|
2013-06-20 12:33:41 +02:00
|
|
|
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)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-11-29 08:33:36 +01:00
|
|
|
"correctly wrap around wheel rounds" taggedAs TimingTest in {
|
2013-01-14 23:21:51 +01:00
|
|
|
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.expectNoMsg(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)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-11-29 08:33:36 +01:00
|
|
|
"correctly execute jobs when clock wraps around" taggedAs TimingTest in {
|
2013-01-14 23:21:51 +01:00
|
|
|
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"))
|
|
|
|
|
expectNoMsg(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)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-11-29 08:33:36 +01:00
|
|
|
"correctly wrap around ticks" taggedAs TimingTest in {
|
2016-05-06 10:53:41 +02:00
|
|
|
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))
|
|
|
|
|
expectNoMsg(step)
|
|
|
|
|
wakeUp(step)
|
|
|
|
|
expectWait(step)
|
|
|
|
|
|
|
|
|
|
nums foreach { i ⇒
|
|
|
|
|
wakeUp(step)
|
|
|
|
|
expectMsg("hello-" + i)
|
|
|
|
|
expectWait(step)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-11-29 08:33:36 +01:00
|
|
|
"reliably reject jobs when shutting down" taggedAs TimingTest in {
|
2013-01-14 23:21:51 +01:00
|
|
|
withScheduler() { (sched, driver) ⇒
|
|
|
|
|
import system.dispatcher
|
|
|
|
|
val counter = new AtomicInteger
|
2013-02-05 16:28:30 +01:00
|
|
|
future { Thread.sleep(5); driver.close(); sched.close() }
|
2013-01-14 23:21:51 +01:00
|
|
|
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
|
2013-12-17 14:25:56 +01:00
|
|
|
s should be < cap
|
2013-01-14 23:21:51 +01:00
|
|
|
awaitCond(s == counter.get, message = s"$s was not ${counter.get}")
|
2015-01-16 11:09:59 +01:00
|
|
|
failure.size should ===(headroom)
|
2013-01-14 23:21:51 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
trait Driver {
|
|
|
|
|
def wakeUp(d: FiniteDuration): Unit
|
|
|
|
|
def expectWait(): FiniteDuration
|
2018-07-25 20:38:27 +09:00
|
|
|
def expectWait(d: FiniteDuration): Unit = { expectWait() should ===(d) }
|
2013-01-14 23:21:51 +01:00
|
|
|
def probe: TestProbe
|
|
|
|
|
def step: FiniteDuration
|
2013-02-05 16:28:30 +01:00
|
|
|
def close(): Unit
|
2013-01-14 23:21:51 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val localEC = new ExecutionContext {
|
2018-07-25 20:38:27 +09:00
|
|
|
def execute(runnable: Runnable): Unit = { runnable.run() }
|
|
|
|
|
def reportFailure(t: Throwable): Unit = { t.printStackTrace() }
|
2013-01-14 23:21:51 +01:00
|
|
|
}
|
|
|
|
|
|
2016-05-06 10:53:41 +02:00
|
|
|
def withScheduler(start: Long = 0L, _startTick: Int = 0, config: Config = ConfigFactory.empty)(thunk: (Scheduler with Closeable, Driver) ⇒ Unit): Unit = {
|
2013-01-14 23:21:51 +01:00
|
|
|
import akka.actor.{ LightArrayRevolverScheduler ⇒ LARS }
|
2013-02-04 09:39:45 +01:00
|
|
|
val lbq = new AtomicReference[LinkedBlockingQueue[Long]](new LinkedBlockingQueue[Long])
|
2013-01-14 23:21:51 +01:00
|
|
|
val prb = TestProbe()
|
|
|
|
|
val tf = system.asInstanceOf[ActorSystemImpl].threadFactory
|
|
|
|
|
val sched =
|
|
|
|
|
new { @volatile var time = start } with LARS(config.withFallback(system.settings.config), log, tf) {
|
|
|
|
|
override protected def clock(): Long = {
|
|
|
|
|
// println(s"clock=$time")
|
|
|
|
|
time
|
|
|
|
|
}
|
2014-02-07 09:16:43 +01:00
|
|
|
|
|
|
|
|
override protected def getShutdownTimeout: FiniteDuration = (10 seconds).dilated
|
2013-02-11 13:06:33 +01:00
|
|
|
|
2013-01-14 23:21:51 +01:00
|
|
|
override protected def waitNanos(ns: Long): Unit = {
|
|
|
|
|
// println(s"waiting $ns")
|
|
|
|
|
prb.ref ! ns
|
2013-02-04 09:39:45 +01:00
|
|
|
try time += (lbq.get match {
|
|
|
|
|
case q: LinkedBlockingQueue[Long] ⇒ q.take()
|
2013-02-05 16:28:30 +01:00
|
|
|
case _ ⇒ 0L
|
2013-02-04 09:39:45 +01:00
|
|
|
})
|
2013-01-14 23:21:51 +01:00
|
|
|
catch {
|
2013-01-24 21:11:07 +01:00
|
|
|
case _: InterruptedException ⇒ Thread.currentThread.interrupt()
|
2013-01-14 23:21:51 +01:00
|
|
|
}
|
|
|
|
|
}
|
2016-05-06 10:53:41 +02:00
|
|
|
|
|
|
|
|
override protected def startTick: Int = _startTick
|
2013-01-14 23:21:51 +01:00
|
|
|
}
|
|
|
|
|
val driver = new Driver {
|
2013-02-04 09:39:45 +01:00
|
|
|
def wakeUp(d: FiniteDuration) = lbq.get match {
|
|
|
|
|
case q: LinkedBlockingQueue[Long] ⇒ q.offer(d.toNanos)
|
|
|
|
|
case _ ⇒
|
|
|
|
|
}
|
2013-01-14 23:21:51 +01:00
|
|
|
def expectWait(): FiniteDuration = probe.expectMsgType[Long].nanos
|
|
|
|
|
def probe = prb
|
|
|
|
|
def step = sched.TickDuration
|
2013-02-05 16:28:30 +01:00
|
|
|
def close() = lbq.getAndSet(null) match {
|
|
|
|
|
case q: LinkedBlockingQueue[Long] ⇒ q.offer(0L)
|
|
|
|
|
case _ ⇒
|
|
|
|
|
}
|
2013-01-14 23:21:51 +01:00
|
|
|
}
|
|
|
|
|
driver.expectWait()
|
|
|
|
|
try thunk(sched, driver)
|
2013-01-24 21:11:07 +01:00
|
|
|
catch {
|
|
|
|
|
case NonFatal(ex) ⇒
|
2013-02-05 16:28:30 +01:00
|
|
|
try {
|
|
|
|
|
driver.close()
|
|
|
|
|
sched.close()
|
|
|
|
|
} catch { case _: Exception ⇒ }
|
2013-01-24 21:11:07 +01:00
|
|
|
throw ex
|
|
|
|
|
}
|
2013-02-05 16:28:30 +01:00
|
|
|
driver.close()
|
2013-01-24 21:11:07 +01:00
|
|
|
sched.close()
|
2013-01-14 23:21:51 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|