!cls #13711 Move cluster sharding
This commit is contained in:
parent
fb72214d99
commit
10a11e415e
17 changed files with 168 additions and 37 deletions
18
akka-cluster-sharding/build.sbt
Normal file
18
akka-cluster-sharding/build.sbt
Normal file
|
|
@ -0,0 +1,18 @@
|
|||
import akka.{ AkkaBuild, Dependencies, Formatting, MultiNode, Unidoc, OSGi }
|
||||
import com.typesafe.tools.mima.plugin.MimaKeys
|
||||
|
||||
AkkaBuild.defaultSettings
|
||||
|
||||
Formatting.formatSettings
|
||||
|
||||
Unidoc.scaladocSettingsNoVerificationOfDiagrams
|
||||
|
||||
Unidoc.javadocSettings
|
||||
|
||||
OSGi.clusterSharding
|
||||
|
||||
MultiNode.multiJvmSettings
|
||||
|
||||
libraryDependencies ++= Dependencies.clusterSharding
|
||||
|
||||
//MimaKeys.previousArtifact := akkaPreviousArtifact("akka-cluster-sharding").value
|
||||
66
akka-cluster-sharding/src/main/resources/reference.conf
Normal file
66
akka-cluster-sharding/src/main/resources/reference.conf
Normal file
|
|
@ -0,0 +1,66 @@
|
|||
###############################################
|
||||
# Akka Cluster Sharding Reference Config File #
|
||||
###############################################
|
||||
|
||||
# This is the reference config file that contains all the default settings.
|
||||
# Make your edits/overrides in your application.conf.
|
||||
|
||||
|
||||
# //#sharding-ext-config
|
||||
# Settings for the ClusterShardingExtension
|
||||
akka.cluster.sharding {
|
||||
|
||||
# The extension creates a top level actor with this name in top level user scope,
|
||||
# e.g. '/user/sharding'
|
||||
guardian-name = sharding
|
||||
|
||||
# If the coordinator can't store state changes it will be stopped
|
||||
# and started again after this duration.
|
||||
coordinator-failure-backoff = 10 s
|
||||
|
||||
# Start the coordinator singleton manager on members tagged with this role.
|
||||
# All members are used if undefined or empty.
|
||||
# ShardRegion actor is started in proxy only mode on nodes that are not tagged
|
||||
# with this role.
|
||||
role = ""
|
||||
|
||||
# The ShardRegion retries registration and shard location requests to the
|
||||
# ShardCoordinator with this interval if it does not reply.
|
||||
retry-interval = 2 s
|
||||
# Maximum number of messages that are buffered by a ShardRegion actor.
|
||||
buffer-size = 100000
|
||||
|
||||
# Timeout of the shard rebalancing process.
|
||||
handoff-timeout = 60 s
|
||||
|
||||
# Time given to a region to acknowdge it's hosting a shard.
|
||||
shard-start-timeout = 10 s
|
||||
|
||||
# If the shard can't store state changes it will retry the action
|
||||
# again after this duration. Any messages sent to an affected entry
|
||||
# will be buffered until the state change is processed
|
||||
shard-failure-backoff = 10 s
|
||||
|
||||
# If the shard is remembering entries and an entry stops itself without
|
||||
# using passivate. The entry will be restarted after this duration or when
|
||||
# the next message for it is received, which ever occurs first.
|
||||
entry-restart-backoff = 10 s
|
||||
|
||||
# Rebalance check is performed periodically with this interval.
|
||||
rebalance-interval = 10 s
|
||||
|
||||
# How often the coordinator saves persistent snapshots, which are
|
||||
# used to reduce recovery times
|
||||
snapshot-interval = 3600 s
|
||||
|
||||
# Setting for the default shard allocation strategy
|
||||
least-shard-allocation-strategy {
|
||||
# Threshold of how large the difference between most and least number of
|
||||
# allocated shards must be to begin the rebalancing.
|
||||
rebalance-threshold = 10
|
||||
|
||||
# The number of ongoing rebalancing processes is limited to this number.
|
||||
max-simultaneous-rebalance = 3
|
||||
}
|
||||
}
|
||||
# //#sharding-ext-config
|
||||
|
|
@ -1,12 +1,12 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.contrib.pattern
|
||||
package akka.cluster.sharding
|
||||
|
||||
import java.net.URLEncoder
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import akka.contrib.pattern.Shard.{ ShardCommand, StateChange }
|
||||
import akka.contrib.pattern.ShardCoordinator.Internal.SnapshotTick
|
||||
import akka.cluster.sharding.Shard.{ ShardCommand, StateChange }
|
||||
import akka.cluster.sharding.ShardCoordinator.Internal.SnapshotTick
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.Await
|
||||
|
|
@ -62,7 +62,7 @@ import akka.cluster.singleton.ClusterSingletonManager
|
|||
* in the cluster, registering the supported entry types with the [[ClusterSharding#start]]
|
||||
* method and then the `ShardRegion` actor for a named entry type can be retrieved with
|
||||
* [[ClusterSharding#shardRegion]]. Messages to the entries are always sent via the local
|
||||
* `ShardRegion`. Some settings can be configured as described in the `akka.contrib.cluster.sharding`
|
||||
* `ShardRegion`. Some settings can be configured as described in the `akka.cluster.sharding`
|
||||
* section of the `reference.conf`.
|
||||
*
|
||||
* The `ShardRegion` actor is started on each node in the cluster, or group of nodes
|
||||
|
|
@ -138,7 +138,7 @@ import akka.cluster.singleton.ClusterSingletonManager
|
|||
* The `ShardRegion` actor can also be started in proxy only mode, i.e. it will not
|
||||
* host any entries itself, but knows how to delegate messages to the right location.
|
||||
* A `ShardRegion` starts in proxy only mode if the roles of the node does not include
|
||||
* the node role specified in `akka.contrib.cluster.sharding.role` config property
|
||||
* the node role specified in `akka.cluster.sharding.role` config property
|
||||
* or if the specified `entryProps` is `None`/`null`.
|
||||
*
|
||||
* If the state of the entries are persistent you may stop entries that are not used to
|
||||
|
|
@ -175,7 +175,7 @@ class ClusterSharding(system: ExtendedActorSystem) extends Extension {
|
|||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object Settings {
|
||||
val config = system.settings.config.getConfig("akka.contrib.cluster.sharding")
|
||||
val config = system.settings.config.getConfig("akka.cluster.sharding")
|
||||
|
||||
val Role: Option[String] = config.getString("role") match {
|
||||
case "" ⇒ None
|
||||
|
|
@ -207,7 +207,7 @@ class ClusterSharding(system: ExtendedActorSystem) extends Extension {
|
|||
* and functions to extract entry and shard identifier from messages. The [[ShardRegion]] actor
|
||||
* for this type can later be retrieved with the [[#shardRegion]] method.
|
||||
*
|
||||
* Some settings can be configured as described in the `akka.contrib.cluster.sharding` section
|
||||
* Some settings can be configured as described in the `akka.cluster.sharding` section
|
||||
* of the `reference.conf`.
|
||||
*
|
||||
* @param typeName the name of the entry type
|
||||
|
|
@ -254,7 +254,7 @@ class ClusterSharding(system: ExtendedActorSystem) extends Extension {
|
|||
* The default shard allocation strategy [[ShardCoordinator.LeastShardAllocationStrategy]]
|
||||
* is used.
|
||||
*
|
||||
* Some settings can be configured as described in the `akka.contrib.cluster.sharding` section
|
||||
* Some settings can be configured as described in the `akka.cluster.sharding` section
|
||||
* of the `reference.conf`.
|
||||
*
|
||||
* @param typeName the name of the entry type
|
||||
|
|
@ -290,7 +290,7 @@ class ClusterSharding(system: ExtendedActorSystem) extends Extension {
|
|||
* and functions to extract entry and shard identifier from messages. The [[ShardRegion]] actor
|
||||
* for this type can later be retrieved with the [[#shardRegion]] method.
|
||||
*
|
||||
* Some settings can be configured as described in the `akka.contrib.cluster.sharding` section
|
||||
* Some settings can be configured as described in the `akka.cluster.sharding` section
|
||||
* of the `reference.conf`.
|
||||
*
|
||||
* @param typeName the name of the entry type
|
||||
|
|
@ -333,7 +333,7 @@ class ClusterSharding(system: ExtendedActorSystem) extends Extension {
|
|||
* The default shard allocation strategy [[ShardCoordinator.LeastShardAllocationStrategy]]
|
||||
* is used.
|
||||
*
|
||||
* Some settings can be configured as described in the `akka.contrib.cluster.sharding` section
|
||||
* Some settings can be configured as described in the `akka.cluster.sharding` section
|
||||
* of the `reference.conf`.
|
||||
*
|
||||
* @param typeName the name of the entry type
|
||||
|
|
@ -1016,8 +1016,8 @@ private[akka] class Shard(
|
|||
import ShardRegion.{ handOffStopperProps, EntryId, Msg, Passivate }
|
||||
import ShardCoordinator.Internal.{ HandOff, ShardStopped }
|
||||
import Shard.{ State, RetryPersistence, RestartEntry, EntryStopped, EntryStarted, SnapshotTick }
|
||||
import akka.contrib.pattern.ShardCoordinator.Internal.CoordinatorMessage
|
||||
import akka.contrib.pattern.ShardRegion.ShardRegionCommand
|
||||
import akka.cluster.sharding.ShardCoordinator.Internal.CoordinatorMessage
|
||||
import akka.cluster.sharding.ShardRegion.ShardRegionCommand
|
||||
import akka.persistence.RecoveryCompleted
|
||||
|
||||
import context.dispatcher
|
||||
|
|
@ -1,10 +1,10 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.contrib.pattern
|
||||
package akka.cluster.sharding
|
||||
|
||||
import java.io.File
|
||||
import akka.contrib.pattern.ShardRegion.Passivate
|
||||
import akka.cluster.sharding.ShardRegion.Passivate
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import org.apache.commons.io.FileUtils
|
||||
|
|
@ -43,8 +43,8 @@ object ClusterShardingFailureSpec extends MultiNodeConfig {
|
|||
}
|
||||
akka.persistence.snapshot-store.plugin = "akka.persistence.snapshot-store.local"
|
||||
akka.persistence.snapshot-store.local.dir = "target/snapshots-ClusterShardingFailureSpec"
|
||||
akka.contrib.cluster.sharding.coordinator-failure-backoff = 3s
|
||||
akka.contrib.cluster.sharding.shard-failure-backoff = 3s
|
||||
akka.cluster.sharding.coordinator-failure-backoff = 3s
|
||||
akka.cluster.sharding.shard-failure-backoff = 3s
|
||||
"""))
|
||||
|
||||
testTransport(on = true)
|
||||
|
|
@ -1,7 +1,7 @@
|
|||
/**
|
||||
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.contrib.pattern
|
||||
package akka.cluster.sharding
|
||||
|
||||
import java.io.File
|
||||
|
||||
|
|
@ -1,10 +1,10 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.contrib.pattern
|
||||
package akka.cluster.sharding
|
||||
|
||||
import akka.contrib.pattern.ShardCoordinator.Internal.{ ShardStopped, HandOff }
|
||||
import akka.contrib.pattern.ShardRegion.Passivate
|
||||
import akka.cluster.sharding.ShardCoordinator.Internal.{ ShardStopped, HandOff }
|
||||
import akka.cluster.sharding.ShardRegion.Passivate
|
||||
|
||||
import language.postfixOps
|
||||
import scala.concurrent.duration._
|
||||
|
|
@ -23,6 +23,7 @@ import akka.testkit._
|
|||
import akka.testkit.TestEvent.Mute
|
||||
import java.io.File
|
||||
import org.apache.commons.io.FileUtils
|
||||
import akka.cluster.singleton.ClusterSingletonManager
|
||||
|
||||
object ClusterShardingSpec extends MultiNodeConfig {
|
||||
val controller = role("controller")
|
||||
|
|
@ -46,7 +47,7 @@ object ClusterShardingSpec extends MultiNodeConfig {
|
|||
}
|
||||
akka.persistence.snapshot-store.plugin = "akka.persistence.snapshot-store.local"
|
||||
akka.persistence.snapshot-store.local.dir = "target/snapshots-ClusterShardingSpec"
|
||||
akka.contrib.cluster.sharding {
|
||||
akka.cluster.sharding {
|
||||
role = backend
|
||||
retry-interval = 1 s
|
||||
handoff-timeout = 10 s
|
||||
|
|
@ -2,7 +2,7 @@
|
|||
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.contrib.pattern;
|
||||
package akka.cluster.sharding;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
6
akka-cluster-sharding/src/test/resources/reference.conf
Normal file
6
akka-cluster-sharding/src/test/resources/reference.conf
Normal file
|
|
@ -0,0 +1,6 @@
|
|||
akka {
|
||||
actor {
|
||||
serialize-creators = on
|
||||
serialize-messages = on
|
||||
}
|
||||
}
|
||||
|
|
@ -1,7 +1,7 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.contrib.pattern
|
||||
package akka.cluster.sharding
|
||||
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.actor.Props
|
||||
|
|
@ -34,7 +34,6 @@ The Current List of Modules
|
|||
throttle
|
||||
jul
|
||||
peek-mailbox
|
||||
cluster-sharding
|
||||
aggregator
|
||||
receive-pipeline
|
||||
|
||||
|
|
|
|||
|
|
@ -9,6 +9,7 @@ Networking
|
|||
../scala/cluster-singleton
|
||||
../scala/distributed-pub-sub
|
||||
../scala/cluster-client
|
||||
../scala/cluster-sharding
|
||||
cluster-metrics
|
||||
remoting
|
||||
serialization
|
||||
|
|
|
|||
|
|
@ -212,3 +212,13 @@ The classes changed package name from ``akka.contrib.pattern`` to ``akka.cluster
|
|||
and ``akka.cluster.client``.
|
||||
|
||||
The configuration properties changed name to ``akka.cluster.pub-sub`` and ``akka.cluster.client``.
|
||||
|
||||
Cluster sharding moved to separate module
|
||||
=========================================
|
||||
|
||||
The Cluster Sharding previously located in the ``akka-contrib`` jar is now moved to a separate module
|
||||
named ``akka-cluster-sharding``. You need to replace this dependency if you use Cluster Sharding.
|
||||
|
||||
The classes changed package name from ``akka.contrib.pattern`` to ``akka.cluster.sharding``.
|
||||
|
||||
The configuration properties changed name to ``akka.cluster.sharding``.
|
||||
|
|
|
|||
|
|
@ -27,7 +27,7 @@ An Example in Java
|
|||
|
||||
This is how an entry actor may look like:
|
||||
|
||||
.. includecode:: @contribSrc@/src/test/java/akka/contrib/pattern/ClusterShardingTest.java#counter-actor
|
||||
.. includecode:: ../../../akka-cluster-sharding/src/test/java/akka/cluster/sharding/ClusterShardingTest.java#counter-actor
|
||||
|
||||
The above actor uses event sourcing and the support provided in ``UntypedPersistentActor`` to store its state.
|
||||
It does not have to be a persistent actor, but in case of failure or migration of entries between nodes it must be able to recover
|
||||
|
|
@ -39,12 +39,12 @@ When using the sharding extension you are first, typically at system startup on
|
|||
in the cluster, supposed to register the supported entry types with the ``ClusterSharding.start``
|
||||
method. ``ClusterSharding.start`` gives you the reference which you can pass along.
|
||||
|
||||
.. includecode:: @contribSrc@/src/test/java/akka/contrib/pattern/ClusterShardingTest.java#counter-start
|
||||
.. includecode:: ../../../akka-cluster-sharding/src/test/java/akka/cluster/sharding/ClusterShardingTest.java#counter-start
|
||||
|
||||
The ``messageExtractor`` defines application specific methods to extract the entry
|
||||
identifier and the shard identifier from incoming messages.
|
||||
|
||||
.. includecode:: @contribSrc@/src/test/java/akka/contrib/pattern/ClusterShardingTest.java#counter-extractor
|
||||
.. includecode:: ../../../akka-cluster-sharding/src/test/java/akka/cluster/sharding/ClusterShardingTest.java#counter-extractor
|
||||
|
||||
This example illustrates two different ways to define the entry identifier in the messages:
|
||||
|
||||
|
|
@ -77,14 +77,14 @@ The ``ShardRegion`` will lookup the location of the shard for the entry if it do
|
|||
delegate the message to the right node and it will create the entry actor on demand, i.e. when the
|
||||
first message for a specific entry is delivered.
|
||||
|
||||
.. includecode:: @contribSrc@/src/test/java/akka/contrib/pattern/ClusterShardingTest.java#counter-usage
|
||||
.. includecode:: ../../../akka-cluster-sharding/src/test/java/akka/cluster/sharding/ClusterShardingTest.java#counter-usage
|
||||
|
||||
An Example in Scala
|
||||
-------------------
|
||||
|
||||
This is how an entry actor may look like:
|
||||
|
||||
.. includecode:: @contribSrc@/src/multi-jvm/scala/akka/contrib/pattern/ClusterShardingSpec.scala#counter-actor
|
||||
.. includecode:: ../../../akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSpec.scala#counter-actor
|
||||
|
||||
The above actor uses event sourcing and the support provided in ``PersistentActor`` to store its state.
|
||||
It does not have to be a persistent actor, but in case of failure or migration of entries between nodes it must be able to recover
|
||||
|
|
@ -96,12 +96,12 @@ When using the sharding extension you are first, typically at system startup on
|
|||
in the cluster, supposed to register the supported entry types with the ``ClusterSharding.start``
|
||||
method. ``ClusterSharding.start`` gives you the reference which you can pass along.
|
||||
|
||||
.. includecode:: @contribSrc@/src/multi-jvm/scala/akka/contrib/pattern/ClusterShardingSpec.scala#counter-start
|
||||
.. includecode:: ../../../akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSpec.scala#counter-start
|
||||
|
||||
The ``idExtractor`` and ``shardResolver`` are two application specific functions to extract the entry
|
||||
identifier and the shard identifier from incoming messages.
|
||||
|
||||
.. includecode:: @contribSrc@/src/multi-jvm/scala/akka/contrib/pattern/ClusterShardingSpec.scala#counter-extractor
|
||||
.. includecode:: ../../../akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSpec.scala#counter-extractor
|
||||
|
||||
This example illustrates two different ways to define the entry identifier in the messages:
|
||||
|
||||
|
|
@ -134,7 +134,7 @@ The ``ShardRegion`` will lookup the location of the shard for the entry if it do
|
|||
delegate the message to the right node and it will create the entry actor on demand, i.e. when the
|
||||
first message for a specific entry is delivered.
|
||||
|
||||
.. includecode:: @contribSrc@/src/multi-jvm/scala/akka/contrib/pattern/ClusterShardingSpec.scala#counter-usage
|
||||
.. includecode:: ../../../akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSpec.scala#counter-usage
|
||||
|
||||
A more comprehensive sample is available in the `Typesafe Activator <http://www.typesafe.com/platform/getstarted>`_
|
||||
tutorial named `Akka Cluster Sharding with Scala! <http://www.typesafe.com/activator/template/akka-cluster-sharding-scala>`_.
|
||||
|
|
@ -274,12 +274,30 @@ using a ``Passivate``.
|
|||
Note that the state of the entries themselves will not be restored unless they have been made persistent,
|
||||
e.g. with ``akka-persistence``.
|
||||
|
||||
Dependencies
|
||||
------------
|
||||
|
||||
To use the Cluster Sharding you must add the following dependency in your project.
|
||||
|
||||
sbt::
|
||||
|
||||
"com.typesafe.akka" %% "akka-cluster-sharding" % "@version@" @crossString@
|
||||
|
||||
maven::
|
||||
|
||||
<dependency>
|
||||
<groupId>com.typesafe.akka</groupId>
|
||||
<artifactId>akka-cluster-sharding_@binVersion@</artifactId>
|
||||
<version>@version@</version>
|
||||
</dependency>
|
||||
|
||||
|
||||
Configuration
|
||||
-------------
|
||||
|
||||
The ``ClusterSharding`` extension can be configured with the following properties:
|
||||
|
||||
.. includecode:: @contribSrc@/src/main/resources/reference.conf#sharding-ext-config
|
||||
.. includecode:: ../../../akka-cluster-sharding/src/main/resources/reference.conf#sharding-ext-config
|
||||
|
||||
Custom shard allocation strategy can be defined in an optional parameter to
|
||||
``ClusterSharding.start``. See the API documentation of ``ShardAllocationStrategy``
|
||||
|
|
@ -9,6 +9,7 @@ Networking
|
|||
cluster-singleton
|
||||
distributed-pub-sub
|
||||
cluster-client
|
||||
cluster-sharding
|
||||
cluster-metrics
|
||||
remoting
|
||||
serialization
|
||||
|
|
|
|||
|
|
@ -55,8 +55,8 @@ object AkkaBuild extends Build {
|
|||
archivesPathFinder.get.map(file => (file -> ("akka/" + file.getName)))
|
||||
}
|
||||
),
|
||||
aggregate = Seq(actor, testkit, actorTests, remote, remoteTests, camel, cluster, clusterMetrics, clusterTools, slf4j, agent,
|
||||
persistence, persistenceTck, kernel, osgi, docs, contrib, samples, multiNodeTestkit, typed)
|
||||
aggregate = Seq(actor, testkit, actorTests, remote, remoteTests, camel, cluster, clusterMetrics, clusterTools, clusterSharding,
|
||||
slf4j, agent, persistence, persistenceTck, kernel, osgi, docs, contrib, samples, multiNodeTestkit, typed)
|
||||
)
|
||||
|
||||
lazy val akkaScalaNightly = Project(
|
||||
|
|
@ -64,8 +64,8 @@ object AkkaBuild extends Build {
|
|||
base = file("akka-scala-nightly"),
|
||||
// remove dependencies that we have to build ourselves (Scala STM)
|
||||
// samples don't work with dbuild right now
|
||||
aggregate = Seq(actor, testkit, actorTests, remote, remoteTests, camel, cluster, clusterMetrics, clusterTools, slf4j,
|
||||
persistence, persistenceTck, kernel, osgi, contrib, multiNodeTestkit, typed)
|
||||
aggregate = Seq(actor, testkit, actorTests, remote, remoteTests, camel, cluster, clusterMetrics, clusterTools, clusterSharding,
|
||||
slf4j, persistence, persistenceTck, kernel, osgi, contrib, multiNodeTestkit, typed)
|
||||
).disablePlugins(ValidatePullRequest)
|
||||
|
||||
lazy val actor = Project(
|
||||
|
|
@ -132,6 +132,13 @@ object AkkaBuild extends Build {
|
|||
base = file("akka-cluster-tools"),
|
||||
dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm")
|
||||
) configs (MultiJvm)
|
||||
|
||||
lazy val clusterSharding = Project(
|
||||
id = "akka-cluster-sharding",
|
||||
base = file("akka-cluster-sharding"),
|
||||
dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm",
|
||||
persistence % "compile;test->provided", clusterTools)
|
||||
) configs (MultiJvm)
|
||||
|
||||
lazy val slf4j = Project(
|
||||
id = "akka-slf4j",
|
||||
|
|
|
|||
|
|
@ -89,6 +89,8 @@ object Dependencies {
|
|||
val cluster = Seq(Test.junit, Test.scalatest)
|
||||
|
||||
val clusterTools = Seq(Test.junit, Test.scalatest)
|
||||
|
||||
val clusterSharding = Seq(Test.junit, Test.scalatest, Test.commonsIo)
|
||||
|
||||
val clusterMetrics = Seq(Provided.sigarLoader, Test.slf4jJul, Test.slf4jLog4j, Test.logback, Test.mockito)
|
||||
|
||||
|
|
|
|||
|
|
@ -30,6 +30,8 @@ object OSGi {
|
|||
|
||||
val clusterTools = exports(Seq("akka.cluster.singleton.*", "akka.cluster.client.*", "akka.cluster.pubsub.*"),
|
||||
imports = Seq(protobufImport()))
|
||||
|
||||
val clusterSharding = exports(Seq("akka.cluster.sharding.*"), imports = Seq(protobufImport()))
|
||||
|
||||
val clusterMetrics = exports(Seq("akka.cluster.metrics.*"), imports = Seq(protobufImport(),kamonImport(),sigarImport()))
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue