From eaafed69ebbcc7b3260df30b80b80e9d9554926b Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 12 Dec 2011 08:37:18 +0100 Subject: [PATCH] DOC: Update Durable Mailboxes Chapter. See #1472 --- akka-docs/cluster/durable-mailbox.rst | 278 ------------------ akka-docs/cluster/index.rst | 1 - .../actor/mailbox/DurableMailboxDocSpec.scala | 31 ++ .../actor/mailbox/DurableMailboxDocTest.scala | 5 + .../mailbox/DurableMailboxDocTestBase.java | 41 +++ akka-docs/modules/durable-mailbox.rst | 221 ++++++++++++++ akka-docs/modules/index.rst | 1 + .../akka/actor/mailbox/DurableMailbox.scala | 16 + project/AkkaBuild.scala | 2 +- 9 files changed, 316 insertions(+), 280 deletions(-) delete mode 100644 akka-docs/cluster/durable-mailbox.rst create mode 100644 akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala create mode 100644 akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTest.scala create mode 100644 akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTestBase.java create mode 100644 akka-docs/modules/durable-mailbox.rst diff --git a/akka-docs/cluster/durable-mailbox.rst b/akka-docs/cluster/durable-mailbox.rst deleted file mode 100644 index 875d6ea9fb..0000000000 --- a/akka-docs/cluster/durable-mailbox.rst +++ /dev/null @@ -1,278 +0,0 @@ - -.. _durable-mailboxes: - -################### - Durable Mailboxes -################### - -Overview -======== - -Akka supports a set of durable mailboxes. A durable mailbox is a replacement for -the standard actor mailbox that is durable. What this means in practice is that -if there are pending messages in the actor's mailbox when the node of the actor -resides on crashes, then when you restart the node, the actor will be able to -continue processing as if nothing had happened; with all pending messages still -in its mailbox. - -.. sidebar:: **IMPORTANT** - - None of these mailboxes work with blocking message send, e.g. the message - send operations that are relying on futures; ``?`` or ``ask``. If the node - has crashed and then restarted, the thread that was blocked waiting for the - reply is gone and there is no way we can deliver the message. - -The durable mailboxes currently supported are: - - - ``FileDurableMailboxStorage`` -- backed by a journaling transaction log on the local file system - - ``RedisDurableMailboxStorage`` -- backed by Redis - - ``ZooKeeperDurableMailboxStorage`` -- backed by ZooKeeper - - ``BeanstalkDurableMailboxStorage`` -- backed by Beanstalkd - - ``MongoNaiveDurableMailboxStorage`` -- backed by MongoDB - -We'll walk through each one of these in detail in the sections below. - -Soon Akka will also have: - - - ``AmqpDurableMailboxStorage`` -- AMQP based mailbox (default RabbitMQ) - - ``JmsDurableMailboxStorage`` -- JMS based mailbox (default ActiveMQ) - - -File-based durable mailbox -========================== - -This mailbox is backed by a journaling transaction log on the local file -system. It is the simplest want to use since it does not require an extra -infrastructure piece to administer, but it is usually sufficient and just what -you need. - -The durable dispatchers and their configuration options reside in the -``akka.actor.mailbox`` package. - -You configure durable mailboxes through the "Akka"-only durable dispatchers, the -actor is oblivious to which type of mailbox it is using. Here is an example:: - - val dispatcher = DurableDispatcher( - "my:service", - FileDurableMailboxStorage) - // Then set the actors dispatcher to this dispatcher - -or for a thread-based durable dispatcher:: - - self.dispatcher = DurablePinnedDispatcher( - self, - FileDurableMailboxStorage) - -There are 2 different durable dispatchers, ``DurableDispatcher`` and -``DurablePinnedDispatcher``, which are durable versions of -``Dispatcher`` and ``PinnedDispatcher``. - -This gives you an excellent way of creating bulkheads in your application, where -groups of actors sharing the same dispatcher also share the same backing -storage. - -Read more about that in the :ref:`dispatchers-scala` documentation. - -You can also configure and tune the file-based durable mailbox. This is done in -the ``akka.actor.mailbox.file-based`` section in the :ref:`configuration`. - -.. code-block:: none - - akka { - actor { - mailbox { - file-based { - directory-path = "./_mb" - max-items = 2147483647 - max-size = 2147483647 - max-items = 2147483647 - max-age = 0 - max-journal-size = 16777216 # 16 * 1024 * 1024 - max-memory-size = 134217728 # 128 * 1024 * 1024 - max-journal-overflow = 10 - max-journal-size-absolute = 9223372036854775807 - discard-old-when-full = on - keep-journal = on - sync-journal = off - } - } - } - } - -.. todo:: explain all the above options in detail - - -Redis-based durable mailbox -=========================== - -This mailbox is backed by a Redis queue. `Redis `_ Is a very -fast NOSQL database that has a wide range of data structure abstractions, one of -them is a queue which is what we are using in this implementation. This means -that you have to start up a Redis server that can host these durable -mailboxes. Read more in the Redis documentation on how to do that. - -Here is an example of how you can configure your dispatcher to use this mailbox:: - - val dispatcher = DurableDispatcher( - "my:service", - RedisDurableMailboxStorage) - -or for a thread-based durable dispatcher:: - - self.dispatcher = DurablePinnedDispatcher( - self, - RedisDurableMailboxStorage) - -You also need to configure the IP and port for the Redis server. This is done in -the ``akka.actor.mailbox.redis`` section in the :ref:`configuration`. - -.. code-block:: none - - akka { - actor { - mailbox { - redis { - hostname = "127.0.0.1" - port = 6379 - } - } - } - } - - -ZooKeeper-based durable mailbox -=============================== - -This mailbox is backed by `ZooKeeper `_. ZooKeeper -is a centralized service for maintaining configuration information, naming, -providing distributed synchronization, and providing group services This means -that you have to start up a ZooKeeper server (for production a ZooKeeper server -ensamble) that can host these durable mailboxes. Read more in the ZooKeeper -documentation on how to do that. - -Akka is using ZooKeeper for many other things, for example the clustering -support so if you're using that you love to run a ZooKeeper server anyway and -there will not be that much more work to set up this durable mailbox. - -Here is an example of how you can configure your dispatcher to use this mailbox:: - - val dispatcher = DurableDispatcher( - "my:service", - ZooKeeperDurableMailboxStorage) - -or for a thread-based durable dispatcher:: - - self.dispatcher = DurablePinnedDispatcher( - self, - ZooKeeperDurableMailboxStorage) - -You also need to configure ZooKeeper server addresses, timeouts, etc. This is -done in the ``akka.actor.mailbox.zookeeper`` section in the :ref:`configuration`. - -.. code-block:: none - - akka { - actor { - mailbox { - zookeeper { - server-addresses = "localhost:2181" - session-timeout = 60 - connection-timeout = 30 - blocking-queue = on - } - } - } - } - - -Beanstalk-based durable mailbox -=============================== - -This mailbox is backed by `Beanstalkd `_. -Beanstalk is a simple, fast work queue. This means that you have to start up a -Beanstalk server that can host these durable mailboxes. Read more in the -Beanstalk documentation on how to do that. :: - - val dispatcher = DurableDispatcher( - "my:service", - BeanstalkDurableMailboxStorage) - -or for a thread-based durable dispatcher. :: - - self.dispatcher = DurablePinnedDispatcher( - self, - BeanstalkDurableMailboxStorage) - -You also need to configure the IP, and port, and so on, for the Beanstalk -server. This is done in the ``akka.actor.mailbox.beanstalk`` section in the -:ref:`configuration`. - -.. code-block:: none - - akka { - actor { - mailbox { - beanstalk { - hostname = "127.0.0.1" - port = 11300 - reconnect-window = 5 - message-submit-delay = 0 - message-submit-timeout = 5 - message-time-to-live = 120 - } - } - } - } - -MongoDB-based Durable Mailboxes -=============================== - -This mailbox is backed by `MongoDB `_. -MongoDB is a fast, lightweight and scalable document-oriented database. It contains a number of -features cohesive to a fast, reliable & durable queueing mechanism which the Akka Mailbox takes advantage of. - - -Akka's implementations of MongoDB mailboxes are built on top of the purely asynchronous MongoDB driver (often known as `Hammersmith `_ and ``com.mongodb.async``) and as such are purely callback based with a Netty network layer. This makes them extremely fast & lightweight versus building on other MongoDB implementations such as `mongo-java-driver `_ and `Casbah `_. - -You will need to configure the URI for the MongoDB server, using the URI Format specified in the `MongoDB Documentation `_. This is done in -the ``akka.actor.mailbox.mongodb`` section in the :ref:`configuration`. - -.. code-block:: none - - mongodb { - # Any specified collection name will be used as a prefix for collections that use durable mongo mailboxes - uri = "mongodb://localhost/akka.mailbox" # Follow Mongo URI Spec - http://www.mongodb.org/display/DOCS/Connections - # Configurable timeouts for certain ops - timeout { - read = 3000 # number of milliseconds to wait for a read to succeed before timing out the future - write = 3000 # number of milliseconds to wait for a write to succeed before timing out the future - } - } - -You must specify a hostname (and optionally port) and at *least* a Database name. If you specify a collection name, it will be used as a 'prefix' for the collections Akka creates to store mailbox messages. Otherwise, collections will be prefixed with ``mailbox.`` - -It is also possible to configure the timeout threshholds for Read and Write operations in the ``timeout`` block. -Currently Akka offers only one "type" of MongoDB based Mailbox but there are plans to support at least -one other kind which uses a different queueing strategy. - - -'Naive' MongoDB-based Durable Mailbox -------------------------------------- -The currently supported mailbox is considered "Naive" as it removes messages (using the ``findAndRemove`` -command) from the MongoDB datastore as soon as the actor consumes them. This could cause message loss -if an actor crashes before completely processing a message. It is not a problem per sé, but behavior -users should be aware of. - -Here is an example of how you can configure your dispatcher to use this mailbox:: - - val dispatcher = DurableDispatcher( - "my:service", - MongoNaiveDurableMailboxStorage) - -or for a thread-based durable dispatcher:: - - self.dispatcher = DurablePinnedDispatcher( - self, - MongoNaiveDurableMailboxStorage) - - diff --git a/akka-docs/cluster/index.rst b/akka-docs/cluster/index.rst index bdbd95bde6..35c4b2250a 100644 --- a/akka-docs/cluster/index.rst +++ b/akka-docs/cluster/index.rst @@ -5,4 +5,3 @@ Cluster :maxdepth: 2 cluster - durable-mailbox diff --git a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala b/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala new file mode 100644 index 0000000000..566aef4c27 --- /dev/null +++ b/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala @@ -0,0 +1,31 @@ +package akka.docs.actor.mailbox + +//#imports +import akka.actor.Actor +import akka.actor.Props +import akka.actor.mailbox.FileDurableMailboxType + +//#imports + +import org.scalatest.{ BeforeAndAfterAll, WordSpec } +import org.scalatest.matchers.MustMatchers +import akka.testkit.AkkaSpec + +class MyActor extends Actor { + def receive = { + case x ⇒ + } +} + +class DurableMailboxDocSpec extends AkkaSpec { + + "define dispatcher with durable mailbox" in { + //#define-dispatcher + val dispatcher = system.dispatcherFactory.newDispatcher( + "my-dispatcher", throughput = 1, mailboxType = FileDurableMailboxType).build + val myActor = system.actorOf(Props[MyActor].withDispatcher(dispatcher), name = "myactor") + //#define-dispatcher + myActor ! "hello" + } + +} diff --git a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTest.scala b/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTest.scala new file mode 100644 index 0000000000..fa31f08b6a --- /dev/null +++ b/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTest.scala @@ -0,0 +1,5 @@ +package akka.docs.actor.mailbox + +import org.scalatest.junit.JUnitSuite + +class DurableMailboxDocTest extends DurableMailboxDocTestBase with JUnitSuite diff --git a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTestBase.java b/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTestBase.java new file mode 100644 index 0000000000..4ac3204d0b --- /dev/null +++ b/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTestBase.java @@ -0,0 +1,41 @@ +package akka.docs.actor.mailbox; + +//#imports +import akka.actor.mailbox.DurableMailboxType; +import akka.dispatch.MessageDispatcher; +import akka.actor.UntypedActorFactory; +import akka.actor.UntypedActor; +import akka.actor.Props; + +//#imports + +import org.junit.Test; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; + +import static org.junit.Assert.*; + +public class DurableMailboxDocTestBase { + + @Test + public void defineDispatcher() { + ActorSystem system = ActorSystem.create("MySystem"); + //#define-dispatcher + MessageDispatcher dispatcher = system.dispatcherFactory() + .newDispatcher("my-dispatcher", 1, DurableMailboxType.fileDurableMailboxType()).build(); + ActorRef myActor = system.actorOf(new Props().withDispatcher(dispatcher).withCreator(new UntypedActorFactory() { + public UntypedActor create() { + return new MyUntypedActor(); + } + })); + //#define-dispatcher + myActor.tell("test"); + system.stop(); + } + + public static class MyUntypedActor extends UntypedActor { + public void onReceive(Object message) { + } + } +} diff --git a/akka-docs/modules/durable-mailbox.rst b/akka-docs/modules/durable-mailbox.rst new file mode 100644 index 0000000000..9291f22d37 --- /dev/null +++ b/akka-docs/modules/durable-mailbox.rst @@ -0,0 +1,221 @@ + +.. _durable-mailboxes: + +################### + Durable Mailboxes +################### + +.. sidebar:: Contents + + .. contents:: :local: + +Overview +======== + +Akka supports a set of durable mailboxes. A durable mailbox is a replacement for +the standard actor mailbox that is durable. What this means in practice is that +if there are pending messages in the actor's mailbox when the node of the actor +resides on crashes, then when you restart the node, the actor will be able to +continue processing as if nothing had happened; with all pending messages still +in its mailbox. + +None of these mailboxes implements transactions for current message. It's possible +if the actor crashes after receiving a message, but before completing processing of +it, that the message could be lost. + +.. warning:: **IMPORTANT** + + None of these mailboxes work with blocking message send, e.g. the message + send operations that are relying on futures; ``?`` or ``ask``. If the node + has crashed and then restarted, the thread that was blocked waiting for the + reply is gone and there is no way we can deliver the message. + +The durable mailboxes currently supported are: + + - ``FileBasedMailbox`` -- backed by a journaling transaction log on the local file system + - ``RedisBasedMailbox`` -- backed by Redis + - ``ZooKeeperBasedMailbox`` -- backed by ZooKeeper + - ``BeanstalkBasedMailbox`` -- backed by Beanstalkd + - ``MongoBasedMailbox`` -- backed by MongoDB + +We'll walk through each one of these in detail in the sections below. + +You can easily implement your own mailbox. Look at the existing implementations for inspiration. + +Soon Akka will also have: + + - ``AmqpBasedMailbox`` -- AMQP based mailbox (default RabbitMQ) + - ``JmsBasedMailbox`` -- JMS based mailbox (default ActiveMQ) + + +.. _DurableMailbox.General: + +General Usage +------------- + +The durable mailboxes and their configuration options reside in the +``akka.actor.mailbox`` package. + +You configure durable mailboxes through the dispatcher. The +actor is oblivious to which type of mailbox it is using. +Here is an example in Scala: + +.. includecode:: code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala + :include: imports,define-dispatcher + +Corresponding example in Java: + +.. includecode:: code/akka/docs/actor/mailbox/DurableMailboxDocTestBase.java + :include: imports,define-dispatcher + +The actor is oblivious to which type of mailbox it is using. + +This gives you an excellent way of creating bulkheads in your application, where +groups of actors sharing the same dispatcher also share the same backing +storage. Read more about that in the :ref:`dispatchers-scala` documentation. + +File-based durable mailbox +========================== + +This mailbox is backed by a journaling transaction log on the local file +system. It is the simplest to use since it does not require an extra +infrastructure piece to administer, but it is usually sufficient and just what +you need. + +You configure durable mailboxes through the dispatcher, as described in +:ref:`DurableMailbox.General` with the following mailbox type. + +Scala:: + + mailbox = akka.actor.mailbox.FileDurableMailboxType + +Java:: + + akka.actor.mailbox.DurableMailboxType.fileDurableMailboxType() + + +You can also configure and tune the file-based durable mailbox. This is done in +the ``akka.actor.mailbox.file-based`` section in the :ref:`configuration`. + +.. literalinclude:: ../../akka-durable-mailboxes/akka-file-mailbox/src/main/resources/reference.conf + :language: none + + +Redis-based durable mailbox +=========================== + +This mailbox is backed by a Redis queue. `Redis `_ Is a very +fast NOSQL database that has a wide range of data structure abstractions, one of +them is a queue which is what we are using in this implementation. This means +that you have to start up a Redis server that can host these durable +mailboxes. Read more in the Redis documentation on how to do that. + +You configure durable mailboxes through the dispatcher, as described in +:ref:`DurableMailbox.General` with the following mailbox type. + +Scala:: + + mailbox = akka.actor.mailbox.RedisDurableMailboxType + +Java:: + + akka.actor.mailbox.DurableMailboxType.redisDurableMailboxType() + + +You also need to configure the IP and port for the Redis server. This is done in +the ``akka.actor.mailbox.redis`` section in the :ref:`configuration`. + +.. literalinclude:: ../../akka-durable-mailboxes/akka-redis-mailbox/src/main/resources/reference.conf + :language: none + + +ZooKeeper-based durable mailbox +=============================== + +This mailbox is backed by `ZooKeeper `_. ZooKeeper +is a centralized service for maintaining configuration information, naming, +providing distributed synchronization, and providing group services This means +that you have to start up a ZooKeeper server (for production a ZooKeeper server +ensamble) that can host these durable mailboxes. Read more in the ZooKeeper +documentation on how to do that. + +You configure durable mailboxes through the dispatcher, as described in +:ref:`DurableMailbox.General` with the following mailbox type. + +Scala:: + + mailbox = akka.actor.mailbox.ZooKeeperDurableMailboxType + +Java:: + + akka.actor.mailbox.DurableMailboxType.zooKeeperDurableMailboxType() + +You also need to configure ZooKeeper server addresses, timeouts, etc. This is +done in the ``akka.actor.mailbox.zookeeper`` section in the :ref:`configuration`. + +.. literalinclude:: ../../akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/resources/reference.conf + :language: none + +Beanstalk-based durable mailbox +=============================== + +This mailbox is backed by `Beanstalkd `_. +Beanstalk is a simple, fast work queue. This means that you have to start up a +Beanstalk server that can host these durable mailboxes. Read more in the +Beanstalk documentation on how to do that. + +You configure durable mailboxes through the dispatcher, as described in +:ref:`DurableMailbox.General` with the following mailbox type. + +Scala:: + + mailbox = akka.actor.mailbox.BeanstalkDurableMailboxType + +Java:: + + akka.actor.mailbox.DurableMailboxType.beanstalkDurableMailboxType() + +You also need to configure the IP, and port, and so on, for the Beanstalk +server. This is done in the ``akka.actor.mailbox.beanstalk`` section in the +:ref:`configuration`. + +.. literalinclude:: ../../akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/resources/reference.conf + :language: none + +MongoDB-based Durable Mailboxes +=============================== + +This mailbox is backed by `MongoDB `_. +MongoDB is a fast, lightweight and scalable document-oriented database. It contains a number of +features cohesive to a fast, reliable & durable queueing mechanism which the Akka Mailbox takes advantage of. + +Akka's implementations of MongoDB mailboxes are built on top of the purely asynchronous MongoDB driver +(often known as `Hammersmith `_ and ``com.mongodb.async``) +and as such are purely callback based with a Netty network layer. This makes them extremely fast & +lightweight versus building on other MongoDB implementations such as +`mongo-java-driver `_ and `Casbah `_. + +You configure durable mailboxes through the dispatcher, as described in +:ref:`DurableMailbox.General` with the following mailbox type. + +Scala:: + + mailbox = akka.actor.mailbox.MongoDurableMailboxType + +Java:: + + akka.actor.mailbox.DurableMailboxType.mongoDurableMailboxType() + +You will need to configure the URI for the MongoDB server, using the URI Format specified in the +`MongoDB Documentation `_. This is done in +the ``akka.actor.mailbox.mongodb`` section in the :ref:`configuration`. + +.. literalinclude:: ../../akka-durable-mailboxes/akka-mongo-mailbox/src/main/resources/reference.conf + :language: none + +You must specify a hostname (and optionally port) and at *least* a Database name. If you specify a +collection name, it will be used as a 'prefix' for the collections Akka creates to store mailbox messages. +Otherwise, collections will be prefixed with ``mailbox.`` + +It is also possible to configure the timeout thresholds for Read and Write operations in the ``timeout`` block. + diff --git a/akka-docs/modules/index.rst b/akka-docs/modules/index.rst index c4d5211562..780d5b23df 100644 --- a/akka-docs/modules/index.rst +++ b/akka-docs/modules/index.rst @@ -4,6 +4,7 @@ Modules .. toctree:: :maxdepth: 2 + durable-mailbox microkernel camel spring diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala index 96cb764615..7eb30b2fdb 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala @@ -107,6 +107,22 @@ case object FileDurableMailboxType extends DurableMailboxType("akka.actor.mailbo case object ZooKeeperDurableMailboxType extends DurableMailboxType("akka.actor.mailbox.ZooKeeperBasedMailbox") case class FqnDurableMailboxType(mailboxFQN: String) extends DurableMailboxType(mailboxFQN) +/** + * Java API for the mailbox types. Usage: + *

+ * MessageDispatcher dispatcher = system.dispatcherFactory()
+ *   .newDispatcher("my-dispatcher", 1, DurableMailboxType.redisDurableMailboxType()).build();
+ * 
+ */ +object DurableMailboxType { + def redisDurableMailboxType(): DurableMailboxType = RedisDurableMailboxType + def mongoDurableMailboxType(): DurableMailboxType = MongoDurableMailboxType + def beanstalkDurableMailboxType(): DurableMailboxType = BeanstalkDurableMailboxType + def fileDurableMailboxType(): DurableMailboxType = FileDurableMailboxType + def zooKeeperDurableMailboxType(): DurableMailboxType = ZooKeeperDurableMailboxType + def fqnDurableMailboxType(mailboxFQN: String): DurableMailboxType = FqnDurableMailboxType(mailboxFQN) +} + /** * Configurator for the DurableMailbox * Do not forget to specify the "storage", valid values are "redis", "beanstalkd", "zookeeper", "mongodb", "file", diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index b92bd7a611..68c7d4e561 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -256,7 +256,7 @@ object AkkaBuild extends Build { lazy val docs = Project( id = "akka-docs", base = file("akka-docs"), - dependencies = Seq(actor, testkit % "test->test", stm, remote, slf4j), + dependencies = Seq(actor, testkit % "test->test", stm, remote, slf4j, fileMailbox, mongoMailbox, redisMailbox, beanstalkMailbox, zookeeperMailbox), settings = defaultSettings ++ Seq( unmanagedSourceDirectories in Test <<= baseDirectory { _ ** "code" get }, libraryDependencies ++= Dependencies.docs,