Merge pull request #353 from jboner/wip-1836-durMB-settings-∂π
switch to dispatcher-scoped settings for durable mailboxes, see #1836
This commit is contained in:
commit
762c60d486
36 changed files with 377 additions and 196 deletions
5
.gitignore
vendored
5
.gitignore
vendored
|
|
@ -62,4 +62,7 @@ akka.sublime-workspace
|
||||||
.multi-jvm
|
.multi-jvm
|
||||||
_mb
|
_mb
|
||||||
schoir.props
|
schoir.props
|
||||||
worker*.log
|
worker*.log
|
||||||
|
mongoDB/
|
||||||
|
redis/
|
||||||
|
beanstalk/
|
||||||
|
|
|
||||||
|
|
@ -8,6 +8,7 @@ import akka.util.duration._
|
||||||
import akka.testkit.AkkaSpec
|
import akka.testkit.AkkaSpec
|
||||||
import akka.actor.{ ActorRef, ActorContext, Props, LocalActorRef }
|
import akka.actor.{ ActorRef, ActorContext, Props, LocalActorRef }
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAndAfterEach {
|
abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAndAfterEach {
|
||||||
|
|
@ -156,7 +157,7 @@ object CustomMailboxSpec {
|
||||||
}
|
}
|
||||||
"""
|
"""
|
||||||
|
|
||||||
class MyMailboxType(config: Config) extends MailboxType {
|
class MyMailboxType(settings: ActorSystem.Settings, config: Config) extends MailboxType {
|
||||||
override def create(owner: Option[ActorContext]) = owner match {
|
override def create(owner: Option[ActorContext]) = owner match {
|
||||||
case Some(o) ⇒ new MyMailbox(o)
|
case Some(o) ⇒ new MyMailbox(o)
|
||||||
case None ⇒ throw new Exception("no mailbox owner given")
|
case None ⇒ throw new Exception("no mailbox owner given")
|
||||||
|
|
|
||||||
|
|
@ -6,6 +6,7 @@ import akka.pattern.ask
|
||||||
import akka.util.duration._
|
import akka.util.duration._
|
||||||
import akka.testkit.DefaultTimeout
|
import akka.testkit.DefaultTimeout
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
|
||||||
object PriorityDispatcherSpec {
|
object PriorityDispatcherSpec {
|
||||||
val config = """
|
val config = """
|
||||||
|
|
@ -17,12 +18,12 @@ object PriorityDispatcherSpec {
|
||||||
}
|
}
|
||||||
"""
|
"""
|
||||||
|
|
||||||
class Unbounded(config: Config) extends UnboundedPriorityMailbox(PriorityGenerator({
|
class Unbounded(settings: ActorSystem.Settings, config: Config) extends UnboundedPriorityMailbox(PriorityGenerator({
|
||||||
case i: Int ⇒ i //Reverse order
|
case i: Int ⇒ i //Reverse order
|
||||||
case 'Result ⇒ Int.MaxValue
|
case 'Result ⇒ Int.MaxValue
|
||||||
}: Any ⇒ Int))
|
}: Any ⇒ Int))
|
||||||
|
|
||||||
class Bounded(config: Config) extends BoundedPriorityMailbox(PriorityGenerator({
|
class Bounded(settings: ActorSystem.Settings, config: Config) extends BoundedPriorityMailbox(PriorityGenerator({
|
||||||
case i: Int ⇒ i //Reverse order
|
case i: Int ⇒ i //Reverse order
|
||||||
case 'Result ⇒ Int.MaxValue
|
case 'Result ⇒ Int.MaxValue
|
||||||
}: Any ⇒ Int), 1000, 10 seconds)
|
}: Any ⇒ Int), 1000, 10 seconds)
|
||||||
|
|
|
||||||
|
|
@ -242,8 +242,8 @@ akka {
|
||||||
mailbox-push-timeout-time = 10s
|
mailbox-push-timeout-time = 10s
|
||||||
|
|
||||||
# FQCN of the MailboxType, if not specified the default bounded or unbounded
|
# FQCN of the MailboxType, if not specified the default bounded or unbounded
|
||||||
# mailbox is used. The Class of the FQCN must have a constructor with a
|
# mailbox is used. The Class of the FQCN must have a constructor with
|
||||||
# com.typesafe.config.Config parameter.
|
# (akka.actor.ActorSystem.Settings, com.typesafe.config.Config) parameters.
|
||||||
mailbox-type = ""
|
mailbox-type = ""
|
||||||
|
|
||||||
# For BalancingDispatcher: If the balancing dispatcher should attempt to
|
# For BalancingDispatcher: If the balancing dispatcher should attempt to
|
||||||
|
|
@ -254,7 +254,8 @@ akka {
|
||||||
|
|
||||||
debug {
|
debug {
|
||||||
# enable function of Actor.loggable(), which is to log any received message at
|
# enable function of Actor.loggable(), which is to log any received message at
|
||||||
# DEBUG level
|
# DEBUG level, see the “Testing Actor Systems” section of the Akka Documentation
|
||||||
|
# at http://akka.io/docs
|
||||||
receive = off
|
receive = off
|
||||||
|
|
||||||
# enable DEBUG logging of all AutoReceiveMessages (Kill, PoisonPill and the like)
|
# enable DEBUG logging of all AutoReceiveMessages (Kill, PoisonPill and the like)
|
||||||
|
|
|
||||||
|
|
@ -481,7 +481,7 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf
|
||||||
def locker: Locker = provider.locker
|
def locker: Locker = provider.locker
|
||||||
|
|
||||||
val dispatchers: Dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(
|
val dispatchers: Dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(
|
||||||
threadFactory, eventStream, deadLetterMailbox, scheduler, dynamicAccess))
|
threadFactory, eventStream, deadLetterMailbox, scheduler, dynamicAccess, settings))
|
||||||
|
|
||||||
val dispatcher: MessageDispatcher = dispatchers.defaultGlobalDispatcher
|
val dispatcher: MessageDispatcher = dispatchers.defaultGlobalDispatcher
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -384,17 +384,17 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
|
||||||
config.getString("mailbox-type") match {
|
config.getString("mailbox-type") match {
|
||||||
case "" ⇒
|
case "" ⇒
|
||||||
if (config.getInt("mailbox-capacity") < 1) UnboundedMailbox()
|
if (config.getInt("mailbox-capacity") < 1) UnboundedMailbox()
|
||||||
else new BoundedMailbox(config)
|
else new BoundedMailbox(prerequisites.settings, config)
|
||||||
case "unbounded" ⇒ UnboundedMailbox()
|
case "unbounded" ⇒ UnboundedMailbox()
|
||||||
case "bounded" ⇒ new BoundedMailbox(config)
|
case "bounded" ⇒ new BoundedMailbox(prerequisites.settings, config)
|
||||||
case fqcn ⇒
|
case fqcn ⇒
|
||||||
val args = Seq(classOf[Config] -> config)
|
val args = Seq(classOf[ActorSystem.Settings] -> prerequisites.settings, classOf[Config] -> config)
|
||||||
prerequisites.dynamicAccess.createInstanceFor[MailboxType](fqcn, args) match {
|
prerequisites.dynamicAccess.createInstanceFor[MailboxType](fqcn, args) match {
|
||||||
case Right(instance) ⇒ instance
|
case Right(instance) ⇒ instance
|
||||||
case Left(exception) ⇒
|
case Left(exception) ⇒
|
||||||
throw new IllegalArgumentException(
|
throw new IllegalArgumentException(
|
||||||
("Cannot instantiate MailboxType [%s], defined in [%s], " +
|
("Cannot instantiate MailboxType [%s], defined in [%s], " +
|
||||||
"make sure it has constructor with a [com.typesafe.config.Config] parameter")
|
"make sure it has constructor with [akka.actor.ActorSystem.Settings, com.typesafe.config.Config] parameters")
|
||||||
.format(fqcn, config.getString("id")), exception)
|
.format(fqcn, config.getString("id")), exception)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -22,6 +22,7 @@ trait DispatcherPrerequisites {
|
||||||
def deadLetterMailbox: Mailbox
|
def deadLetterMailbox: Mailbox
|
||||||
def scheduler: Scheduler
|
def scheduler: Scheduler
|
||||||
def dynamicAccess: DynamicAccess
|
def dynamicAccess: DynamicAccess
|
||||||
|
def settings: ActorSystem.Settings
|
||||||
}
|
}
|
||||||
|
|
||||||
case class DefaultDispatcherPrerequisites(
|
case class DefaultDispatcherPrerequisites(
|
||||||
|
|
@ -29,7 +30,8 @@ case class DefaultDispatcherPrerequisites(
|
||||||
val eventStream: EventStream,
|
val eventStream: EventStream,
|
||||||
val deadLetterMailbox: Mailbox,
|
val deadLetterMailbox: Mailbox,
|
||||||
val scheduler: Scheduler,
|
val scheduler: Scheduler,
|
||||||
val dynamicAccess: DynamicAccess) extends DispatcherPrerequisites
|
val dynamicAccess: DynamicAccess,
|
||||||
|
val settings: ActorSystem.Settings) extends DispatcherPrerequisites
|
||||||
|
|
||||||
object Dispatchers {
|
object Dispatchers {
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -12,6 +12,7 @@ import annotation.tailrec
|
||||||
import akka.event.Logging.Error
|
import akka.event.Logging.Error
|
||||||
import akka.actor.ActorContext
|
import akka.actor.ActorContext
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
|
||||||
class MessageQueueAppendFailedException(message: String, cause: Throwable = null) extends AkkaException(message, cause)
|
class MessageQueueAppendFailedException(message: String, cause: Throwable = null) extends AkkaException(message, cause)
|
||||||
|
|
||||||
|
|
@ -357,7 +358,7 @@ trait MailboxType {
|
||||||
*/
|
*/
|
||||||
case class UnboundedMailbox() extends MailboxType {
|
case class UnboundedMailbox() extends MailboxType {
|
||||||
|
|
||||||
def this(config: Config) = this()
|
def this(settings: ActorSystem.Settings, config: Config) = this()
|
||||||
|
|
||||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
final override def create(owner: Option[ActorContext]): MessageQueue =
|
||||||
new ConcurrentLinkedQueue[Envelope]() with QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
|
new ConcurrentLinkedQueue[Envelope]() with QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
|
||||||
|
|
@ -367,7 +368,7 @@ case class UnboundedMailbox() extends MailboxType {
|
||||||
|
|
||||||
case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Duration) extends MailboxType {
|
case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Duration) extends MailboxType {
|
||||||
|
|
||||||
def this(config: Config) = this(config.getInt("mailbox-capacity"),
|
def this(settings: ActorSystem.Settings, config: Config) = this(config.getInt("mailbox-capacity"),
|
||||||
Duration(config.getNanoseconds("mailbox-push-timeout-time"), TimeUnit.NANOSECONDS))
|
Duration(config.getNanoseconds("mailbox-push-timeout-time"), TimeUnit.NANOSECONDS))
|
||||||
|
|
||||||
if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative")
|
if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative")
|
||||||
|
|
|
||||||
|
|
@ -124,7 +124,7 @@ public class DispatcherDocTestBase {
|
||||||
|
|
||||||
//#prio-mailbox
|
//#prio-mailbox
|
||||||
public static class MyPrioMailbox extends UnboundedPriorityMailbox {
|
public static class MyPrioMailbox extends UnboundedPriorityMailbox {
|
||||||
public MyPrioMailbox(Config config) { // needed for reflective instantiation
|
public MyPrioMailbox(ActorSystem.Settings settings, Config config) { // needed for reflective instantiation
|
||||||
// Create a new PriorityGenerator, lower prio means more important
|
// Create a new PriorityGenerator, lower prio means more important
|
||||||
super(new PriorityGenerator() {
|
super(new PriorityGenerator() {
|
||||||
@Override
|
@Override
|
||||||
|
|
|
||||||
|
|
@ -152,7 +152,7 @@ Mailbox configuration examples
|
||||||
|
|
||||||
How to create a PriorityMailbox:
|
How to create a PriorityMailbox:
|
||||||
|
|
||||||
.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherTestBase.java#prio-mailbox
|
.. includecode:: ../java/code/akka/docs/dispatcher/DispatcherDocTestBase.java#prio-mailbox
|
||||||
|
|
||||||
And then add it to the configuration:
|
And then add it to the configuration:
|
||||||
|
|
||||||
|
|
@ -160,4 +160,14 @@ And then add it to the configuration:
|
||||||
|
|
||||||
And then an example on how you would use it:
|
And then an example on how you would use it:
|
||||||
|
|
||||||
.. includecode:: ../java/code/akka/docs/dispatcher/DispatcherDocTestBase.java#prio-dispatcher
|
.. includecode:: ../java/code/akka/docs/dispatcher/DispatcherDocTestBase.java#prio-dispatcher
|
||||||
|
|
||||||
|
.. note::
|
||||||
|
|
||||||
|
Make sure to include a constructor which takes
|
||||||
|
``akka.actor.ActorSystem.Settings`` and ``com.typesafe.config.Config``
|
||||||
|
arguments, as this constructor is invoked reflectively to construct your
|
||||||
|
mailbox type. The config passed in as second argument is that section from
|
||||||
|
the configuration which describes the dispatcher using this mailbox type; the
|
||||||
|
mailbox type will be instantiated once for each dispatcher using it.
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -9,7 +9,7 @@ import akka.testkit.AkkaSpec
|
||||||
import akka.event.Logging
|
import akka.event.Logging
|
||||||
import akka.event.LoggingAdapter
|
import akka.event.LoggingAdapter
|
||||||
import akka.util.duration._
|
import akka.util.duration._
|
||||||
import akka.actor.{ Props, Actor, PoisonPill }
|
import akka.actor.{ Props, Actor, PoisonPill, ActorSystem }
|
||||||
|
|
||||||
object DispatcherDocSpec {
|
object DispatcherDocSpec {
|
||||||
val config = """
|
val config = """
|
||||||
|
|
@ -110,7 +110,7 @@ object DispatcherDocSpec {
|
||||||
|
|
||||||
// We inherit, in this case, from UnboundedPriorityMailbox
|
// We inherit, in this case, from UnboundedPriorityMailbox
|
||||||
// and seed it with the priority generator
|
// and seed it with the priority generator
|
||||||
class MyPrioMailbox(config: Config) extends UnboundedPriorityMailbox(
|
class MyPrioMailbox(settings: ActorSystem.Settings, config: Config) extends UnboundedPriorityMailbox(
|
||||||
// Create a new PriorityGenerator, lower prio means more important
|
// Create a new PriorityGenerator, lower prio means more important
|
||||||
PriorityGenerator {
|
PriorityGenerator {
|
||||||
// 'highpriority messages should be treated first if possible
|
// 'highpriority messages should be treated first if possible
|
||||||
|
|
@ -146,7 +146,7 @@ object DispatcherDocSpec {
|
||||||
}
|
}
|
||||||
|
|
||||||
// This constructor signature must exist, it will be called by Akka
|
// This constructor signature must exist, it will be called by Akka
|
||||||
def this(config: Config) = this()
|
def this(settings: ActorSystem.Settings, config: Config) = this()
|
||||||
|
|
||||||
// The create method is called to create the MessageQueue
|
// The create method is called to create the MessageQueue
|
||||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
final override def create(owner: Option[ActorContext]): MessageQueue =
|
||||||
|
|
|
||||||
|
|
@ -169,4 +169,14 @@ An example is worth a thousand quacks:
|
||||||
|
|
||||||
.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#mailbox-implementation-example
|
.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#mailbox-implementation-example
|
||||||
|
|
||||||
And then you just specify the FQCN of your MailboxType as the value of the "mailbox-type" in the dispatcher configuration.
|
And then you just specify the FQCN of your MailboxType as the value of the "mailbox-type" in the dispatcher configuration.
|
||||||
|
|
||||||
|
.. note::
|
||||||
|
|
||||||
|
Make sure to include a constructor which takes
|
||||||
|
``akka.actor.ActorSystem.Settings`` and ``com.typesafe.config.Config``
|
||||||
|
arguments, as this constructor is invoked reflectively to construct your
|
||||||
|
mailbox type. The config passed in as second argument is that section from
|
||||||
|
the configuration which describes the dispatcher using this mailbox type; the
|
||||||
|
mailbox type will be instantiated once for each dispatcher using it.
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -14,12 +14,14 @@ import akka.dispatch.MailboxType
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import akka.config.ConfigurationException
|
import akka.config.ConfigurationException
|
||||||
import akka.dispatch.MessageQueue
|
import akka.dispatch.MessageQueue
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
|
||||||
class BeanstalkBasedMailboxException(message: String) extends AkkaException(message) {}
|
class BeanstalkBasedMailboxException(message: String) extends AkkaException(message) {}
|
||||||
|
|
||||||
class BeanstalkBasedMailboxType(config: Config) extends MailboxType {
|
class BeanstalkBasedMailboxType(systemSettings: ActorSystem.Settings, config: Config) extends MailboxType {
|
||||||
|
private val settings = new BeanstalkMailboxSettings(systemSettings, config)
|
||||||
override def create(owner: Option[ActorContext]): MessageQueue = owner match {
|
override def create(owner: Option[ActorContext]): MessageQueue = owner match {
|
||||||
case Some(o) ⇒ new BeanstalkBasedMessageQueue(o)
|
case Some(o) ⇒ new BeanstalkBasedMessageQueue(o, settings)
|
||||||
case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)")
|
case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -27,9 +29,8 @@ class BeanstalkBasedMailboxType(config: Config) extends MailboxType {
|
||||||
/**
|
/**
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
class BeanstalkBasedMessageQueue(_owner: ActorContext) extends DurableMessageQueue(_owner) with DurableMessageSerialization {
|
class BeanstalkBasedMessageQueue(_owner: ActorContext, val settings: BeanstalkMailboxSettings) extends DurableMessageQueue(_owner) with DurableMessageSerialization {
|
||||||
|
|
||||||
private val settings = BeanstalkBasedMailboxExtension(owner.system)
|
|
||||||
private val messageSubmitDelaySeconds = settings.MessageSubmitDelay.toSeconds.toInt
|
private val messageSubmitDelaySeconds = settings.MessageSubmitDelay.toSeconds.toInt
|
||||||
private val messageTimeToLiveSeconds = settings.MessageTimeToLive.toSeconds.toInt
|
private val messageTimeToLiveSeconds = settings.MessageTimeToLive.toSeconds.toInt
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -1,28 +0,0 @@
|
||||||
/**
|
|
||||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
|
||||||
*/
|
|
||||||
package akka.actor.mailbox
|
|
||||||
|
|
||||||
import com.typesafe.config.Config
|
|
||||||
import akka.util.Duration
|
|
||||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
|
||||||
import akka.actor._
|
|
||||||
|
|
||||||
object BeanstalkBasedMailboxExtension extends ExtensionId[BeanstalkMailboxSettings] with ExtensionIdProvider {
|
|
||||||
override def get(system: ActorSystem): BeanstalkMailboxSettings = super.get(system)
|
|
||||||
def lookup() = this
|
|
||||||
def createExtension(system: ExtendedActorSystem) = new BeanstalkMailboxSettings(system.settings.config)
|
|
||||||
}
|
|
||||||
|
|
||||||
class BeanstalkMailboxSettings(val config: Config) extends Extension {
|
|
||||||
|
|
||||||
import config._
|
|
||||||
|
|
||||||
val Hostname = getString("akka.actor.mailbox.beanstalk.hostname")
|
|
||||||
val Port = getInt("akka.actor.mailbox.beanstalk.port")
|
|
||||||
val ReconnectWindow = Duration(getMilliseconds("akka.actor.mailbox.beanstalk.reconnect-window"), MILLISECONDS)
|
|
||||||
val MessageSubmitDelay = Duration(getMilliseconds("akka.actor.mailbox.beanstalk.message-submit-delay"), MILLISECONDS)
|
|
||||||
val MessageSubmitTimeout = Duration(getMilliseconds("akka.actor.mailbox.beanstalk.message-submit-timeout"), MILLISECONDS)
|
|
||||||
val MessageTimeToLive = Duration(getMilliseconds("akka.actor.mailbox.beanstalk.message-time-to-live"), MILLISECONDS)
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
@ -0,0 +1,27 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||||
|
*/
|
||||||
|
package akka.actor.mailbox
|
||||||
|
|
||||||
|
import com.typesafe.config.Config
|
||||||
|
import akka.util.Duration
|
||||||
|
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
|
||||||
|
class BeanstalkMailboxSettings(val systemSettings: ActorSystem.Settings, val userConfig: Config)
|
||||||
|
extends DurableMailboxSettings {
|
||||||
|
|
||||||
|
def name = "beanstalk"
|
||||||
|
|
||||||
|
val config = initialize
|
||||||
|
|
||||||
|
import config._
|
||||||
|
|
||||||
|
val Hostname = getString("hostname")
|
||||||
|
val Port = getInt("port")
|
||||||
|
val ReconnectWindow = Duration(getMilliseconds("reconnect-window"), MILLISECONDS)
|
||||||
|
val MessageSubmitDelay = Duration(getMilliseconds("message-submit-delay"), MILLISECONDS)
|
||||||
|
val MessageSubmitTimeout = Duration(getMilliseconds("message-submit-timeout"), MILLISECONDS)
|
||||||
|
val MessageTimeToLive = Duration(getMilliseconds("message-time-to-live"), MILLISECONDS)
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -5,9 +5,26 @@ object BeanstalkBasedMailboxSpec {
|
||||||
Beanstalkd-dispatcher {
|
Beanstalkd-dispatcher {
|
||||||
mailbox-type = akka.actor.mailbox.BeanstalkBasedMailboxType
|
mailbox-type = akka.actor.mailbox.BeanstalkBasedMailboxType
|
||||||
throughput = 1
|
throughput = 1
|
||||||
|
beanstalk {
|
||||||
|
hostname = "127.0.0.1"
|
||||||
|
port = 11400
|
||||||
|
}
|
||||||
}
|
}
|
||||||
"""
|
"""
|
||||||
}
|
}
|
||||||
|
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class BeanstalkBasedMailboxSpec extends DurableMailboxSpec("Beanstalkd", BeanstalkBasedMailboxSpec.config)
|
class BeanstalkBasedMailboxSpec extends DurableMailboxSpec("Beanstalkd", BeanstalkBasedMailboxSpec.config) {
|
||||||
|
|
||||||
|
lazy val beanstalkd = new ProcessBuilder("beanstalkd", "-b", "beanstalk", "-l", "127.0.0.1", "-p", "11400").start()
|
||||||
|
|
||||||
|
override def atStartup(): Unit = {
|
||||||
|
new java.io.File("beanstalk").mkdir()
|
||||||
|
beanstalkd
|
||||||
|
Thread.sleep(3000)
|
||||||
|
}
|
||||||
|
|
||||||
|
override def atTermination(): Unit = beanstalkd.destroy()
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -13,19 +13,20 @@ import akka.dispatch.MailboxType
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import akka.util.NonFatal
|
import akka.util.NonFatal
|
||||||
import akka.config.ConfigurationException
|
import akka.config.ConfigurationException
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
|
||||||
class FileBasedMailboxType(config: Config) extends MailboxType {
|
class FileBasedMailboxType(systemSettings: ActorSystem.Settings, config: Config) extends MailboxType {
|
||||||
|
private val settings = new FileBasedMailboxSettings(systemSettings, config)
|
||||||
override def create(owner: Option[ActorContext]): MessageQueue = owner match {
|
override def create(owner: Option[ActorContext]): MessageQueue = owner match {
|
||||||
case Some(o) ⇒ new FileBasedMessageQueue(o)
|
case Some(o) ⇒ new FileBasedMessageQueue(o, settings)
|
||||||
case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)")
|
case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class FileBasedMessageQueue(_owner: ActorContext) extends DurableMessageQueue(_owner) with DurableMessageSerialization {
|
class FileBasedMessageQueue(_owner: ActorContext, val settings: FileBasedMailboxSettings) extends DurableMessageQueue(_owner) with DurableMessageSerialization {
|
||||||
|
|
||||||
val log = Logging(system, "FileBasedMessageQueue")
|
val log = Logging(system, "FileBasedMessageQueue")
|
||||||
|
|
||||||
private val settings = FileBasedMailboxExtension(owner.system)
|
|
||||||
val queuePath = settings.QueuePath
|
val queuePath = settings.QueuePath
|
||||||
|
|
||||||
private val queue = try {
|
private val queue = try {
|
||||||
|
|
@ -1,35 +0,0 @@
|
||||||
/**
|
|
||||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
|
||||||
*/
|
|
||||||
package akka.actor.mailbox
|
|
||||||
|
|
||||||
import com.typesafe.config.Config
|
|
||||||
import akka.util.Duration
|
|
||||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
|
||||||
import akka.actor._
|
|
||||||
|
|
||||||
object FileBasedMailboxExtension extends ExtensionId[FileBasedMailboxSettings] with ExtensionIdProvider {
|
|
||||||
override def get(system: ActorSystem): FileBasedMailboxSettings = super.get(system)
|
|
||||||
def lookup() = this
|
|
||||||
def createExtension(system: ExtendedActorSystem) = new FileBasedMailboxSettings(system.settings.config)
|
|
||||||
}
|
|
||||||
|
|
||||||
class FileBasedMailboxSettings(val config: Config) extends Extension {
|
|
||||||
|
|
||||||
import config._
|
|
||||||
|
|
||||||
val QueuePath = getString("akka.actor.mailbox.file-based.directory-path")
|
|
||||||
|
|
||||||
val MaxItems = getInt("akka.actor.mailbox.file-based.max-items")
|
|
||||||
val MaxSize = getBytes("akka.actor.mailbox.file-based.max-size")
|
|
||||||
val MaxItemSize = getBytes("akka.actor.mailbox.file-based.max-item-size")
|
|
||||||
val MaxAge = Duration(getMilliseconds("akka.actor.mailbox.file-based.max-age"), MILLISECONDS)
|
|
||||||
val MaxJournalSize = getBytes("akka.actor.mailbox.file-based.max-journal-size")
|
|
||||||
val MaxMemorySize = getBytes("akka.actor.mailbox.file-based.max-memory-size")
|
|
||||||
val MaxJournalOverflow = getInt("akka.actor.mailbox.file-based.max-journal-overflow")
|
|
||||||
val MaxJournalSizeAbsolute = getBytes("akka.actor.mailbox.file-based.max-journal-size-absolute")
|
|
||||||
val DiscardOldWhenFull = getBoolean("akka.actor.mailbox.file-based.discard-old-when-full")
|
|
||||||
val KeepJournal = getBoolean("akka.actor.mailbox.file-based.keep-journal")
|
|
||||||
val SyncJournal = getBoolean("akka.actor.mailbox.file-based.sync-journal")
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
@ -0,0 +1,34 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||||
|
*/
|
||||||
|
package akka.actor.mailbox
|
||||||
|
|
||||||
|
import com.typesafe.config.Config
|
||||||
|
import akka.util.Duration
|
||||||
|
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
|
||||||
|
class FileBasedMailboxSettings(val systemSettings: ActorSystem.Settings, val userConfig: Config)
|
||||||
|
extends DurableMailboxSettings {
|
||||||
|
|
||||||
|
def name = "file-based"
|
||||||
|
|
||||||
|
val config = initialize
|
||||||
|
|
||||||
|
import config._
|
||||||
|
|
||||||
|
val QueuePath = getString("directory-path")
|
||||||
|
|
||||||
|
val MaxItems = getInt("max-items")
|
||||||
|
val MaxSize = getBytes("max-size")
|
||||||
|
val MaxItemSize = getBytes("max-item-size")
|
||||||
|
val MaxAge = Duration(getMilliseconds("max-age"), MILLISECONDS)
|
||||||
|
val MaxJournalSize = getBytes("max-journal-size")
|
||||||
|
val MaxMemorySize = getBytes("max-memory-size")
|
||||||
|
val MaxJournalOverflow = getInt("max-journal-overflow")
|
||||||
|
val MaxJournalSizeAbsolute = getBytes("max-journal-size-absolute")
|
||||||
|
val DiscardOldWhenFull = getBoolean("discard-old-when-full")
|
||||||
|
val KeepJournal = getBoolean("keep-journal")
|
||||||
|
val SyncJournal = getBoolean("sync-journal")
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -1,12 +1,14 @@
|
||||||
package akka.actor.mailbox
|
package akka.actor.mailbox
|
||||||
|
|
||||||
import org.apache.commons.io.FileUtils
|
import org.apache.commons.io.FileUtils
|
||||||
|
import com.typesafe.config.ConfigFactory
|
||||||
|
|
||||||
object FileBasedMailboxSpec {
|
object FileBasedMailboxSpec {
|
||||||
val config = """
|
val config = """
|
||||||
File-dispatcher {
|
File-dispatcher {
|
||||||
mailbox-type = akka.actor.mailbox.FileBasedMailboxType
|
mailbox-type = akka.actor.mailbox.FileBasedMailboxType
|
||||||
throughput = 1
|
throughput = 1
|
||||||
|
file-based.directory-path = "file-based"
|
||||||
}
|
}
|
||||||
"""
|
"""
|
||||||
}
|
}
|
||||||
|
|
@ -14,8 +16,15 @@ object FileBasedMailboxSpec {
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FileBasedMailboxSpec extends DurableMailboxSpec("File", FileBasedMailboxSpec.config) {
|
class FileBasedMailboxSpec extends DurableMailboxSpec("File", FileBasedMailboxSpec.config) {
|
||||||
|
|
||||||
|
val queuePath = new FileBasedMailboxSettings(system.settings, system.settings.config.getConfig("File-dispatcher")).QueuePath
|
||||||
|
|
||||||
|
"FileBasedMailboxSettings" must {
|
||||||
|
"read the file-based section" in {
|
||||||
|
queuePath must be("file-based")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
def clean {
|
def clean {
|
||||||
val queuePath = FileBasedMailboxExtension(system).QueuePath
|
|
||||||
FileUtils.deleteDirectory(new java.io.File(queuePath))
|
FileUtils.deleteDirectory(new java.io.File(queuePath))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -7,6 +7,8 @@ import akka.actor.{ ActorContext, ActorRef, ExtendedActorSystem }
|
||||||
import akka.dispatch.{ Envelope, MessageQueue }
|
import akka.dispatch.{ Envelope, MessageQueue }
|
||||||
import akka.remote.MessageSerializer
|
import akka.remote.MessageSerializer
|
||||||
import akka.remote.RemoteProtocol.{ ActorRefProtocol, RemoteMessageProtocol }
|
import akka.remote.RemoteProtocol.{ ActorRefProtocol, RemoteMessageProtocol }
|
||||||
|
import com.typesafe.config.Config
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
|
||||||
private[akka] object DurableExecutableMailboxConfig {
|
private[akka] object DurableExecutableMailboxConfig {
|
||||||
val Name = "[\\.\\/\\$\\s]".r
|
val Name = "[\\.\\/\\$\\s]".r
|
||||||
|
|
@ -50,3 +52,55 @@ trait DurableMessageSerialization { this: DurableMessageQueue ⇒
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Conventional organization of durable mailbox settings:
|
||||||
|
*
|
||||||
|
* {{{
|
||||||
|
* my-durable-dispatcher {
|
||||||
|
* mailbox-type = "my.durable.mailbox"
|
||||||
|
* my-durable-mailbox {
|
||||||
|
* setting1 = 1
|
||||||
|
* setting2 = 2
|
||||||
|
* }
|
||||||
|
* }
|
||||||
|
* }}}
|
||||||
|
*
|
||||||
|
* where name=“my-durable-mailbox” in this example.
|
||||||
|
*/
|
||||||
|
trait DurableMailboxSettings {
|
||||||
|
/**
|
||||||
|
* A reference to the enclosing actor system.
|
||||||
|
*/
|
||||||
|
def systemSettings: ActorSystem.Settings
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A reference to the config section which the user specified for this mailbox’s dispatcher.
|
||||||
|
*/
|
||||||
|
def userConfig: Config
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The extracted config section for this mailbox, which is the “name”
|
||||||
|
* section (if that exists), falling back to system defaults. Typical
|
||||||
|
* implementation looks like:
|
||||||
|
*
|
||||||
|
* {{{
|
||||||
|
* val config = initialize
|
||||||
|
* }}}
|
||||||
|
*/
|
||||||
|
def config: Config
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Name of this mailbox type for purposes of configuration scoping. Reference
|
||||||
|
* defaults go into “akka.actor.mailbox.<name>”.
|
||||||
|
*/
|
||||||
|
def name: String
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Obtain default extracted mailbox config section from userConfig and system.
|
||||||
|
*/
|
||||||
|
def initialize: Config =
|
||||||
|
if (userConfig.hasPath(name))
|
||||||
|
userConfig.getConfig(name).withFallback(systemSettings.config.getConfig("akka.actor.mailbox." + name))
|
||||||
|
else systemSettings.config.getConfig("akka.actor.mailbox." + name)
|
||||||
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -11,6 +11,9 @@ import akka.dispatch.Await
|
||||||
import akka.testkit.AkkaSpec
|
import akka.testkit.AkkaSpec
|
||||||
import akka.testkit.TestLatch
|
import akka.testkit.TestLatch
|
||||||
import akka.util.duration._
|
import akka.util.duration._
|
||||||
|
import java.io.InputStream
|
||||||
|
import scala.annotation.tailrec
|
||||||
|
import com.typesafe.config.Config
|
||||||
|
|
||||||
object DurableMailboxSpecActorFactory {
|
object DurableMailboxSpecActorFactory {
|
||||||
|
|
||||||
|
|
@ -31,6 +34,26 @@ object DurableMailboxSpecActorFactory {
|
||||||
abstract class DurableMailboxSpec(val backendName: String, config: String) extends AkkaSpec(config) {
|
abstract class DurableMailboxSpec(val backendName: String, config: String) extends AkkaSpec(config) {
|
||||||
import DurableMailboxSpecActorFactory._
|
import DurableMailboxSpecActorFactory._
|
||||||
|
|
||||||
|
protected def streamMustContain(in: InputStream, words: String): Unit = {
|
||||||
|
val output = new Array[Byte](8192)
|
||||||
|
|
||||||
|
def now = System.currentTimeMillis
|
||||||
|
|
||||||
|
def string(len: Int) = new String(output, 0, len, "ISO-8859-1") // don’t want parse errors
|
||||||
|
|
||||||
|
@tailrec def read(end: Int = 0, start: Long = now): Int =
|
||||||
|
in.read(output, end, output.length - end) match {
|
||||||
|
case -1 ⇒ end
|
||||||
|
case x ⇒
|
||||||
|
val next = end + x
|
||||||
|
if (string(next).contains(words) || now - start > 10000 || next == output.length) next
|
||||||
|
else read(next, start)
|
||||||
|
}
|
||||||
|
|
||||||
|
val result = string(read())
|
||||||
|
if (!result.contains(words)) throw new Exception("stream did not contain '" + words + "':\n" + result)
|
||||||
|
}
|
||||||
|
|
||||||
def createMailboxTestActor(id: String): ActorRef =
|
def createMailboxTestActor(id: String): ActorRef =
|
||||||
system.actorOf(Props(new MailboxTestActor).withDispatcher(backendName + "-dispatcher"))
|
system.actorOf(Props(new MailboxTestActor).withDispatcher(backendName + "-dispatcher"))
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -16,12 +16,14 @@ import akka.dispatch.MailboxType
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import akka.config.ConfigurationException
|
import akka.config.ConfigurationException
|
||||||
import akka.dispatch.MessageQueue
|
import akka.dispatch.MessageQueue
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
|
||||||
class MongoBasedMailboxException(message: String) extends AkkaException(message)
|
class MongoBasedMailboxException(message: String) extends AkkaException(message)
|
||||||
|
|
||||||
class MongoBasedMailboxType(config: Config) extends MailboxType {
|
class MongoBasedMailboxType(systemSettings: ActorSystem.Settings, config: Config) extends MailboxType {
|
||||||
|
private val settings = new MongoBasedMailboxSettings(systemSettings, config)
|
||||||
override def create(owner: Option[ActorContext]): MessageQueue = owner match {
|
override def create(owner: Option[ActorContext]): MessageQueue = owner match {
|
||||||
case Some(o) ⇒ new MongoBasedMessageQueue(o)
|
case Some(o) ⇒ new MongoBasedMessageQueue(o, settings)
|
||||||
case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)")
|
case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -37,15 +39,13 @@ class MongoBasedMailboxType(config: Config) extends MailboxType {
|
||||||
*
|
*
|
||||||
* @author <a href="http://evilmonkeylabs.com">Brendan W. McAdams</a>
|
* @author <a href="http://evilmonkeylabs.com">Brendan W. McAdams</a>
|
||||||
*/
|
*/
|
||||||
class MongoBasedMessageQueue(_owner: ActorContext) extends DurableMessageQueue(_owner) {
|
class MongoBasedMessageQueue(_owner: ActorContext, val settings: MongoBasedMailboxSettings) extends DurableMessageQueue(_owner) {
|
||||||
// this implicit object provides the context for reading/writing things as MongoDurableMessage
|
// this implicit object provides the context for reading/writing things as MongoDurableMessage
|
||||||
implicit val mailboxBSONSer = new BSONSerializableMessageQueue(system)
|
implicit val mailboxBSONSer = new BSONSerializableMessageQueue(system)
|
||||||
implicit val safeWrite = WriteConcern.Safe // TODO - Replica Safe when appropriate!
|
implicit val safeWrite = WriteConcern.Safe // TODO - Replica Safe when appropriate!
|
||||||
|
|
||||||
private val dispatcher = owner.dispatcher
|
private val dispatcher = owner.dispatcher
|
||||||
|
|
||||||
private val settings = MongoBasedMailboxExtension(owner.system)
|
|
||||||
|
|
||||||
val log = Logging(system, "MongoBasedMessageQueue")
|
val log = Logging(system, "MongoBasedMessageQueue")
|
||||||
|
|
||||||
@volatile
|
@volatile
|
||||||
|
|
@ -98,9 +98,8 @@ class MongoBasedMessageQueue(_owner: ActorContext) extends DurableMessageQueue(_
|
||||||
def hasMessages: Boolean = numberOfMessages > 0
|
def hasMessages: Boolean = numberOfMessages > 0
|
||||||
|
|
||||||
private[akka] def connect() = {
|
private[akka] def connect() = {
|
||||||
require(settings.MongoURI.isDefined, "Mongo URI (%s) must be explicitly defined in akka.conf; will not assume defaults for safety sake.".format(settings.UriConfigKey))
|
|
||||||
log.info("CONNECTING mongodb uri : [{}]", settings.MongoURI)
|
log.info("CONNECTING mongodb uri : [{}]", settings.MongoURI)
|
||||||
val _dbh = MongoConnection.fromURI(settings.MongoURI.get) match {
|
val _dbh = MongoConnection.fromURI(settings.MongoURI) match {
|
||||||
case (conn, None, None) ⇒ {
|
case (conn, None, None) ⇒ {
|
||||||
throw new UnsupportedOperationException("You must specify a database name to use with MongoDB; please see the MongoDB Connection URI Spec: 'http://www.mongodb.org/display/DOCS/Connections'")
|
throw new UnsupportedOperationException("You must specify a database name to use with MongoDB; please see the MongoDB Connection URI Spec: 'http://www.mongodb.org/display/DOCS/Connections'")
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -1,26 +0,0 @@
|
||||||
/**
|
|
||||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
|
||||||
*/
|
|
||||||
package akka.actor.mailbox
|
|
||||||
|
|
||||||
import com.typesafe.config.Config
|
|
||||||
import akka.util.Duration
|
|
||||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
|
||||||
import akka.actor._
|
|
||||||
|
|
||||||
object MongoBasedMailboxExtension extends ExtensionId[MongoBasedMailboxSettings] with ExtensionIdProvider {
|
|
||||||
override def get(system: ActorSystem): MongoBasedMailboxSettings = super.get(system)
|
|
||||||
def lookup() = this
|
|
||||||
def createExtension(system: ExtendedActorSystem) = new MongoBasedMailboxSettings(system.settings.config)
|
|
||||||
}
|
|
||||||
|
|
||||||
class MongoBasedMailboxSettings(val config: Config) extends Extension {
|
|
||||||
|
|
||||||
import config._
|
|
||||||
|
|
||||||
val UriConfigKey = "akka.actor.mailbox.mongodb.uri"
|
|
||||||
val MongoURI = if (config.hasPath(UriConfigKey)) Some(config.getString(UriConfigKey)) else None
|
|
||||||
val WriteTimeout = Duration(config.getMilliseconds("akka.actor.mailbox.mongodb.timeout.write"), MILLISECONDS)
|
|
||||||
val ReadTimeout = Duration(config.getMilliseconds("akka.actor.mailbox.mongodb.timeout.read"), MILLISECONDS)
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
@ -0,0 +1,24 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||||
|
*/
|
||||||
|
package akka.actor.mailbox
|
||||||
|
|
||||||
|
import com.typesafe.config.Config
|
||||||
|
import akka.util.Duration
|
||||||
|
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
|
||||||
|
class MongoBasedMailboxSettings(val systemSettings: ActorSystem.Settings, val userConfig: Config)
|
||||||
|
extends DurableMailboxSettings {
|
||||||
|
|
||||||
|
def name = "mongodb"
|
||||||
|
|
||||||
|
val config = initialize
|
||||||
|
|
||||||
|
import config._
|
||||||
|
|
||||||
|
val MongoURI = getString("uri")
|
||||||
|
val WriteTimeout = Duration(config.getMilliseconds("timeout.write"), MILLISECONDS)
|
||||||
|
val ReadTimeout = Duration(config.getMilliseconds("timeout.read"), MILLISECONDS)
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -14,6 +14,7 @@ object MongoBasedMailboxSpec {
|
||||||
mongodb-dispatcher {
|
mongodb-dispatcher {
|
||||||
mailbox-type = akka.actor.mailbox.MongoBasedMailboxType
|
mailbox-type = akka.actor.mailbox.MongoBasedMailboxType
|
||||||
throughput = 1
|
throughput = 1
|
||||||
|
mongodb.uri = "mongodb://localhost:27123/akka.mailbox"
|
||||||
}
|
}
|
||||||
"""
|
"""
|
||||||
}
|
}
|
||||||
|
|
@ -23,9 +24,23 @@ class MongoBasedMailboxSpec extends DurableMailboxSpec("mongodb", MongoBasedMail
|
||||||
|
|
||||||
import com.mongodb.async._
|
import com.mongodb.async._
|
||||||
|
|
||||||
val mongo = MongoConnection("localhost", 27017)("akka")
|
lazy val mongod = new ProcessBuilder("mongod", "--dbpath", "mongoDB", "--bind_ip", "127.0.0.1", "--port", "27123").start()
|
||||||
|
lazy val mongo = MongoConnection("localhost", 27123)("akka")
|
||||||
|
|
||||||
mongo.dropDatabase() { success ⇒ }
|
override def atStartup(): Unit = {
|
||||||
|
// start MongoDB daemon
|
||||||
|
new java.io.File("mongoDB").mkdir()
|
||||||
|
val in = mongod.getInputStream
|
||||||
|
|
||||||
|
try {
|
||||||
|
streamMustContain(in, "waiting for connections on port")
|
||||||
|
mongo.dropDatabase() { success ⇒ }
|
||||||
|
} catch {
|
||||||
|
case e ⇒ mongod.destroy(); throw e
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
override def atTermination(): Unit = mongod.destroy()
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -14,19 +14,19 @@ import com.typesafe.config.Config
|
||||||
import akka.util.NonFatal
|
import akka.util.NonFatal
|
||||||
import akka.config.ConfigurationException
|
import akka.config.ConfigurationException
|
||||||
import akka.dispatch.MessageQueue
|
import akka.dispatch.MessageQueue
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
|
||||||
class RedisBasedMailboxException(message: String) extends AkkaException(message)
|
class RedisBasedMailboxException(message: String) extends AkkaException(message)
|
||||||
|
|
||||||
class RedisBasedMailboxType(config: Config) extends MailboxType {
|
class RedisBasedMailboxType(systemSettings: ActorSystem.Settings, config: Config) extends MailboxType {
|
||||||
|
private val settings = new RedisBasedMailboxSettings(systemSettings, config)
|
||||||
override def create(owner: Option[ActorContext]): MessageQueue = owner match {
|
override def create(owner: Option[ActorContext]): MessageQueue = owner match {
|
||||||
case Some(o) ⇒ new RedisBasedMessageQueue(o)
|
case Some(o) ⇒ new RedisBasedMessageQueue(o, settings)
|
||||||
case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)")
|
case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class RedisBasedMessageQueue(_owner: ActorContext) extends DurableMessageQueue(_owner) with DurableMessageSerialization {
|
class RedisBasedMessageQueue(_owner: ActorContext, val settings: RedisBasedMailboxSettings) extends DurableMessageQueue(_owner) with DurableMessageSerialization {
|
||||||
|
|
||||||
private val settings = RedisBasedMailboxExtension(owner.system)
|
|
||||||
|
|
||||||
@volatile
|
@volatile
|
||||||
private var clients = connect() // returns a RedisClientPool for multiple asynchronous message handling
|
private var clients = connect() // returns a RedisClientPool for multiple asynchronous message handling
|
||||||
|
|
|
||||||
|
|
@ -1,21 +0,0 @@
|
||||||
/**
|
|
||||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
|
||||||
*/
|
|
||||||
package akka.actor.mailbox
|
|
||||||
|
|
||||||
import com.typesafe.config.Config
|
|
||||||
import akka.actor._
|
|
||||||
|
|
||||||
object RedisBasedMailboxExtension extends ExtensionId[RedisBasedMailboxSettings] with ExtensionIdProvider {
|
|
||||||
override def get(system: ActorSystem): RedisBasedMailboxSettings = super.get(system)
|
|
||||||
def lookup() = this
|
|
||||||
def createExtension(system: ExtendedActorSystem) = new RedisBasedMailboxSettings(system.settings.config)
|
|
||||||
}
|
|
||||||
|
|
||||||
class RedisBasedMailboxSettings(val config: Config) extends Extension {
|
|
||||||
|
|
||||||
import config._
|
|
||||||
|
|
||||||
val Hostname = getString("akka.actor.mailbox.redis.hostname")
|
|
||||||
val Port = getInt("akka.actor.mailbox.redis.port")
|
|
||||||
}
|
|
||||||
|
|
@ -0,0 +1,20 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||||
|
*/
|
||||||
|
package akka.actor.mailbox
|
||||||
|
|
||||||
|
import com.typesafe.config.Config
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
|
||||||
|
class RedisBasedMailboxSettings(val systemSettings: ActorSystem.Settings, val userConfig: Config)
|
||||||
|
extends DurableMailboxSettings {
|
||||||
|
|
||||||
|
def name = "redis"
|
||||||
|
|
||||||
|
val config = initialize
|
||||||
|
|
||||||
|
import config._
|
||||||
|
|
||||||
|
val Hostname = getString("hostname")
|
||||||
|
val Port = getInt("port")
|
||||||
|
}
|
||||||
|
|
@ -5,9 +5,40 @@ object RedisBasedMailboxSpec {
|
||||||
Redis-dispatcher {
|
Redis-dispatcher {
|
||||||
mailbox-type = akka.actor.mailbox.RedisBasedMailboxType
|
mailbox-type = akka.actor.mailbox.RedisBasedMailboxType
|
||||||
throughput = 1
|
throughput = 1
|
||||||
|
redis {
|
||||||
|
hostname = "127.0.0.1"
|
||||||
|
port = 6479
|
||||||
|
}
|
||||||
}
|
}
|
||||||
"""
|
"""
|
||||||
}
|
}
|
||||||
|
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class RedisBasedMailboxSpec extends DurableMailboxSpec("Redis", RedisBasedMailboxSpec.config)
|
class RedisBasedMailboxSpec extends DurableMailboxSpec("Redis", RedisBasedMailboxSpec.config) {
|
||||||
|
|
||||||
|
lazy val redisServer = new ProcessBuilder("redis-server", "-").start()
|
||||||
|
|
||||||
|
override def atStartup(): Unit = {
|
||||||
|
new java.io.File("redis").mkdir()
|
||||||
|
|
||||||
|
val out = redisServer.getOutputStream
|
||||||
|
|
||||||
|
val config = """
|
||||||
|
port 6479
|
||||||
|
bind 127.0.0.1
|
||||||
|
dir redis
|
||||||
|
""".getBytes("UTF-8")
|
||||||
|
|
||||||
|
try {
|
||||||
|
out.write(config)
|
||||||
|
out.close()
|
||||||
|
|
||||||
|
streamMustContain(redisServer.getInputStream, "ready to accept connections on port")
|
||||||
|
} catch {
|
||||||
|
case e ⇒ redisServer.destroy(); throw e
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
override def atTermination(): Unit = redisServer.destroy()
|
||||||
|
|
||||||
|
}
|
||||||
|
|
|
||||||
|
|
@ -15,19 +15,20 @@ import com.typesafe.config.Config
|
||||||
import akka.util.NonFatal
|
import akka.util.NonFatal
|
||||||
import akka.config.ConfigurationException
|
import akka.config.ConfigurationException
|
||||||
import akka.dispatch.MessageQueue
|
import akka.dispatch.MessageQueue
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
|
||||||
class ZooKeeperBasedMailboxException(message: String) extends AkkaException(message)
|
class ZooKeeperBasedMailboxException(message: String) extends AkkaException(message)
|
||||||
|
|
||||||
class ZooKeeperBasedMailboxType(config: Config) extends MailboxType {
|
class ZooKeeperBasedMailboxType(systemSettings: ActorSystem.Settings, config: Config) extends MailboxType {
|
||||||
|
private val settings = new ZooKeeperBasedMailboxSettings(systemSettings, config)
|
||||||
override def create(owner: Option[ActorContext]): MessageQueue = owner match {
|
override def create(owner: Option[ActorContext]): MessageQueue = owner match {
|
||||||
case Some(o) ⇒ new ZooKeeperBasedMessageQueue(o)
|
case Some(o) ⇒ new ZooKeeperBasedMessageQueue(o, settings)
|
||||||
case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)")
|
case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class ZooKeeperBasedMessageQueue(_owner: ActorContext) extends DurableMessageQueue(_owner) with DurableMessageSerialization {
|
class ZooKeeperBasedMessageQueue(_owner: ActorContext, val settings: ZooKeeperBasedMailboxSettings) extends DurableMessageQueue(_owner) with DurableMessageSerialization {
|
||||||
|
|
||||||
private val settings = ZooKeeperBasedMailboxExtension(owner.system)
|
|
||||||
val queueNode = "/queues"
|
val queueNode = "/queues"
|
||||||
val queuePathTemplate = queueNode + "/%s"
|
val queuePathTemplate = queueNode + "/%s"
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -1,25 +0,0 @@
|
||||||
/**
|
|
||||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
|
||||||
*/
|
|
||||||
package akka.actor.mailbox
|
|
||||||
|
|
||||||
import com.typesafe.config.Config
|
|
||||||
import akka.util.Duration
|
|
||||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
|
||||||
import akka.actor._
|
|
||||||
|
|
||||||
object ZooKeeperBasedMailboxExtension extends ExtensionId[ZooKeeperBasedMailboxSettings] with ExtensionIdProvider {
|
|
||||||
override def get(system: ActorSystem): ZooKeeperBasedMailboxSettings = super.get(system)
|
|
||||||
def lookup() = this
|
|
||||||
def createExtension(system: ExtendedActorSystem) = new ZooKeeperBasedMailboxSettings(system.settings.config)
|
|
||||||
}
|
|
||||||
class ZooKeeperBasedMailboxSettings(val config: Config) extends Extension {
|
|
||||||
|
|
||||||
import config._
|
|
||||||
|
|
||||||
val ZkServerAddresses = getString("akka.actor.mailbox.zookeeper.server-addresses")
|
|
||||||
val SessionTimeout = Duration(getMilliseconds("akka.actor.mailbox.zookeeper.session-timeout"), MILLISECONDS)
|
|
||||||
val ConnectionTimeout = Duration(getMilliseconds("akka.actor.mailbox.zookeeper.connection-timeout"), MILLISECONDS)
|
|
||||||
val BlockingQueue = getBoolean("akka.actor.mailbox.zookeeper.blocking-queue")
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
@ -0,0 +1,25 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||||
|
*/
|
||||||
|
package akka.actor.mailbox
|
||||||
|
|
||||||
|
import com.typesafe.config.Config
|
||||||
|
import akka.util.Duration
|
||||||
|
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
|
||||||
|
class ZooKeeperBasedMailboxSettings(val systemSettings: ActorSystem.Settings, val userConfig: Config)
|
||||||
|
extends DurableMailboxSettings {
|
||||||
|
|
||||||
|
def name = "zookeeper"
|
||||||
|
|
||||||
|
val config = initialize
|
||||||
|
|
||||||
|
import config._
|
||||||
|
|
||||||
|
val ZkServerAddresses = getString("server-addresses")
|
||||||
|
val SessionTimeout = Duration(getMilliseconds("session-timeout"), MILLISECONDS)
|
||||||
|
val ConnectionTimeout = Duration(getMilliseconds("connection-timeout"), MILLISECONDS)
|
||||||
|
val BlockingQueue = getBoolean("blocking-queue")
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -5,12 +5,15 @@ import akka.cluster.zookeeper._
|
||||||
import org.I0Itec.zkclient._
|
import org.I0Itec.zkclient._
|
||||||
import akka.dispatch.MessageDispatcher
|
import akka.dispatch.MessageDispatcher
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
|
import com.typesafe.config.ConfigFactory
|
||||||
|
import akka.util.duration._
|
||||||
|
|
||||||
object ZooKeeperBasedMailboxSpec {
|
object ZooKeeperBasedMailboxSpec {
|
||||||
val config = """
|
val config = """
|
||||||
ZooKeeper-dispatcher {
|
ZooKeeper-dispatcher {
|
||||||
mailbox-type = akka.actor.mailbox.ZooKeeperBasedMailboxType
|
mailbox-type = akka.actor.mailbox.ZooKeeperBasedMailboxType
|
||||||
throughput = 1
|
throughput = 1
|
||||||
|
zookeeper.session-timeout = 30s
|
||||||
}
|
}
|
||||||
"""
|
"""
|
||||||
}
|
}
|
||||||
|
|
@ -21,6 +24,12 @@ class ZooKeeperBasedMailboxSpec extends DurableMailboxSpec("ZooKeeper", ZooKeepe
|
||||||
val dataPath = "_akka_cluster/data"
|
val dataPath = "_akka_cluster/data"
|
||||||
val logPath = "_akka_cluster/log"
|
val logPath = "_akka_cluster/log"
|
||||||
|
|
||||||
|
"ZookeeperBasedMailboxSettings" must {
|
||||||
|
"read the right settings" in {
|
||||||
|
new ZooKeeperBasedMailboxSettings(system.settings, system.settings.config.getConfig("ZooKeeper-dispatcher")).SessionTimeout must be(30 seconds)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
var zkServer: ZkServer = _
|
var zkServer: ZkServer = _
|
||||||
|
|
||||||
override def atStartup() {
|
override def atStartup() {
|
||||||
|
|
|
||||||
|
|
@ -5,3 +5,4 @@
|
||||||
(externalResolvers in LsKeys.lsync) := Seq("Akka Repository" at "http://akka.io/repository/")
|
(externalResolvers in LsKeys.lsync) := Seq("Akka Repository" at "http://akka.io/repository/")
|
||||||
|
|
||||||
(description in LsKeys.lsync) := "Akka is the platform for the next generation of event-driven, scalable and fault-tolerant architectures on the JVM."
|
(description in LsKeys.lsync) := "Akka is the platform for the next generation of event-driven, scalable and fault-tolerant architectures on the JVM."
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -26,6 +26,7 @@ object AkkaBuild extends Build {
|
||||||
id = "akka",
|
id = "akka",
|
||||||
base = file("."),
|
base = file("."),
|
||||||
settings = parentSettings ++ Release.settings ++ Unidoc.settings ++ Rstdoc.settings ++ Publish.versionSettings ++ Dist.settings ++ Seq(
|
settings = parentSettings ++ Release.settings ++ Unidoc.settings ++ Rstdoc.settings ++ Publish.versionSettings ++ Dist.settings ++ Seq(
|
||||||
|
testMailbox in GlobalScope := System.getProperty("akka.testMailbox", "false").toBoolean,
|
||||||
parallelExecution in GlobalScope := System.getProperty("akka.parallelExecution", "false").toBoolean,
|
parallelExecution in GlobalScope := System.getProperty("akka.parallelExecution", "false").toBoolean,
|
||||||
Publish.defaultPublishTo in ThisBuild <<= crossTarget / "repository",
|
Publish.defaultPublishTo in ThisBuild <<= crossTarget / "repository",
|
||||||
Unidoc.unidocExclude := Seq(samples.id, tutorials.id),
|
Unidoc.unidocExclude := Seq(samples.id, tutorials.id),
|
||||||
|
|
@ -141,6 +142,8 @@ object AkkaBuild extends Build {
|
||||||
// )
|
// )
|
||||||
// )
|
// )
|
||||||
|
|
||||||
|
val testMailbox = SettingKey[Boolean]("test-mailbox")
|
||||||
|
|
||||||
lazy val mailboxes = Project(
|
lazy val mailboxes = Project(
|
||||||
id = "akka-durable-mailboxes",
|
id = "akka-durable-mailboxes",
|
||||||
base = file("akka-durable-mailboxes"),
|
base = file("akka-durable-mailboxes"),
|
||||||
|
|
@ -165,8 +168,7 @@ object AkkaBuild extends Build {
|
||||||
dependencies = Seq(mailboxesCommon % "compile;test->test"),
|
dependencies = Seq(mailboxesCommon % "compile;test->test"),
|
||||||
settings = defaultSettings ++ Seq(
|
settings = defaultSettings ++ Seq(
|
||||||
libraryDependencies ++= Dependencies.beanstalkMailbox,
|
libraryDependencies ++= Dependencies.beanstalkMailbox,
|
||||||
testBeanstalkMailbox := false,
|
testOptions in Test <+= testMailbox map { test => Tests.Filter(s => test) }
|
||||||
testOptions in Test <+= testBeanstalkMailbox map { test => Tests.Filter(s => test) }
|
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
@ -179,16 +181,13 @@ object AkkaBuild extends Build {
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
|
||||||
val testRedisMailbox = SettingKey[Boolean]("test-redis-mailbox")
|
|
||||||
|
|
||||||
lazy val redisMailbox = Project(
|
lazy val redisMailbox = Project(
|
||||||
id = "akka-redis-mailbox",
|
id = "akka-redis-mailbox",
|
||||||
base = file("akka-durable-mailboxes/akka-redis-mailbox"),
|
base = file("akka-durable-mailboxes/akka-redis-mailbox"),
|
||||||
dependencies = Seq(mailboxesCommon % "compile;test->test"),
|
dependencies = Seq(mailboxesCommon % "compile;test->test"),
|
||||||
settings = defaultSettings ++ Seq(
|
settings = defaultSettings ++ Seq(
|
||||||
libraryDependencies ++= Dependencies.redisMailbox,
|
libraryDependencies ++= Dependencies.redisMailbox,
|
||||||
testRedisMailbox := false,
|
testOptions in Test <+= testMailbox map { test => Tests.Filter(s => test) }
|
||||||
testOptions in Test <+= testRedisMailbox map { test => Tests.Filter(s => test) }
|
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
@ -201,8 +200,6 @@ object AkkaBuild extends Build {
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
|
||||||
val testMongoMailbox = SettingKey[Boolean]("test-mongo-mailbox")
|
|
||||||
|
|
||||||
lazy val mongoMailbox = Project(
|
lazy val mongoMailbox = Project(
|
||||||
id = "akka-mongo-mailbox",
|
id = "akka-mongo-mailbox",
|
||||||
base = file("akka-durable-mailboxes/akka-mongo-mailbox"),
|
base = file("akka-durable-mailboxes/akka-mongo-mailbox"),
|
||||||
|
|
@ -210,8 +207,7 @@ object AkkaBuild extends Build {
|
||||||
settings = defaultSettings ++ Seq(
|
settings = defaultSettings ++ Seq(
|
||||||
libraryDependencies ++= Dependencies.mongoMailbox,
|
libraryDependencies ++= Dependencies.mongoMailbox,
|
||||||
ivyXML := Dependencies.mongoMailboxExcludes,
|
ivyXML := Dependencies.mongoMailboxExcludes,
|
||||||
testMongoMailbox := false,
|
testOptions in Test <+= testMailbox map { test => Tests.Filter(s => test) }
|
||||||
testOptions in Test <+= testMongoMailbox map { test => Tests.Filter(s => test) }
|
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue