!cls #13711 Move cluster sharding

This commit is contained in:
Patrik Nordwall 2015-04-27 14:48:28 +02:00
parent fb72214d99
commit 10a11e415e
17 changed files with 168 additions and 37 deletions

View 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

View 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

View file

@ -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

View file

@ -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)

View file

@ -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

View 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

View file

@ -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;

View file

@ -0,0 +1,6 @@
akka {
actor {
serialize-creators = on
serialize-messages = on
}
}

View file

@ -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

View file

@ -34,7 +34,6 @@ The Current List of Modules
throttle
jul
peek-mailbox
cluster-sharding
aggregator
receive-pipeline

View file

@ -9,6 +9,7 @@ Networking
../scala/cluster-singleton
../scala/distributed-pub-sub
../scala/cluster-client
../scala/cluster-sharding
cluster-metrics
remoting
serialization

View file

@ -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``.

View file

@ -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``

View file

@ -9,6 +9,7 @@ Networking
cluster-singleton
distributed-pub-sub
cluster-client
cluster-sharding
cluster-metrics
remoting
serialization

View file

@ -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",

View file

@ -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)

View file

@ -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()))