Clean shutdown of ReliableProxySpec, see #2846

* Solved by adding missing Dequeue in throttler
* Changed to FSM timers in throttler
* Some boy scouting
This commit is contained in:
Patrik Nordwall 2013-01-03 14:51:41 +01:00
parent 3ceb02f846
commit 48c6374721
7 changed files with 105 additions and 103 deletions

View file

@ -37,8 +37,9 @@ class ReliableProxySpec extends MultiNodeSpec(ReliableProxySpec) with STMultiNod
override def afterEach { override def afterEach {
runOn(local) { runOn(local) {
testConductor.throttle(local, remote, Direction.Both, -1).await testConductor.passThrough(local, remote, Direction.Both).await
} }
enterBarrier("after-each")
} }
@volatile var target: ActorRef = system.deadLetters @volatile var target: ActorRef = system.deadLetters
@ -82,6 +83,8 @@ class ReliableProxySpec extends MultiNodeSpec(ReliableProxySpec) with STMultiNod
runOn(remote) { runOn(remote) {
expectMsg("hello") expectMsg("hello")
} }
enterBarrier("initialize-done")
} }
"forward messages in sequence" in { "forward messages in sequence" in {
@ -131,7 +134,7 @@ class ReliableProxySpec extends MultiNodeSpec(ReliableProxySpec) with STMultiNod
enterBarrier("test2b") enterBarrier("test2b")
runOn(local) { runOn(local) {
testConductor.throttle(local, remote, Direction.Send, -1).await testConductor.passThrough(local, remote, Direction.Send).await
within(5 seconds) { expectTransition(Active, Idle) } within(5 seconds) { expectTransition(Active, Idle) }
} }
runOn(remote) { runOn(remote) {
@ -161,7 +164,7 @@ class ReliableProxySpec extends MultiNodeSpec(ReliableProxySpec) with STMultiNod
enterBarrier("test3a") enterBarrier("test3a")
runOn(local) { runOn(local) {
testConductor.throttle(local, remote, Direction.Receive, -1).await testConductor.passThrough(local, remote, Direction.Receive).await
within(5 seconds) { expectTransition(Active, Idle) } within(5 seconds) { expectTransition(Active, Idle) }
} }
@ -186,7 +189,7 @@ class ReliableProxySpec extends MultiNodeSpec(ReliableProxySpec) with STMultiNod
enterBarrier("test4a") enterBarrier("test4a")
runOn(local) { runOn(local) {
testConductor.throttle(local, remote, Direction.Send, rateMBit = -1).await testConductor.passThrough(local, remote, Direction.Send).await
testConductor.throttle(local, remote, Direction.Receive, rateMBit = 0.1).await testConductor.throttle(local, remote, Direction.Receive, rateMBit = 0.1).await
sendN(50) sendN(50)
within(5 seconds) { within(5 seconds) {
@ -200,7 +203,7 @@ class ReliableProxySpec extends MultiNodeSpec(ReliableProxySpec) with STMultiNod
} }
} }
enterBarrier("test4a") enterBarrier("test4b")
} }
} }

View file

@ -207,8 +207,8 @@ surprising ways.
* Don't issue a shutdown of the first node. The first node is the controller and if it shuts down your test will break. * Don't issue a shutdown of the first node. The first node is the controller and if it shuts down your test will break.
* To be able to use ``blackhole``, ``passThrough``, and ``throttle`` you must activate the ``TestConductorTranport`` * To be able to use ``blackhole``, ``passThrough``, and ``throttle`` you must activate the failure injector and
by specifying ``testTransport(on = true)`` in your MultiNodeConfig. throttler transport adapters by specifying ``testTransport(on = true)`` in your MultiNodeConfig.
* Throttling, shutdown and other failure injections can only be done from the first node, which again is the controller. * Throttling, shutdown and other failure injections can only be done from the first node, which again is the controller.

View file

@ -94,8 +94,8 @@ trait Conductor { this: TestConductorExt ⇒
* increased latency. * increased latency.
* *
* ====Note==== * ====Note====
* To use this feature you must activate the `TestConductorTranport` * To use this feature you must activate the failure injector and throttler
* by specifying `testTransport(on = true)` in your MultiNodeConfig. * transport adapters by specifying `testTransport(on = true)` in your MultiNodeConfig.
* *
* @param node is the symbolic name of the node which is to be affected * @param node is the symbolic name of the node which is to be affected
* @param target is the symbolic name of the other node to which connectivity shall be throttled * @param target is the symbolic name of the other node to which connectivity shall be throttled
@ -115,18 +115,15 @@ trait Conductor { this: TestConductorExt ⇒
* Socket. * Socket.
* *
* ====Note==== * ====Note====
* To use this feature you must activate the `TestConductorTranport` * To use this feature you must activate the failure injector and throttler
* by specifying `testTransport(on = true)` in your MultiNodeConfig. * transport adapters by specifying `testTransport(on = true)` in your MultiNodeConfig.
* *
* @param node is the symbolic name of the node which is to be affected * @param node is the symbolic name of the node which is to be affected
* @param target is the symbolic name of the other node to which connectivity shall be impeded * @param target is the symbolic name of the other node to which connectivity shall be impeded
* @param direction can be either `Direction.Send`, `Direction.Receive` or `Direction.Both` * @param direction can be either `Direction.Send`, `Direction.Receive` or `Direction.Both`
*/ */
def blackhole(node: RoleName, target: RoleName, direction: Direction): Future[Done] = { def blackhole(node: RoleName, target: RoleName, direction: Direction): Future[Done] =
import Settings.QueryTimeout throttle(node, target, direction, 0f)
requireTestConductorTranport()
controller ? Throttle(node, target, direction, 0f) mapTo classTag[Done]
}
private def requireTestConductorTranport(): Unit = if (!transport.defaultAddress.protocol.contains(".gremlin.trttl.")) private def requireTestConductorTranport(): Unit = if (!transport.defaultAddress.protocol.contains(".gremlin.trttl."))
throw new ConfigurationException("To use this feature you must activate the failure injector adapters " + throw new ConfigurationException("To use this feature you must activate the failure injector adapters " +
@ -137,18 +134,15 @@ trait Conductor { this: TestConductorExt ⇒
* sending and/or receiving. * sending and/or receiving.
* *
* ====Note==== * ====Note====
* To use this feature you must activate the `TestConductorTranport` * To use this feature you must activate the failure injector and throttler
* by specifying `testTransport(on = true)` in your MultiNodeConfig. * transport adapters by specifying `testTransport(on = true)` in your MultiNodeConfig.
* *
* @param node is the symbolic name of the node which is to be affected * @param node is the symbolic name of the node which is to be affected
* @param target is the symbolic name of the other node to which connectivity shall be impeded * @param target is the symbolic name of the other node to which connectivity shall be impeded
* @param direction can be either `Direction.Send`, `Direction.Receive` or `Direction.Both` * @param direction can be either `Direction.Send`, `Direction.Receive` or `Direction.Both`
*/ */
def passThrough(node: RoleName, target: RoleName, direction: Direction): Future[Done] = { def passThrough(node: RoleName, target: RoleName, direction: Direction): Future[Done] =
import Settings.QueryTimeout throttle(node, target, direction, -1f)
requireTestConductorTranport()
controller ? Throttle(node, target, direction, -1f) mapTo classTag[Done]
}
/** /**
* Tell the remote support to shutdown the connection to the given remote * Tell the remote support to shutdown the connection to the given remote

View file

@ -36,8 +36,8 @@ object TestConductor extends ExtensionKey[TestConductorExt] {
* to be a [[akka.remote.RemoteActorRefProvider]]. * to be a [[akka.remote.RemoteActorRefProvider]].
* *
* To use ``blackhole``, ``passThrough``, and ``throttle`` you must activate the * To use ``blackhole``, ``passThrough``, and ``throttle`` you must activate the
* `TestConductorTranport` by specifying `testTransport(on = true)` in your * failure injector and throttler transport adapters by specifying `testTransport(on = true)`
* MultiNodeConfig. * in your MultiNodeConfig.
* *
*/ */
class TestConductorExt(val system: ExtendedActorSystem) extends Extension with Conductor with Player { class TestConductorExt(val system: ExtendedActorSystem) extends Extension with Conductor with Player {

View file

@ -87,8 +87,8 @@ abstract class MultiNodeConfig {
/** /**
* To be able to use `blackhole`, `passThrough`, and `throttle` you must * To be able to use `blackhole`, `passThrough`, and `throttle` you must
* activate the TestConductorTranport by specifying * activate the failure injector and throttler transport adapters by
* `testTransport(on = true)` in your MultiNodeConfig. * specifying `testTransport(on = true)` in your MultiNodeConfig.
*/ */
def testTransport(on: Boolean): Unit = _testTransport = on def testTransport(on: Boolean): Unit = _testTransport = on

View file

@ -59,43 +59,42 @@ object ThrottlerTransportAdapter {
case class SetThrottle(address: Address, direction: Direction, mode: ThrottleMode) case class SetThrottle(address: Address, direction: Direction, mode: ThrottleMode)
sealed trait ThrottleMode { sealed trait ThrottleMode {
def tryConsumeTokens(timeOfSend: Long, tokens: Int): (ThrottleMode, Boolean) def tryConsumeTokens(nanoTimeOfSend: Long, tokens: Int): (ThrottleMode, Boolean)
def timeToAvailable(currentTime: Long, tokens: Int): Long def timeToAvailable(currentNanoTime: Long, tokens: Int): FiniteDuration
} }
case class TokenBucket(capacity: Int, tokensPerSecond: Double, lastSend: Long, availableTokens: Int) case class TokenBucket(capacity: Int, tokensPerSecond: Double, nanoTimeOfLastSend: Long, availableTokens: Int)
extends ThrottleMode { extends ThrottleMode {
private def isAvailable(timeOfSend: Long, tokens: Int): Boolean = if ((tokens > capacity && availableTokens > 0)) { private def isAvailable(nanoTimeOfSend: Long, tokens: Int): Boolean = if ((tokens > capacity && availableTokens > 0)) {
true // Allow messages larger than capacity through, it will be recorded as negative tokens true // Allow messages larger than capacity through, it will be recorded as negative tokens
} else min((availableTokens + tokensGenerated(timeOfSend)), capacity) >= tokens } else min((availableTokens + tokensGenerated(nanoTimeOfSend)), capacity) >= tokens
override def tryConsumeTokens(timeOfSend: Long, tokens: Int): (ThrottleMode, Boolean) = { override def tryConsumeTokens(nanoTimeOfSend: Long, tokens: Int): (ThrottleMode, Boolean) = {
if (isAvailable(timeOfSend, tokens)) if (isAvailable(nanoTimeOfSend, tokens))
(this.copy( (this.copy(
lastSend = timeOfSend, nanoTimeOfLastSend = nanoTimeOfSend,
availableTokens = min(availableTokens - tokens + tokensGenerated(timeOfSend), capacity)), true) availableTokens = min(availableTokens - tokens + tokensGenerated(nanoTimeOfSend), capacity)), true)
else (this, false) else (this, false)
} }
override def timeToAvailable(currentTime: Long, tokens: Int): Long = { override def timeToAvailable(currentNanoTime: Long, tokens: Int): FiniteDuration = {
val needed = (if (tokens > capacity) 1 else tokens) - tokensGenerated(currentTime) val needed = (if (tokens > capacity) 1 else tokens) - tokensGenerated(currentNanoTime)
TimeUnit.SECONDS.toNanos((needed / tokensPerSecond).toLong) (needed / tokensPerSecond).seconds
} }
private def tokensGenerated(timeOfSend: Long): Int = private def tokensGenerated(nanoTimeOfSend: Long): Int =
(TimeUnit.NANOSECONDS.toMillis(timeOfSend - lastSend) * tokensPerSecond / 1000.0).toInt (TimeUnit.NANOSECONDS.toMillis(nanoTimeOfSend - nanoTimeOfLastSend) * tokensPerSecond / 1000.0).toInt
} }
case object Unthrottled extends ThrottleMode { case object Unthrottled extends ThrottleMode {
override def tryConsumeTokens(nanoTimeOfSend: Long, tokens: Int): (ThrottleMode, Boolean) = (this, true)
override def tryConsumeTokens(timeOfSend: Long, tokens: Int): (ThrottleMode, Boolean) = (this, true) override def timeToAvailable(currentNanoTime: Long, tokens: Int): FiniteDuration = Duration.Zero
override def timeToAvailable(currentTime: Long, tokens: Int): Long = 1L
} }
case object Blackhole extends ThrottleMode { case object Blackhole extends ThrottleMode {
override def tryConsumeTokens(timeOfSend: Long, tokens: Int): (ThrottleMode, Boolean) = (this, false) override def tryConsumeTokens(nanoTimeOfSend: Long, tokens: Int): (ThrottleMode, Boolean) = (this, false)
override def timeToAvailable(currentTime: Long, tokens: Int): Long = 0L override def timeToAvailable(currentNanoTime: Long, tokens: Int): FiniteDuration = Duration.Zero
} }
} }
@ -215,6 +214,8 @@ private[transport] class ThrottlerManager(wrappedTransport: Transport) extends A
} }
object ThrottledAssociation { object ThrottledAssociation {
private final val DequeueTimerName = "dequeue"
case object Dequeue case object Dequeue
sealed trait ThrottlerState sealed trait ThrottlerState
@ -253,7 +254,7 @@ private[transport] class ThrottledAssociation(
import context.dispatcher import context.dispatcher
var inboundThrottleMode: ThrottleMode = _ var inboundThrottleMode: ThrottleMode = _
var queue = Queue.empty[ByteString] var throttledMessages = Queue.empty[ByteString]
var upstreamListener: HandleEventListener = _ var upstreamListener: HandleEventListener = _
override def postStop(): Unit = originalHandle.disassociate() override def postStop(): Unit = originalHandle.disassociate()
@ -272,7 +273,7 @@ private[transport] class ThrottledAssociation(
when(WaitOrigin) { when(WaitOrigin) {
case Event(InboundPayload(p), ExposedHandle(exposedHandle)) case Event(InboundPayload(p), ExposedHandle(exposedHandle))
queue = queue enqueue p throttledMessages = throttledMessages enqueue p
peekOrigin(p) match { peekOrigin(p) match {
case Some(origin) case Some(origin)
manager ! Checkin(origin, exposedHandle) manager ! Checkin(origin, exposedHandle)
@ -283,12 +284,12 @@ private[transport] class ThrottledAssociation(
when(WaitMode) { when(WaitMode) {
case Event(InboundPayload(p), _) case Event(InboundPayload(p), _)
queue = queue enqueue p throttledMessages = throttledMessages enqueue p
stay() stay()
case Event(mode: ThrottleMode, ExposedHandle(exposedHandle)) case Event(mode: ThrottleMode, ExposedHandle(exposedHandle))
inboundThrottleMode = mode inboundThrottleMode = mode
if (inboundThrottleMode == Blackhole) { if (inboundThrottleMode == Blackhole) {
queue = Queue.empty[ByteString] throttledMessages = Queue.empty[ByteString]
exposedHandle.disassociate() exposedHandle.disassociate()
stop() stop()
} else { } else {
@ -300,7 +301,7 @@ private[transport] class ThrottledAssociation(
when(WaitUpstreamListener) { when(WaitUpstreamListener) {
case Event(InboundPayload(p), _) case Event(InboundPayload(p), _)
queue = queue enqueue p throttledMessages = throttledMessages enqueue p
stay() stay()
case Event(listener: HandleEventListener, _) case Event(listener: HandleEventListener, _)
upstreamListener = listener upstreamListener = listener
@ -315,30 +316,30 @@ private[transport] class ThrottledAssociation(
self ! Dequeue self ! Dequeue
goto(Throttling) goto(Throttling)
case Event(InboundPayload(p), _) case Event(InboundPayload(p), _)
queue = queue enqueue p throttledMessages = throttledMessages enqueue p
stay() stay()
} }
when(Throttling) { when(Throttling) {
case Event(mode: ThrottleMode, _) case Event(mode: ThrottleMode, _)
inboundThrottleMode = mode inboundThrottleMode = mode
if (inboundThrottleMode == Blackhole) queue = Queue.empty[ByteString] if (inboundThrottleMode == Blackhole) throttledMessages = Queue.empty[ByteString]
cancelTimer(DequeueTimerName)
if (throttledMessages.nonEmpty)
scheduleDequeue(inboundThrottleMode.timeToAvailable(System.nanoTime(), throttledMessages.head.length))
stay() stay()
case Event(InboundPayload(p), _) case Event(InboundPayload(p), _)
forwardOrDelay(p) forwardOrDelay(p)
stay() stay()
case Event(Dequeue, _) case Event(Dequeue, _)
if (!queue.isEmpty) { if (throttledMessages.nonEmpty) {
val (payload, newqueue) = queue.dequeue val (payload, newqueue) = throttledMessages.dequeue
upstreamListener notify InboundPayload(payload) upstreamListener notify InboundPayload(payload)
queue = newqueue throttledMessages = newqueue
inboundThrottleMode = inboundThrottleMode.tryConsumeTokens(System.nanoTime(), payload.length)._1 inboundThrottleMode = inboundThrottleMode.tryConsumeTokens(System.nanoTime(), payload.length)._1
if (inboundThrottleMode == Unthrottled && !queue.isEmpty) self ! Dequeue if (throttledMessages.nonEmpty)
else if (!queue.isEmpty) { scheduleDequeue(inboundThrottleMode.timeToAvailable(System.nanoTime(), throttledMessages.head.length))
context.system.scheduler.scheduleOnce(
inboundThrottleMode.timeToAvailable(System.nanoTime(), queue.head.length).nanos, self, Dequeue)
}
} }
stay() stay()
@ -370,24 +371,28 @@ private[transport] class ThrottledAssociation(
if (inboundThrottleMode == Blackhole) { if (inboundThrottleMode == Blackhole) {
// Do nothing // Do nothing
} else { } else {
if (queue.isEmpty) { if (throttledMessages.isEmpty) {
val tokens = payload.length val tokens = payload.length
val (newbucket, success) = inboundThrottleMode.tryConsumeTokens(System.nanoTime(), tokens) val (newbucket, success) = inboundThrottleMode.tryConsumeTokens(System.nanoTime(), tokens)
if (success) { if (success) {
inboundThrottleMode = newbucket inboundThrottleMode = newbucket
upstreamListener notify InboundPayload(payload) upstreamListener notify InboundPayload(payload)
} else { } else {
queue = queue.enqueue(payload) throttledMessages = throttledMessages.enqueue(payload)
scheduleDequeue(inboundThrottleMode.timeToAvailable(System.nanoTime(), tokens))
context.system.scheduler.scheduleOnce(
inboundThrottleMode.timeToAvailable(System.nanoTime(), tokens).nanos, self, Dequeue)
} }
} else { } else {
queue = queue.enqueue(payload) throttledMessages = throttledMessages.enqueue(payload)
} }
} }
} }
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)
}
} }
private[transport] case class ThrottlerHandle(_wrappedHandle: AssociationHandle, throttlerActor: ActorRef) private[transport] case class ThrottlerHandle(_wrappedHandle: AssociationHandle, throttlerActor: ActorRef)

View file

@ -18,77 +18,77 @@ class ThrottleModeSpec extends AkkaSpec {
} }
"in tokenbucket mode allow consuming tokens up to capacity" in { "in tokenbucket mode allow consuming tokens up to capacity" in {
val bucket = TokenBucket(capacity = 100, tokensPerSecond = 100, lastSend = 0L, availableTokens = 100) val bucket = TokenBucket(capacity = 100, tokensPerSecond = 100, nanoTimeOfLastSend = 0L, availableTokens = 100)
val (bucket1, success1) = bucket.tryConsumeTokens(timeOfSend = 0L, 10) val (bucket1, success1) = bucket.tryConsumeTokens(nanoTimeOfSend = 0L, 10)
bucket1 must be(TokenBucket(100, 100, 0, 90)) bucket1 must be(TokenBucket(100, 100, 0, 90))
success1 must be(true) success1 must be(true)
val (bucket2, success2) = bucket1.tryConsumeTokens(timeOfSend = 0L, 40) val (bucket2, success2) = bucket1.tryConsumeTokens(nanoTimeOfSend = 0L, 40)
bucket2 must be(TokenBucket(100, 100, 0, 50)) bucket2 must be(TokenBucket(100, 100, 0, 50))
success2 must be(true) success2 must be(true)
val (bucket3, success3) = bucket2.tryConsumeTokens(timeOfSend = 0L, 50) val (bucket3, success3) = bucket2.tryConsumeTokens(nanoTimeOfSend = 0L, 50)
bucket3 must be(TokenBucket(100, 100, 0, 0)) bucket3 must be(TokenBucket(100, 100, 0, 0))
success3 must be(true) success3 must be(true)
val (bucket4, success4) = bucket3.tryConsumeTokens(timeOfSend = 0, 1) val (bucket4, success4) = bucket3.tryConsumeTokens(nanoTimeOfSend = 0, 1)
bucket4 must be(TokenBucket(100, 100, 0, 0)) bucket4 must be(TokenBucket(100, 100, 0, 0))
success4 must be(false) success4 must be(false)
} }
"accurately replenish tokens" in { "accurately replenish tokens" in {
val bucket = TokenBucket(capacity = 100, tokensPerSecond = 100, lastSend = 0L, availableTokens = 0) val bucket = TokenBucket(capacity = 100, tokensPerSecond = 100, nanoTimeOfLastSend = 0L, availableTokens = 0)
val (bucket1, success1) = bucket.tryConsumeTokens(timeOfSend = 0L, 0) val (bucket1, success1) = bucket.tryConsumeTokens(nanoTimeOfSend = 0L, 0)
bucket1 must be(TokenBucket(100, 100, 0, 0)) bucket1 must be(TokenBucket(100, 100, 0, 0))
success1 must be(true) success1 must be(true)
val (bucket2, success2) = bucket1.tryConsumeTokens(timeOfSend = halfSecond, 0) val (bucket2, success2) = bucket1.tryConsumeTokens(nanoTimeOfSend = halfSecond, 0)
bucket2 must be(TokenBucket(100, 100, halfSecond, 50)) bucket2 must be(TokenBucket(100, 100, halfSecond, 50))
success2 must be(true) success2 must be(true)
val (bucket3, success3) = bucket2.tryConsumeTokens(timeOfSend = 2 * halfSecond, 0) val (bucket3, success3) = bucket2.tryConsumeTokens(nanoTimeOfSend = 2 * halfSecond, 0)
bucket3 must be(TokenBucket(100, 100, 2 * halfSecond, 100)) bucket3 must be(TokenBucket(100, 100, 2 * halfSecond, 100))
success3 must be(true) success3 must be(true)
val (bucket4, success4) = bucket3.tryConsumeTokens(timeOfSend = 3 * halfSecond, 0) val (bucket4, success4) = bucket3.tryConsumeTokens(nanoTimeOfSend = 3 * halfSecond, 0)
bucket4 must be(TokenBucket(100, 100, 3 * halfSecond, 100)) bucket4 must be(TokenBucket(100, 100, 3 * halfSecond, 100))
success4 must be(true) success4 must be(true)
} }
"accurately interleave replenish and consume" in { "accurately interleave replenish and consume" in {
val bucket = TokenBucket(capacity = 100, tokensPerSecond = 100, lastSend = 0L, availableTokens = 20) val bucket = TokenBucket(capacity = 100, tokensPerSecond = 100, nanoTimeOfLastSend = 0L, availableTokens = 20)
val (bucket1, success1) = bucket.tryConsumeTokens(timeOfSend = 0L, 10) val (bucket1, success1) = bucket.tryConsumeTokens(nanoTimeOfSend = 0L, 10)
bucket1 must be(TokenBucket(100, 100, 0, 10)) bucket1 must be(TokenBucket(100, 100, 0, 10))
success1 must be(true) success1 must be(true)
val (bucket2, success2) = bucket1.tryConsumeTokens(timeOfSend = halfSecond, 60) val (bucket2, success2) = bucket1.tryConsumeTokens(nanoTimeOfSend = halfSecond, 60)
bucket2 must be(TokenBucket(100, 100, halfSecond, 0)) bucket2 must be(TokenBucket(100, 100, halfSecond, 0))
success2 must be(true) success2 must be(true)
val (bucket3, success3) = bucket2.tryConsumeTokens(timeOfSend = 2 * halfSecond, 40) val (bucket3, success3) = bucket2.tryConsumeTokens(nanoTimeOfSend = 2 * halfSecond, 40)
bucket3 must be(TokenBucket(100, 100, 2 * halfSecond, 10)) bucket3 must be(TokenBucket(100, 100, 2 * halfSecond, 10))
success3 must be(true) success3 must be(true)
val (bucket4, success4) = bucket3.tryConsumeTokens(timeOfSend = 3 * halfSecond, 70) val (bucket4, success4) = bucket3.tryConsumeTokens(nanoTimeOfSend = 3 * halfSecond, 70)
bucket4 must be(TokenBucket(100, 100, 2 * halfSecond, 10)) bucket4 must be(TokenBucket(100, 100, 2 * halfSecond, 10))
success4 must be(false) success4 must be(false)
} }
"allow oversized packets through by loaning" in { "allow oversized packets through by loaning" in {
val bucket = TokenBucket(capacity = 100, tokensPerSecond = 100, lastSend = 0L, availableTokens = 20) val bucket = TokenBucket(capacity = 100, tokensPerSecond = 100, nanoTimeOfLastSend = 0L, availableTokens = 20)
val (bucket1, success1) = bucket.tryConsumeTokens(timeOfSend = 0L, 30) val (bucket1, success1) = bucket.tryConsumeTokens(nanoTimeOfSend = 0L, 30)
bucket1 must be(TokenBucket(100, 100, 0, 20)) bucket1 must be(TokenBucket(100, 100, 0, 20))
success1 must be(false) success1 must be(false)
val (bucket2, success2) = bucket1.tryConsumeTokens(timeOfSend = halfSecond, 110) val (bucket2, success2) = bucket1.tryConsumeTokens(nanoTimeOfSend = halfSecond, 110)
bucket2 must be(TokenBucket(100, 100, halfSecond, -40)) bucket2 must be(TokenBucket(100, 100, halfSecond, -40))
success2 must be(true) success2 must be(true)
val (bucket3, success3) = bucket2.tryConsumeTokens(timeOfSend = 2 * halfSecond, 20) val (bucket3, success3) = bucket2.tryConsumeTokens(nanoTimeOfSend = 2 * halfSecond, 20)
bucket3 must be(TokenBucket(100, 100, halfSecond, -40)) bucket3 must be(TokenBucket(100, 100, halfSecond, -40))
success3 must be(false) success3 must be(false)
val (bucket4, success4) = bucket3.tryConsumeTokens(timeOfSend = 3 * halfSecond, 20) val (bucket4, success4) = bucket3.tryConsumeTokens(nanoTimeOfSend = 3 * halfSecond, 20)
bucket4 must be(TokenBucket(100, 100, 3 * halfSecond, 40)) bucket4 must be(TokenBucket(100, 100, 3 * halfSecond, 40))
success4 must be(true) success4 must be(true)
} }