* add withDataCenter in Entity, following same patterna as the role * update cluster-dc.md, split in classic and new pages * fix bug in ClusterSharding shouldHostShard * contains on String * update multi-dc singleton sample
This commit is contained in:
parent
c83d04c1f8
commit
46fcca5f39
18 changed files with 480 additions and 207 deletions
|
|
@ -283,7 +283,7 @@ final class ClusterShardingSettings(
|
||||||
@InternalApi
|
@InternalApi
|
||||||
private[akka] def shouldHostShard(cluster: Cluster): Boolean =
|
private[akka] def shouldHostShard(cluster: Cluster): Boolean =
|
||||||
role.forall(cluster.selfMember.roles.contains) &&
|
role.forall(cluster.selfMember.roles.contains) &&
|
||||||
dataCenter.forall(cluster.selfMember.dataCenter.contains)
|
dataCenter.forall(_ == cluster.selfMember.dataCenter)
|
||||||
|
|
||||||
// no withNumberOfShards because it should be defined in configuration to be able to verify same
|
// no withNumberOfShards because it should be defined in configuration to be able to verify same
|
||||||
// value on all nodes with `JoinConfigCompatChecker`
|
// value on all nodes with `JoinConfigCompatChecker`
|
||||||
|
|
|
||||||
|
|
@ -11,8 +11,10 @@ import java.util.concurrent.CompletionStage
|
||||||
import java.util.concurrent.ConcurrentHashMap
|
import java.util.concurrent.ConcurrentHashMap
|
||||||
|
|
||||||
import scala.compat.java8.FutureConverters._
|
import scala.compat.java8.FutureConverters._
|
||||||
|
|
||||||
import akka.util.JavaDurationConverters._
|
import akka.util.JavaDurationConverters._
|
||||||
import scala.concurrent.Future
|
import scala.concurrent.Future
|
||||||
|
|
||||||
import akka.actor.ActorRefProvider
|
import akka.actor.ActorRefProvider
|
||||||
import akka.actor.ExtendedActorSystem
|
import akka.actor.ExtendedActorSystem
|
||||||
import akka.actor.InternalActorRef
|
import akka.actor.InternalActorRef
|
||||||
|
|
@ -28,6 +30,7 @@ import akka.actor.typed.internal.adapter.ActorRefAdapter
|
||||||
import akka.actor.typed.internal.adapter.ActorSystemAdapter
|
import akka.actor.typed.internal.adapter.ActorSystemAdapter
|
||||||
import akka.actor.typed.scaladsl.Behaviors
|
import akka.actor.typed.scaladsl.Behaviors
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
|
import akka.cluster.ClusterSettings.DataCenter
|
||||||
import akka.cluster.sharding.ShardCoordinator.LeastShardAllocationStrategy
|
import akka.cluster.sharding.ShardCoordinator.LeastShardAllocationStrategy
|
||||||
import akka.cluster.sharding.ShardCoordinator.ShardAllocationStrategy
|
import akka.cluster.sharding.ShardCoordinator.ShardAllocationStrategy
|
||||||
import akka.cluster.sharding.ShardRegion
|
import akka.cluster.sharding.ShardRegion
|
||||||
|
|
@ -119,12 +122,15 @@ import akka.util.Timeout
|
||||||
case Some(e) => e
|
case Some(e) => e
|
||||||
}).asInstanceOf[ShardingMessageExtractor[E, M]]
|
}).asInstanceOf[ShardingMessageExtractor[E, M]]
|
||||||
|
|
||||||
|
val settingsWithRole = entity.role.fold(settings)(settings.withRole)
|
||||||
|
val settingsWithDataCenter = entity.dataCenter.fold(settingsWithRole)(settingsWithRole.withDataCenter)
|
||||||
|
|
||||||
internalInit(
|
internalInit(
|
||||||
entity.createBehavior,
|
entity.createBehavior,
|
||||||
entity.entityProps,
|
entity.entityProps,
|
||||||
entity.typeKey,
|
entity.typeKey,
|
||||||
entity.stopMessage,
|
entity.stopMessage,
|
||||||
entity.role.fold(settings)(settings.withRole),
|
settingsWithDataCenter,
|
||||||
extractor,
|
extractor,
|
||||||
entity.allocationStrategy)
|
entity.allocationStrategy)
|
||||||
}
|
}
|
||||||
|
|
@ -142,7 +148,8 @@ import akka.util.Timeout
|
||||||
settings = entity.settings.asScala,
|
settings = entity.settings.asScala,
|
||||||
messageExtractor = entity.messageExtractor.asScala,
|
messageExtractor = entity.messageExtractor.asScala,
|
||||||
allocationStrategy = entity.allocationStrategy.asScala,
|
allocationStrategy = entity.allocationStrategy.asScala,
|
||||||
role = entity.role.asScala))
|
role = entity.role.asScala,
|
||||||
|
dataCenter = entity.dataCenter.asScala))
|
||||||
}
|
}
|
||||||
|
|
||||||
private def internalInit[M, E](
|
private def internalInit[M, E](
|
||||||
|
|
@ -240,6 +247,19 @@ import akka.util.Timeout
|
||||||
typeKey.asInstanceOf[EntityTypeKeyImpl[M]])
|
typeKey.asInstanceOf[EntityTypeKeyImpl[M]])
|
||||||
}
|
}
|
||||||
|
|
||||||
|
override def entityRefFor[M](
|
||||||
|
typeKey: scaladsl.EntityTypeKey[M],
|
||||||
|
entityId: String,
|
||||||
|
dataCenter: DataCenter): scaladsl.EntityRef[M] = {
|
||||||
|
if (dataCenter == cluster.selfMember.dataCenter)
|
||||||
|
entityRefFor(typeKey, entityId)
|
||||||
|
else
|
||||||
|
new EntityRefImpl[M](
|
||||||
|
classicSharding.shardRegionProxy(typeKey.name, dataCenter),
|
||||||
|
entityId,
|
||||||
|
typeKey.asInstanceOf[EntityTypeKeyImpl[M]])
|
||||||
|
}
|
||||||
|
|
||||||
override def entityRefFor[M](typeKey: javadsl.EntityTypeKey[M], entityId: String): javadsl.EntityRef[M] = {
|
override def entityRefFor[M](typeKey: javadsl.EntityTypeKey[M], entityId: String): javadsl.EntityRef[M] = {
|
||||||
new EntityRefImpl[M](
|
new EntityRefImpl[M](
|
||||||
classicSharding.shardRegion(typeKey.name),
|
classicSharding.shardRegion(typeKey.name),
|
||||||
|
|
@ -247,6 +267,19 @@ import akka.util.Timeout
|
||||||
typeKey.asInstanceOf[EntityTypeKeyImpl[M]])
|
typeKey.asInstanceOf[EntityTypeKeyImpl[M]])
|
||||||
}
|
}
|
||||||
|
|
||||||
|
override def entityRefFor[M](
|
||||||
|
typeKey: javadsl.EntityTypeKey[M],
|
||||||
|
entityId: String,
|
||||||
|
dataCenter: String): javadsl.EntityRef[M] = {
|
||||||
|
if (dataCenter == cluster.selfMember.dataCenter)
|
||||||
|
entityRefFor(typeKey, entityId)
|
||||||
|
else
|
||||||
|
new EntityRefImpl[M](
|
||||||
|
classicSharding.shardRegionProxy(typeKey.name, dataCenter),
|
||||||
|
entityId,
|
||||||
|
typeKey.asInstanceOf[EntityTypeKeyImpl[M]])
|
||||||
|
}
|
||||||
|
|
||||||
override def defaultShardAllocationStrategy(settings: ClusterShardingSettings): ShardAllocationStrategy = {
|
override def defaultShardAllocationStrategy(settings: ClusterShardingSettings): ShardAllocationStrategy = {
|
||||||
val threshold = settings.tuningParameters.leastShardAllocationRebalanceThreshold
|
val threshold = settings.tuningParameters.leastShardAllocationRebalanceThreshold
|
||||||
val maxSimultaneousRebalance = settings.tuningParameters.leastShardAllocationMaxSimultaneousRebalance
|
val maxSimultaneousRebalance = settings.tuningParameters.leastShardAllocationMaxSimultaneousRebalance
|
||||||
|
|
|
||||||
|
|
@ -178,7 +178,8 @@ abstract class ClusterSharding {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create an `ActorRef`-like reference to a specific sharded entity.
|
* Create an `ActorRef`-like reference to a specific sharded entity.
|
||||||
* Currently you have to correctly specify the type of messages the target can handle.
|
*
|
||||||
|
* You have to correctly specify the type of messages the target can handle via the `typeKey`.
|
||||||
*
|
*
|
||||||
* Messages sent through this [[EntityRef]] will be wrapped in a [[ShardingEnvelope]] including the
|
* Messages sent through this [[EntityRef]] will be wrapped in a [[ShardingEnvelope]] including the
|
||||||
* here provided `entityId`.
|
* here provided `entityId`.
|
||||||
|
|
@ -187,6 +188,18 @@ abstract class ClusterSharding {
|
||||||
*/
|
*/
|
||||||
def entityRefFor[M](typeKey: EntityTypeKey[M], entityId: String): EntityRef[M]
|
def entityRefFor[M](typeKey: EntityTypeKey[M], entityId: String): EntityRef[M]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create an `ActorRef`-like reference to a specific sharded entity running in another data center.
|
||||||
|
*
|
||||||
|
* You have to correctly specify the type of messages the target can handle via the `typeKey`.
|
||||||
|
*
|
||||||
|
* Messages sent through this [[EntityRef]] will be wrapped in a [[ShardingEnvelope]] including the
|
||||||
|
* provided `entityId`.
|
||||||
|
*
|
||||||
|
* For in-depth documentation of its semantics, see [[EntityRef]].
|
||||||
|
*/
|
||||||
|
def entityRefFor[M](typeKey: EntityTypeKey[M], entityId: String, dataCenter: String): EntityRef[M]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Actor for querying Cluster Sharding state
|
* Actor for querying Cluster Sharding state
|
||||||
*/
|
*/
|
||||||
|
|
@ -220,6 +233,7 @@ object Entity {
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
|
Optional.empty(),
|
||||||
Optional.empty())
|
Optional.empty())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -236,7 +250,8 @@ final class Entity[M, E] private (
|
||||||
val settings: Optional[ClusterShardingSettings],
|
val settings: Optional[ClusterShardingSettings],
|
||||||
val messageExtractor: Optional[ShardingMessageExtractor[E, M]],
|
val messageExtractor: Optional[ShardingMessageExtractor[E, M]],
|
||||||
val allocationStrategy: Optional[ShardAllocationStrategy],
|
val allocationStrategy: Optional[ShardAllocationStrategy],
|
||||||
val role: Optional[String]) {
|
val role: Optional[String],
|
||||||
|
val dataCenter: Optional[String]) {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* [[akka.actor.typed.Props]] of the entity actors, such as dispatcher settings.
|
* [[akka.actor.typed.Props]] of the entity actors, such as dispatcher settings.
|
||||||
|
|
@ -276,7 +291,8 @@ final class Entity[M, E] private (
|
||||||
settings,
|
settings,
|
||||||
Optional.ofNullable(newExtractor),
|
Optional.ofNullable(newExtractor),
|
||||||
allocationStrategy,
|
allocationStrategy,
|
||||||
role)
|
role,
|
||||||
|
dataCenter)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Run the Entity actors on nodes with the given role.
|
* Run the Entity actors on nodes with the given role.
|
||||||
|
|
@ -284,6 +300,14 @@ final class Entity[M, E] private (
|
||||||
def withRole(role: String): Entity[M, E] =
|
def withRole(role: String): Entity[M, E] =
|
||||||
copy(role = Optional.ofNullable(role))
|
copy(role = Optional.ofNullable(role))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The data center of the cluster nodes where the cluster sharding is running.
|
||||||
|
* If the dataCenter is not specified then the same data center as current node. If the given
|
||||||
|
* dataCenter does not match the data center of the current node the `ShardRegion` will be started
|
||||||
|
* in proxy mode.
|
||||||
|
*/
|
||||||
|
def withDataCenter(newDataCenter: String): Entity[M, E] = copy(dataCenter = Optional.ofNullable(newDataCenter))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Allocation strategy which decides on which nodes to allocate new shards,
|
* Allocation strategy which decides on which nodes to allocate new shards,
|
||||||
* [[ClusterSharding#defaultShardAllocationStrategy]] is used if this is not specified.
|
* [[ClusterSharding#defaultShardAllocationStrategy]] is used if this is not specified.
|
||||||
|
|
@ -298,8 +322,18 @@ final class Entity[M, E] private (
|
||||||
entityProps: Props = entityProps,
|
entityProps: Props = entityProps,
|
||||||
settings: Optional[ClusterShardingSettings] = settings,
|
settings: Optional[ClusterShardingSettings] = settings,
|
||||||
allocationStrategy: Optional[ShardAllocationStrategy] = allocationStrategy,
|
allocationStrategy: Optional[ShardAllocationStrategy] = allocationStrategy,
|
||||||
role: Optional[String] = role): Entity[M, E] = {
|
role: Optional[String] = role,
|
||||||
new Entity(createBehavior, typeKey, stopMessage, entityProps, settings, messageExtractor, allocationStrategy, role)
|
dataCenter: Optional[String] = role): Entity[M, E] = {
|
||||||
|
new Entity(
|
||||||
|
createBehavior,
|
||||||
|
typeKey,
|
||||||
|
stopMessage,
|
||||||
|
entityProps,
|
||||||
|
settings,
|
||||||
|
messageExtractor,
|
||||||
|
allocationStrategy,
|
||||||
|
role,
|
||||||
|
dataCenter)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -20,6 +20,7 @@ import akka.actor.typed.Props
|
||||||
import akka.actor.typed.internal.InternalRecipientRef
|
import akka.actor.typed.internal.InternalRecipientRef
|
||||||
import akka.annotation.DoNotInherit
|
import akka.annotation.DoNotInherit
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
|
import akka.cluster.ClusterSettings.DataCenter
|
||||||
import akka.cluster.sharding.ShardCoordinator.ShardAllocationStrategy
|
import akka.cluster.sharding.ShardCoordinator.ShardAllocationStrategy
|
||||||
import akka.cluster.sharding.typed.internal.ClusterShardingImpl
|
import akka.cluster.sharding.typed.internal.ClusterShardingImpl
|
||||||
import akka.cluster.sharding.typed.internal.EntityTypeKeyImpl
|
import akka.cluster.sharding.typed.internal.EntityTypeKeyImpl
|
||||||
|
|
@ -178,7 +179,8 @@ trait ClusterSharding extends Extension { javadslSelf: javadsl.ClusterSharding =
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create an `ActorRef`-like reference to a specific sharded entity.
|
* Create an `ActorRef`-like reference to a specific sharded entity.
|
||||||
* Currently you have to correctly specify the type of messages the target can handle.
|
*
|
||||||
|
* You have to correctly specify the type of messages the target can handle via the `typeKey`.
|
||||||
*
|
*
|
||||||
* Messages sent through this [[EntityRef]] will be wrapped in a [[ShardingEnvelope]] including the
|
* Messages sent through this [[EntityRef]] will be wrapped in a [[ShardingEnvelope]] including the
|
||||||
* here provided `entityId`.
|
* here provided `entityId`.
|
||||||
|
|
@ -187,6 +189,18 @@ trait ClusterSharding extends Extension { javadslSelf: javadsl.ClusterSharding =
|
||||||
*/
|
*/
|
||||||
def entityRefFor[M](typeKey: EntityTypeKey[M], entityId: String): EntityRef[M]
|
def entityRefFor[M](typeKey: EntityTypeKey[M], entityId: String): EntityRef[M]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create an `ActorRef`-like reference to a specific sharded entity running in another data center.
|
||||||
|
*
|
||||||
|
* You have to correctly specify the type of messages the target can handle via the `typeKey`.
|
||||||
|
*
|
||||||
|
* Messages sent through this [[EntityRef]] will be wrapped in a [[ShardingEnvelope]] including the
|
||||||
|
* here provided `entityId`.
|
||||||
|
*
|
||||||
|
* For in-depth documentation of its semantics, see [[EntityRef]].
|
||||||
|
*/
|
||||||
|
def entityRefFor[M](typeKey: EntityTypeKey[M], entityId: String, dataCenter: DataCenter): EntityRef[M]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Actor for querying Cluster Sharding state
|
* Actor for querying Cluster Sharding state
|
||||||
*/
|
*/
|
||||||
|
|
@ -217,7 +231,7 @@ object Entity {
|
||||||
*/
|
*/
|
||||||
def apply[M](typeKey: EntityTypeKey[M])(
|
def apply[M](typeKey: EntityTypeKey[M])(
|
||||||
createBehavior: EntityContext[M] => Behavior[M]): Entity[M, ShardingEnvelope[M]] =
|
createBehavior: EntityContext[M] => Behavior[M]): Entity[M, ShardingEnvelope[M]] =
|
||||||
new Entity(createBehavior, typeKey, None, Props.empty, None, None, None, None)
|
new Entity(createBehavior, typeKey, None, Props.empty, None, None, None, None, None)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -231,7 +245,8 @@ final class Entity[M, E] private[akka] (
|
||||||
val settings: Option[ClusterShardingSettings],
|
val settings: Option[ClusterShardingSettings],
|
||||||
val messageExtractor: Option[ShardingMessageExtractor[E, M]],
|
val messageExtractor: Option[ShardingMessageExtractor[E, M]],
|
||||||
val allocationStrategy: Option[ShardAllocationStrategy],
|
val allocationStrategy: Option[ShardAllocationStrategy],
|
||||||
val role: Option[String]) {
|
val role: Option[String],
|
||||||
|
val dataCenter: Option[DataCenter]) {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* [[akka.actor.typed.Props]] of the entity actors, such as dispatcher settings.
|
* [[akka.actor.typed.Props]] of the entity actors, such as dispatcher settings.
|
||||||
|
|
@ -271,7 +286,8 @@ final class Entity[M, E] private[akka] (
|
||||||
settings,
|
settings,
|
||||||
Option(newExtractor),
|
Option(newExtractor),
|
||||||
allocationStrategy,
|
allocationStrategy,
|
||||||
role)
|
role,
|
||||||
|
dataCenter)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Allocation strategy which decides on which nodes to allocate new shards,
|
* Allocation strategy which decides on which nodes to allocate new shards,
|
||||||
|
|
@ -283,7 +299,15 @@ final class Entity[M, E] private[akka] (
|
||||||
/**
|
/**
|
||||||
* Run the Entity actors on nodes with the given role.
|
* Run the Entity actors on nodes with the given role.
|
||||||
*/
|
*/
|
||||||
def withRole(role: String): Entity[M, E] = copy(role = Some(role))
|
def withRole(newRole: String): Entity[M, E] = copy(role = Some(newRole))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The data center of the cluster nodes where the cluster sharding is running.
|
||||||
|
* If the dataCenter is not specified then the same data center as current node. If the given
|
||||||
|
* dataCenter does not match the data center of the current node the `ShardRegion` will be started
|
||||||
|
* in proxy mode.
|
||||||
|
*/
|
||||||
|
def withDataCenter(newDataCenter: DataCenter): Entity[M, E] = copy(dataCenter = Some(newDataCenter))
|
||||||
|
|
||||||
private def copy(
|
private def copy(
|
||||||
createBehavior: EntityContext[M] => Behavior[M] = createBehavior,
|
createBehavior: EntityContext[M] => Behavior[M] = createBehavior,
|
||||||
|
|
@ -292,8 +316,18 @@ final class Entity[M, E] private[akka] (
|
||||||
entityProps: Props = entityProps,
|
entityProps: Props = entityProps,
|
||||||
settings: Option[ClusterShardingSettings] = settings,
|
settings: Option[ClusterShardingSettings] = settings,
|
||||||
allocationStrategy: Option[ShardAllocationStrategy] = allocationStrategy,
|
allocationStrategy: Option[ShardAllocationStrategy] = allocationStrategy,
|
||||||
role: Option[String] = role): Entity[M, E] = {
|
role: Option[String] = role,
|
||||||
new Entity(createBehavior, typeKey, stopMessage, entityProps, settings, messageExtractor, allocationStrategy, role)
|
dataCenter: Option[DataCenter] = dataCenter): Entity[M, E] = {
|
||||||
|
new Entity(
|
||||||
|
createBehavior,
|
||||||
|
typeKey,
|
||||||
|
stopMessage,
|
||||||
|
entityProps,
|
||||||
|
settings,
|
||||||
|
messageExtractor,
|
||||||
|
allocationStrategy,
|
||||||
|
role,
|
||||||
|
dataCenter)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -9,7 +9,7 @@ import akka.actor.typed.ActorRef
|
||||||
import akka.cluster.sharding.typed.scaladsl.EntityTypeKey
|
import akka.cluster.sharding.typed.scaladsl.EntityTypeKey
|
||||||
import akka.cluster.sharding.typed.scaladsl.ClusterSharding
|
import akka.cluster.sharding.typed.scaladsl.ClusterSharding
|
||||||
import akka.cluster.sharding.typed.scaladsl.Entity
|
import akka.cluster.sharding.typed.scaladsl.Entity
|
||||||
import akka.cluster.typed.{ MultiDcClusterActors, MultiNodeTypedClusterSpec }
|
import akka.cluster.typed.{ MultiDcPinger, MultiNodeTypedClusterSpec }
|
||||||
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
|
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec }
|
||||||
import akka.actor.testkit.typed.scaladsl.TestProbe
|
import akka.actor.testkit.typed.scaladsl.TestProbe
|
||||||
import akka.cluster.MultiNodeClusterSpec
|
import akka.cluster.MultiNodeClusterSpec
|
||||||
|
|
@ -54,9 +54,9 @@ abstract class MultiDcClusterShardingSpec
|
||||||
with ScalaFutures {
|
with ScalaFutures {
|
||||||
|
|
||||||
import MultiDcClusterShardingSpecConfig._
|
import MultiDcClusterShardingSpecConfig._
|
||||||
import MultiDcClusterActors._
|
import MultiDcPinger._
|
||||||
|
|
||||||
val typeKey = EntityTypeKey[PingProtocol]("ping")
|
val typeKey = EntityTypeKey[Command]("ping")
|
||||||
val entityId = "ping-1"
|
val entityId = "ping-1"
|
||||||
|
|
||||||
"Cluster sharding in multi dc cluster" must {
|
"Cluster sharding in multi dc cluster" must {
|
||||||
|
|
@ -66,7 +66,7 @@ abstract class MultiDcClusterShardingSpec
|
||||||
|
|
||||||
"init sharding" in {
|
"init sharding" in {
|
||||||
val sharding = ClusterSharding(typedSystem)
|
val sharding = ClusterSharding(typedSystem)
|
||||||
val shardRegion: ActorRef[ShardingEnvelope[PingProtocol]] = sharding.init(Entity(typeKey)(_ => multiDcPinger))
|
val shardRegion: ActorRef[ShardingEnvelope[Command]] = sharding.init(Entity(typeKey)(_ => MultiDcPinger()))
|
||||||
val probe = TestProbe[Pong]
|
val probe = TestProbe[Pong]
|
||||||
shardRegion ! ShardingEnvelope(entityId, Ping(probe.ref))
|
shardRegion ! ShardingEnvelope(entityId, Ping(probe.ref))
|
||||||
probe.expectMessage(max = 15.seconds, Pong(cluster.selfMember.dataCenter))
|
probe.expectMessage(max = 15.seconds, Pong(cluster.selfMember.dataCenter))
|
||||||
|
|
@ -90,14 +90,45 @@ abstract class MultiDcClusterShardingSpec
|
||||||
enterBarrier("ask")
|
enterBarrier("ask")
|
||||||
}
|
}
|
||||||
|
|
||||||
"be able to message cross dc via proxy" in {
|
"be able to message cross dc via proxy, defined with ClusterShardingSettings" in {
|
||||||
runOn(first, second) {
|
runOn(first, second) {
|
||||||
val proxy: ActorRef[ShardingEnvelope[PingProtocol]] = ClusterSharding(typedSystem).init(
|
val proxy: ActorRef[ShardingEnvelope[Command]] = ClusterSharding(typedSystem).init(
|
||||||
Entity(typeKey)(_ => multiDcPinger).withSettings(ClusterShardingSettings(typedSystem).withDataCenter("dc2")))
|
Entity(typeKey)(_ => MultiDcPinger()).withSettings(ClusterShardingSettings(typedSystem).withDataCenter("dc2")))
|
||||||
val probe = TestProbe[Pong]
|
val probe = TestProbe[Pong]
|
||||||
proxy ! ShardingEnvelope(entityId, Ping(probe.ref))
|
proxy ! ShardingEnvelope(entityId, Ping(probe.ref))
|
||||||
probe.expectMessage(remainingOrDefault, Pong("dc2"))
|
probe.expectMessage(remainingOrDefault, Pong("dc2"))
|
||||||
}
|
}
|
||||||
enterBarrier("done")
|
enterBarrier("cross-dc-1")
|
||||||
|
}
|
||||||
|
|
||||||
|
"be able to message cross dc via proxy, defined with Entity" in {
|
||||||
|
runOn(first, second) {
|
||||||
|
val system = typedSystem
|
||||||
|
//#proxy-dc
|
||||||
|
val proxy: ActorRef[ShardingEnvelope[Command]] =
|
||||||
|
ClusterSharding(system).init(Entity(typeKey)(_ => MultiDcPinger()).withDataCenter("dc2"))
|
||||||
|
//#proxy-dc
|
||||||
|
val probe = TestProbe[Pong]
|
||||||
|
proxy ! ShardingEnvelope(entityId, Ping(probe.ref))
|
||||||
|
probe.expectMessage(remainingOrDefault, Pong("dc2"))
|
||||||
|
}
|
||||||
|
enterBarrier("cross-dc-2")
|
||||||
|
}
|
||||||
|
|
||||||
|
"be able to message cross dc via proxy, defined with EntityRef" in {
|
||||||
|
runOn(first, second) {
|
||||||
|
val system = typedSystem
|
||||||
|
//#proxy-dc-entityref
|
||||||
|
// it must still be started before usage
|
||||||
|
ClusterSharding(system).init(Entity(typeKey)(_ => MultiDcPinger()).withDataCenter("dc2"))
|
||||||
|
|
||||||
|
val entityRef = ClusterSharding(system).entityRefFor(typeKey, entityId, "dc2")
|
||||||
|
//#proxy-dc-entityref
|
||||||
|
|
||||||
|
val probe = TestProbe[Pong]
|
||||||
|
entityRef ! Ping(probe.ref)
|
||||||
|
probe.expectMessage(remainingOrDefault, Pong("dc2"))
|
||||||
|
}
|
||||||
|
enterBarrier("cross-dc-3")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -226,4 +226,26 @@ interface ShardingCompileOnlyTest {
|
||||||
entityContext.getEntityTypeKey().name(), entityContext.getEntityId()))));
|
entityContext.getEntityTypeKey().name(), entityContext.getEntityId()))));
|
||||||
// #persistence
|
// #persistence
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static void dataCenterExample() {
|
||||||
|
ActorSystem system = ActorSystem.create(Behaviors.empty(), "ShardingExample");
|
||||||
|
EntityTypeKey<Counter.Command> typeKey = EntityTypeKey.create(Counter.Command.class, "Counter");
|
||||||
|
String entityId = "a";
|
||||||
|
|
||||||
|
// #proxy-dc
|
||||||
|
ActorRef<ShardingEnvelope<Counter.Command>> proxy =
|
||||||
|
ClusterSharding.get(system)
|
||||||
|
.init(
|
||||||
|
Entity.of(typeKey, ctx -> Counter.create(ctx.getEntityId())).withDataCenter("dc2"));
|
||||||
|
// #proxy-dc
|
||||||
|
|
||||||
|
// #proxy-dc-entityref
|
||||||
|
// it must still be started before usage
|
||||||
|
ClusterSharding.get(system)
|
||||||
|
.init(Entity.of(typeKey, ctx -> Counter.create(ctx.getEntityId())).withDataCenter("dc2"));
|
||||||
|
|
||||||
|
EntityRef<Counter.Command> entityRef =
|
||||||
|
ClusterSharding.get(system).entityRefFor(typeKey, entityId, "dc2");
|
||||||
|
// #proxy-dc-entityref
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -242,7 +242,7 @@ class ClusterSingletonManagerSpec
|
||||||
val proxyDcB = system.actorOf(
|
val proxyDcB = system.actorOf(
|
||||||
ClusterSingletonProxy.props(
|
ClusterSingletonProxy.props(
|
||||||
singletonManagerPath = "/user/consumer",
|
singletonManagerPath = "/user/consumer",
|
||||||
settings = ClusterSingletonProxySettings(system).withRole("worker").withDataCenter("B")),
|
settings = ClusterSingletonProxySettings(system).withDataCenter("B")),
|
||||||
name = "consumerProxyDcB")
|
name = "consumerProxyDcB")
|
||||||
//#create-singleton-proxy-dc
|
//#create-singleton-proxy-dc
|
||||||
proxyDcB
|
proxyDcB
|
||||||
|
|
|
||||||
|
|
@ -41,7 +41,7 @@ abstract class MultiDcClusterSingletonSpec
|
||||||
extends MultiNodeSpec(MultiDcClusterSingletonSpecConfig)
|
extends MultiNodeSpec(MultiDcClusterSingletonSpecConfig)
|
||||||
with MultiNodeTypedClusterSpec {
|
with MultiNodeTypedClusterSpec {
|
||||||
|
|
||||||
import MultiDcClusterActors._
|
import MultiDcPinger._
|
||||||
import MultiDcClusterSingletonSpecConfig._
|
import MultiDcClusterSingletonSpecConfig._
|
||||||
|
|
||||||
"A cluster with multiple data centers" must {
|
"A cluster with multiple data centers" must {
|
||||||
|
|
@ -64,7 +64,7 @@ abstract class MultiDcClusterSingletonSpec
|
||||||
"be able to create and ping singleton in same DC" in {
|
"be able to create and ping singleton in same DC" in {
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
val singleton = ClusterSingleton(typedSystem)
|
val singleton = ClusterSingleton(typedSystem)
|
||||||
val pinger = singleton.init(SingletonActor(multiDcPinger, "ping").withStopMessage(NoMore))
|
val pinger = singleton.init(SingletonActor(MultiDcPinger(), "ping").withStopMessage(NoMore))
|
||||||
val probe = TestProbe[Pong]
|
val probe = TestProbe[Pong]
|
||||||
pinger ! Ping(probe.ref)
|
pinger ! Ping(probe.ref)
|
||||||
probe.expectMessage(Pong("dc1"))
|
probe.expectMessage(Pong("dc1"))
|
||||||
|
|
@ -79,7 +79,7 @@ abstract class MultiDcClusterSingletonSpec
|
||||||
runOn(second) {
|
runOn(second) {
|
||||||
val singleton = ClusterSingleton(system.toTyped)
|
val singleton = ClusterSingleton(system.toTyped)
|
||||||
val pinger = singleton.init(
|
val pinger = singleton.init(
|
||||||
SingletonActor(multiDcPinger, "ping")
|
SingletonActor(MultiDcPinger(), "ping")
|
||||||
.withStopMessage(NoMore)
|
.withStopMessage(NoMore)
|
||||||
.withSettings(ClusterSingletonSettings(typedSystem).withDataCenter("dc1")))
|
.withSettings(ClusterSingletonSettings(typedSystem).withDataCenter("dc1")))
|
||||||
val probe = TestProbe[Pong]
|
val probe = TestProbe[Pong]
|
||||||
|
|
@ -93,7 +93,7 @@ abstract class MultiDcClusterSingletonSpec
|
||||||
"be able to target singleton with the same name in own dc " in {
|
"be able to target singleton with the same name in own dc " in {
|
||||||
runOn(second, third) {
|
runOn(second, third) {
|
||||||
val singleton = ClusterSingleton(typedSystem)
|
val singleton = ClusterSingleton(typedSystem)
|
||||||
val pinger = singleton.init(SingletonActor(multiDcPinger, "ping").withStopMessage(NoMore))
|
val pinger = singleton.init(SingletonActor(MultiDcPinger(), "ping").withStopMessage(NoMore))
|
||||||
val probe = TestProbe[Pong]
|
val probe = TestProbe[Pong]
|
||||||
pinger ! Ping(probe.ref)
|
pinger ! Ping(probe.ref)
|
||||||
probe.expectMessage(Pong("dc2"))
|
probe.expectMessage(Pong("dc2"))
|
||||||
|
|
|
||||||
|
|
@ -5,18 +5,20 @@
|
||||||
package akka.cluster.typed
|
package akka.cluster.typed
|
||||||
|
|
||||||
import akka.actor.typed.ActorRef
|
import akka.actor.typed.ActorRef
|
||||||
|
import akka.actor.typed.Behavior
|
||||||
import akka.actor.typed.scaladsl.Behaviors
|
import akka.actor.typed.scaladsl.Behaviors
|
||||||
import akka.serialization.jackson.CborSerializable
|
import akka.serialization.jackson.CborSerializable
|
||||||
|
|
||||||
object MultiDcClusterActors {
|
object MultiDcPinger {
|
||||||
case class Pong(dc: String) extends CborSerializable
|
|
||||||
sealed trait PingProtocol extends CborSerializable
|
|
||||||
case class Ping(ref: ActorRef[Pong]) extends PingProtocol
|
|
||||||
case object NoMore extends PingProtocol
|
|
||||||
|
|
||||||
val multiDcPinger = Behaviors.setup[PingProtocol] { ctx =>
|
sealed trait Command extends CborSerializable
|
||||||
|
case class Ping(ref: ActorRef[Pong]) extends Command
|
||||||
|
case object NoMore extends Command
|
||||||
|
case class Pong(dc: String) extends CborSerializable
|
||||||
|
|
||||||
|
def apply(): Behavior[Command] = Behaviors.setup[Command] { ctx =>
|
||||||
val cluster = Cluster(ctx.system)
|
val cluster = Cluster(ctx.system)
|
||||||
Behaviors.receiveMessage[PingProtocol] {
|
Behaviors.receiveMessage[Command] {
|
||||||
case Ping(ref) =>
|
case Ping(ref) =>
|
||||||
ref ! Pong(cluster.selfMember.dataCenter)
|
ref ! Pong(cluster.selfMember.dataCenter)
|
||||||
Behaviors.same
|
Behaviors.same
|
||||||
|
|
@ -24,6 +24,7 @@ import com.typesafe.config.ConfigFactory;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
// FIXME use awaitAssert to await cluster forming like in BasicClusterExampleSpec
|
// FIXME use awaitAssert to await cluster forming like in BasicClusterExampleSpec
|
||||||
public class BasicClusterExampleTest { // extends JUnitSuite {
|
public class BasicClusterExampleTest { // extends JUnitSuite {
|
||||||
|
|
@ -146,4 +147,19 @@ public class BasicClusterExampleTest { // extends JUnitSuite {
|
||||||
}
|
}
|
||||||
// #hasRole
|
// #hasRole
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void illustrateDcAccess() {
|
||||||
|
ActorSystem<Void> system = null;
|
||||||
|
|
||||||
|
// #dcAccess
|
||||||
|
final Cluster cluster = Cluster.get(system);
|
||||||
|
// this node's data center
|
||||||
|
String dc = cluster.selfMember().dataCenter();
|
||||||
|
// all known data centers
|
||||||
|
Set<String> allDc = cluster.state().getAllDataCenters();
|
||||||
|
// a specific member's data center
|
||||||
|
Member aMember = cluster.state().getMembers().iterator().next();
|
||||||
|
String aDc = aMember.dataCenter();
|
||||||
|
// #dcAccess
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -14,6 +14,7 @@ import java.time.Duration;
|
||||||
|
|
||||||
// #import
|
// #import
|
||||||
import akka.cluster.typed.ClusterSingleton;
|
import akka.cluster.typed.ClusterSingleton;
|
||||||
|
import akka.cluster.typed.ClusterSingletonSettings;
|
||||||
import akka.cluster.typed.SingletonActor;
|
import akka.cluster.typed.SingletonActor;
|
||||||
|
|
||||||
// #import
|
// #import
|
||||||
|
|
@ -117,4 +118,15 @@ public interface SingletonCompileOnlyTest {
|
||||||
// #backoff
|
// #backoff
|
||||||
proxy.tell(Counter.Increment.INSTANCE); // avoid unused warning
|
proxy.tell(Counter.Increment.INSTANCE); // avoid unused warning
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static void dcProxy() {
|
||||||
|
// #create-singleton-proxy-dc
|
||||||
|
ActorRef<Counter.Command> singletonProxy =
|
||||||
|
ClusterSingleton.get(system)
|
||||||
|
.init(
|
||||||
|
SingletonActor.of(Counter.create(), "GlobalCounter")
|
||||||
|
.withSettings(ClusterSingletonSettings.create(system).withDataCenter("B")));
|
||||||
|
// #create-singleton-proxy-dc
|
||||||
|
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -6,6 +6,7 @@ package docs.akka.cluster.typed
|
||||||
|
|
||||||
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
||||||
import akka.testkit.SocketUtil
|
import akka.testkit.SocketUtil
|
||||||
|
import com.github.ghik.silencer.silent
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import org.scalatest.{ Matchers, WordSpec }
|
import org.scalatest.{ Matchers, WordSpec }
|
||||||
//#cluster-imports
|
//#cluster-imports
|
||||||
|
|
@ -83,6 +84,22 @@ akka {
|
||||||
}
|
}
|
||||||
//#hasRole
|
//#hasRole
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@silent("never used")
|
||||||
|
def illustrateDcAccess(): Unit = {
|
||||||
|
val system: ActorSystem[_] = ???
|
||||||
|
|
||||||
|
//#dcAccess
|
||||||
|
val cluster = Cluster(system)
|
||||||
|
// this node's data center
|
||||||
|
val dc = cluster.selfMember.dataCenter
|
||||||
|
// all known data centers
|
||||||
|
val allDc = cluster.state.allDataCenters
|
||||||
|
// a specific member's data center
|
||||||
|
val aMember = cluster.state.members.head
|
||||||
|
val aDc = aMember.dataCenter
|
||||||
|
//#dcAccess
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class BasicClusterConfigSpec extends WordSpec with ScalaFutures with Eventually with Matchers with LogCapturing {
|
class BasicClusterConfigSpec extends WordSpec with ScalaFutures with Eventually with Matchers with LogCapturing {
|
||||||
|
|
|
||||||
|
|
@ -6,9 +6,10 @@ package docs.akka.cluster.typed
|
||||||
|
|
||||||
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, SupervisorStrategy }
|
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, SupervisorStrategy }
|
||||||
import akka.actor.typed.scaladsl.Behaviors
|
import akka.actor.typed.scaladsl.Behaviors
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
|
import akka.cluster.typed.ClusterSingletonSettings
|
||||||
|
|
||||||
object SingletonCompileOnlySpec {
|
object SingletonCompileOnlySpec {
|
||||||
|
|
||||||
val system = ActorSystem(Behaviors.empty, "Singleton")
|
val system = ActorSystem(Behaviors.empty, "Singleton")
|
||||||
|
|
@ -64,4 +65,9 @@ object SingletonCompileOnlySpec {
|
||||||
.onFailure[Exception](SupervisorStrategy.restartWithBackoff(1.second, 10.seconds, 0.2)),
|
.onFailure[Exception](SupervisorStrategy.restartWithBackoff(1.second, 10.seconds, 0.2)),
|
||||||
"GlobalCounter"))
|
"GlobalCounter"))
|
||||||
//#backoff
|
//#backoff
|
||||||
|
|
||||||
|
//#create-singleton-proxy-dc
|
||||||
|
val singletonProxy: ActorRef[Counter.Command] = ClusterSingleton(system).init(
|
||||||
|
SingletonActor(Counter(), "GlobalCounter").withSettings(ClusterSingletonSettings(system).withDataCenter("dc2")))
|
||||||
|
//#create-singleton-proxy-dc
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -1,100 +1,12 @@
|
||||||
# Cluster across multiple data centers
|
# Classic Multi-DC Cluster
|
||||||
|
|
||||||
This chapter describes how @ref[Akka Cluster](cluster-usage.md) can be used across
|
This chapter describes how @ref[Akka Cluster](cluster-usage.md) can be used across
|
||||||
multiple data centers, availability zones or regions.
|
multiple data centers, availability zones or regions.
|
||||||
|
|
||||||
The reason for making the Akka Cluster aware of data center boundaries is that
|
For the full documentation of this feature and for new projects see @ref:[Multi-DC Cluster](typed/cluster-dc.md).
|
||||||
communication across data centers typically has much higher latency and higher failure
|
|
||||||
rate than communication between nodes in the same data center.
|
|
||||||
|
|
||||||
However, the grouping of nodes is not limited to the physical boundaries of data centers,
|
|
||||||
even though that is the primary use case. It could also be used as a logical grouping
|
|
||||||
for other reasons, such as isolation of certain nodes to improve stability or splitting
|
|
||||||
up a large cluster into smaller groups of nodes for better scalability.
|
|
||||||
|
|
||||||
## Motivation
|
|
||||||
|
|
||||||
There can be many reasons for using more than one data center, such as:
|
|
||||||
|
|
||||||
* Redundancy to tolerate failures in one location and still be operational.
|
|
||||||
* Serve requests from a location near the user to provide better responsiveness.
|
|
||||||
* Balance the load over many servers.
|
|
||||||
|
|
||||||
It's possible to run an ordinary Akka Cluster with default settings that spans multiple
|
|
||||||
data centers but that may result in problems like:
|
|
||||||
|
|
||||||
* Management of Cluster membership is stalled during network partitions as described in a
|
|
||||||
separate section below. This means that nodes would not be able to be added and removed
|
|
||||||
during network partitions between data centers.
|
|
||||||
* More frequent false positive failure detection for network connections across data centers.
|
|
||||||
It's not possible to have different settings for the failure detection within vs. across
|
|
||||||
data centers.
|
|
||||||
* Downing/removal of nodes in the case of network partitions should typically be treated
|
|
||||||
differently for failures within vs. across data centers. For network partitions between
|
|
||||||
data centers the system should typically not down the unreachable nodes, but instead wait until it heals or
|
|
||||||
a decision is made by a human or external monitoring system. For failures within same
|
|
||||||
data center automatic, more aggressive, downing mechanisms can be employed for quick fail over.
|
|
||||||
* Quick fail over of Cluster Singleton and Cluster Sharding from one data center to another
|
|
||||||
is difficult to do in a safe way. There is a risk that singletons or sharded entities become
|
|
||||||
active on both sides of a network partition.
|
|
||||||
* Lack of location information makes it difficult to optimize communication to prefer nodes
|
|
||||||
that are close over distant nodes. E.g. a cluster aware router would be more efficient
|
|
||||||
if it would prefer routing messages to nodes in the own data center.
|
|
||||||
|
|
||||||
To avoid some of these problems one can run a separate Akka Cluster per data center and use another
|
|
||||||
communication channel between the data centers, such as HTTP, an external message broker or
|
|
||||||
@ref[Cluster Client](cluster-singleton.md). However, many of the nice tools that are built on
|
|
||||||
top of the Cluster membership information are lost. For example, it wouldn't be possible
|
|
||||||
to use @ref[Distributed Data](distributed-data.md) across the separate clusters.
|
|
||||||
|
|
||||||
We often recommend implementing a micro-service as one Akka Cluster. The external API of the
|
|
||||||
service would be HTTP, gRPC or a message broker, and not Akka Remoting or Cluster (see additional discussion
|
|
||||||
in the Lagom Framework docs:
|
|
||||||
@scala[[Internal and External Communication](https://www.lagomframework.com/documentation/current/scala/InternalAndExternalCommunication.html)]
|
|
||||||
@java[[Internal and External Communication](https://www.lagomframework.com/documentation/current/java/InternalAndExternalCommunication.html)]),
|
|
||||||
but the internal communication within the service that is running on several nodes would use ordinary actor
|
|
||||||
messaging or the tools based on Akka Cluster. When deploying this service to multiple data
|
|
||||||
centers it would be inconvenient if the internal communication could not use ordinary actor
|
|
||||||
messaging because it was separated into several Akka Clusters. The benefit of using Akka
|
|
||||||
messaging internally is performance as well as ease of development and reasoning about
|
|
||||||
your domain in terms of Actors.
|
|
||||||
|
|
||||||
Therefore, it's possible to make the Akka Cluster aware of data centers so that one Akka
|
|
||||||
Cluster can span multiple data centers and still be tolerant to network partitions.
|
|
||||||
|
|
||||||
## Defining the data centers
|
|
||||||
|
|
||||||
The features are based on the idea that nodes can be assigned to a group of nodes
|
|
||||||
by setting the `akka.cluster.multi-data-center.self-data-center` configuration property.
|
|
||||||
A node can only belong to one data center and if nothing is specified a node will belong
|
|
||||||
to the `default` data center.
|
|
||||||
|
|
||||||
The grouping of nodes is not limited to the physical boundaries of data centers,
|
|
||||||
even though that is the primary use case. It could also be used as a logical grouping
|
|
||||||
for other reasons, such as isolation of certain nodes to improve stability or splitting
|
|
||||||
up a large cluster into smaller groups of nodes for better scalability.
|
|
||||||
|
|
||||||
## Membership
|
## Membership
|
||||||
|
|
||||||
Some @ref[membership transitions](typed/cluster-membership.md#membership-lifecycle) are managed by
|
|
||||||
one node called the @ref[leader](typed/cluster-concepts.md#leader). There is one leader per data center
|
|
||||||
and it is responsible for these transitions for the members within the same data center. Members of
|
|
||||||
other data centers are managed independently by the leader of the respective data center. These actions
|
|
||||||
cannot be performed while there are any unreachability observations among the nodes in the data center,
|
|
||||||
but unreachability across different data centers don't influence the progress of membership management
|
|
||||||
within a data center. Nodes can be added and removed also when there are network partitions between
|
|
||||||
data centers, which is impossible if nodes are not grouped into data centers.
|
|
||||||
|
|
||||||

|
|
||||||
|
|
||||||
User actions like joining, leaving, and downing can be sent to any node in the cluster,
|
|
||||||
not only to the nodes in the data center of the node. Seed nodes are also global.
|
|
||||||
|
|
||||||
The data center membership is implemented by adding the data center name prefixed with `"dc-"` to the
|
|
||||||
@ref[roles](typed/cluster.md#node-roles) of the member and thereby this information is known
|
|
||||||
by all other members in the cluster. This is an implementation detail, but it can be good to know
|
|
||||||
if you see this in log messages.
|
|
||||||
|
|
||||||
You can retrieve information about what data center a member belongs to:
|
You can retrieve information about what data center a member belongs to:
|
||||||
|
|
||||||
Scala
|
Scala
|
||||||
|
|
@ -103,58 +15,10 @@ Scala
|
||||||
Java
|
Java
|
||||||
: @@snip [ClusterDocTest.java](/akka-docs/src/test/java/jdocs/cluster/ClusterDocTest.java) { #dcAccess }
|
: @@snip [ClusterDocTest.java](/akka-docs/src/test/java/jdocs/cluster/ClusterDocTest.java) { #dcAccess }
|
||||||
|
|
||||||
## Failure Detection
|
For the full documentation of this feature and for new projects see @ref:[Multi-DC Cluster](typed/cluster-dc.md#membership).
|
||||||
|
|
||||||
@ref[Failure detection](typed/cluster-concepts.md#failure-detector) is performed by sending heartbeat messages
|
|
||||||
to detect if a node is unreachable. This is done more frequently and with more certainty among
|
|
||||||
the nodes in the same data center than across data centers. The failure detection across different data centers should
|
|
||||||
be interpreted as an indication of problem with the network link between the data centers.
|
|
||||||
|
|
||||||
Two different failure detectors can be configured for these two purposes:
|
|
||||||
|
|
||||||
* `akka.cluster.failure-detector` for failure detection within own data center
|
|
||||||
* `akka.cluster.multi-data-center.failure-detector` for failure detection across different data centers
|
|
||||||
|
|
||||||
When @ref[subscribing to cluster events](cluster-usage.md#cluster-subscriber) the `UnreachableMember` and
|
|
||||||
`ReachableMember` events are for observations within the own data center. The same data center as where the
|
|
||||||
subscription was registered.
|
|
||||||
|
|
||||||
For cross data center unreachability notifications you can subscribe to `UnreachableDataCenter` and `ReachableDataCenter`
|
|
||||||
events.
|
|
||||||
|
|
||||||
Heartbeat messages for failure detection across data centers are only performed between a number of the
|
|
||||||
oldest nodes on each side. The number of nodes is configured with `akka.cluster.multi-data-center.cross-data-center-connections`.
|
|
||||||
The reason for only using a limited number of nodes is to keep the number of connections across data
|
|
||||||
centers low. The same nodes are also used for the gossip protocol when disseminating the membership
|
|
||||||
information across data centers. Within a data center all nodes are involved in gossip and failure detection.
|
|
||||||
|
|
||||||
This influences how rolling upgrades should be performed. Don't stop all of the oldest that are used for gossip
|
|
||||||
at the same time. Stop one or a few at a time so that new nodes can take over the responsibility.
|
|
||||||
It's best to leave the oldest nodes until last.
|
|
||||||
|
|
||||||
See the @ref:[failure detector](typed/cluster.md#failure-detector) for more details.
|
|
||||||
|
|
||||||
## Cluster Singleton
|
## Cluster Singleton
|
||||||
|
|
||||||
The @ref[Cluster Singleton](cluster-singleton.md) is a singleton per data center. If you start the
|
|
||||||
`ClusterSingletonManager` on all nodes and you have defined 3 different data centers there will be
|
|
||||||
3 active singleton instances in the cluster, one in each data center. This is taken care of automatically,
|
|
||||||
but is important to be aware of. Designing the system for one singleton per data center makes it possible
|
|
||||||
for the system to be available also during network partitions between data centers.
|
|
||||||
|
|
||||||
The reason why the singleton is per data center and not global is that membership information is not
|
|
||||||
guaranteed to be consistent across data centers when using one leader per data center and that makes it
|
|
||||||
difficult to select a single global singleton.
|
|
||||||
|
|
||||||
If you need a global singleton you have to pick one data center to host that singleton and only start the
|
|
||||||
`ClusterSingletonManager` on nodes of that data center. If the data center is unreachable from another data center the
|
|
||||||
singleton is inaccessible, which is a reasonable trade-off when selecting consistency over availability.
|
|
||||||
|
|
||||||
The `ClusterSingletonProxy` is by default routing messages to the singleton in the own data center, but
|
|
||||||
it can be started with a `data-center` parameter in the `ClusterSingletonProxySettings` to define that
|
|
||||||
it should route messages to a singleton located in another data center. That is useful for example when
|
|
||||||
having a global singleton in one data center and accessing it from other data centers.
|
|
||||||
|
|
||||||
This is how to create a singleton proxy for a specific data center:
|
This is how to create a singleton proxy for a specific data center:
|
||||||
|
|
||||||
Scala
|
Scala
|
||||||
|
|
@ -166,32 +30,10 @@ Java
|
||||||
If using the own data center as the `withDataCenter` parameter that would be a proxy for the singleton in the own data center, which
|
If using the own data center as the `withDataCenter` parameter that would be a proxy for the singleton in the own data center, which
|
||||||
is also the default if `withDataCenter` is not given.
|
is also the default if `withDataCenter` is not given.
|
||||||
|
|
||||||
|
For the full documentation of this feature and for new projects see @ref:[Multi-DC Cluster](typed/cluster-dc.md#cluster-singleton).
|
||||||
|
|
||||||
## Cluster Sharding
|
## Cluster Sharding
|
||||||
|
|
||||||
The coordinator in @ref[Cluster Sharding](cluster-sharding.md) is a Cluster Singleton and therefore,
|
|
||||||
as explained above, Cluster Sharding is also per data center. Each data center will have its own coordinator
|
|
||||||
and regions, isolated from other data centers. If you start an entity type with the same name on all
|
|
||||||
nodes and you have defined 3 different data centers and then send messages to the same entity id to
|
|
||||||
sharding regions in all data centers you will end up with 3 active entity instances for that entity id,
|
|
||||||
one in each data center. This is because the region/coordinator is only aware of its own data center
|
|
||||||
and will activate the entity there. It's unaware of the existence of corresponding entities in the
|
|
||||||
other data centers.
|
|
||||||
|
|
||||||
Especially when used together with Akka Persistence that is based on the single-writer principle
|
|
||||||
it is important to avoid running the same entity at multiple locations at the same time with a
|
|
||||||
shared data store. That would result in corrupt data since the events stored by different instances
|
|
||||||
may be interleaved and would be interpreted differently in a later replay. For active active persistent
|
|
||||||
entities see Lightbend's [Multi-DC Persistence](https://doc.akka.io/docs/akka-enhancements/current/persistence-dc/index.html)
|
|
||||||
|
|
||||||
If you need global entities you have to pick one data center to host that entity type and only start
|
|
||||||
`ClusterSharding` on nodes of that data center. If the data center is unreachable from another data center the
|
|
||||||
entities are inaccessible, which is a reasonable trade-off when selecting consistency over availability.
|
|
||||||
|
|
||||||
The Cluster Sharding proxy is by default routing messages to the shard regions in their own data center, but
|
|
||||||
it can be started with a `data-center` parameter to define that it should route messages to a shard region
|
|
||||||
located in another data center. That is useful for example when having global entities in one data center and
|
|
||||||
accessing them from other data centers.
|
|
||||||
|
|
||||||
This is how to create a sharding proxy for a specific data center:
|
This is how to create a sharding proxy for a specific data center:
|
||||||
|
|
||||||
Scala
|
Scala
|
||||||
|
|
@ -200,9 +42,4 @@ Scala
|
||||||
Java
|
Java
|
||||||
: @@snip [ClusterShardingTest.java](/akka-docs/src/test/java/jdocs/sharding/ClusterShardingTest.java) { #proxy-dc }
|
: @@snip [ClusterShardingTest.java](/akka-docs/src/test/java/jdocs/sharding/ClusterShardingTest.java) { #proxy-dc }
|
||||||
|
|
||||||
Another way to manage global entities is to make sure that certain entity ids are located in
|
For the full documentation of this feature and for new projects see @ref:[Multi-DC Cluster](typed/cluster-dc.md#cluster-sharding).
|
||||||
only one data center by routing the messages to the right region. For example, the routing function
|
|
||||||
could be that odd entity ids are routed to data center A and even entity ids to data center B.
|
|
||||||
Before sending the message to the local region actor you make the decision of which data center it should
|
|
||||||
be routed to. Messages for another data center can be sent with a sharding proxy as explained above and
|
|
||||||
messages for the own data center are sent to the local region.
|
|
||||||
|
|
|
||||||
|
|
@ -15,6 +15,7 @@ For the new API see @ref[Cluster](typed/index-cluster.md).
|
||||||
* [cluster-sharding](cluster-sharding.md)
|
* [cluster-sharding](cluster-sharding.md)
|
||||||
* [cluster-metrics](cluster-metrics.md)
|
* [cluster-metrics](cluster-metrics.md)
|
||||||
* [distributed-data](distributed-data.md)
|
* [distributed-data](distributed-data.md)
|
||||||
|
* [cluster-dc](cluster-dc.md)
|
||||||
* [serialization](serialization-classic.md)
|
* [serialization](serialization-classic.md)
|
||||||
|
|
||||||
@@@
|
@@@
|
||||||
|
|
|
||||||
228
akka-docs/src/main/paradox/typed/cluster-dc.md
Normal file
228
akka-docs/src/main/paradox/typed/cluster-dc.md
Normal file
|
|
@ -0,0 +1,228 @@
|
||||||
|
# Multi-DC Cluster
|
||||||
|
|
||||||
|
@@@ note
|
||||||
|
For the Akka Classic documentation of this feature see @ref:[Classic Multi-DC Cluster](../cluster-dc.md)
|
||||||
|
@@@
|
||||||
|
|
||||||
|
This chapter describes how @ref[Akka Cluster](cluster.md) can be used across
|
||||||
|
multiple data centers, availability zones or regions.
|
||||||
|
|
||||||
|
The reason for making the Akka Cluster aware of data center boundaries is that
|
||||||
|
communication across data centers typically has much higher latency and higher failure
|
||||||
|
rate than communication between nodes in the same data center.
|
||||||
|
|
||||||
|
However, the grouping of nodes is not limited to the physical boundaries of data centers,
|
||||||
|
even though that is the primary use case. It could also be used as a logical grouping
|
||||||
|
for other reasons, such as isolation of certain nodes to improve stability or splitting
|
||||||
|
up a large cluster into smaller groups of nodes for better scalability.
|
||||||
|
|
||||||
|
## Dependency
|
||||||
|
|
||||||
|
To use Akka Cluster add the following dependency in your project:
|
||||||
|
|
||||||
|
@@dependency[sbt,Maven,Gradle] {
|
||||||
|
group=com.typesafe.akka
|
||||||
|
artifact=akka-cluster-typed_$scala.binary_version$
|
||||||
|
version=$akka.version$
|
||||||
|
}
|
||||||
|
|
||||||
|
## Motivation
|
||||||
|
|
||||||
|
There can be many reasons for using more than one data center, such as:
|
||||||
|
|
||||||
|
* Redundancy to tolerate failures in one location and still be operational.
|
||||||
|
* Serve requests from a location near the user to provide better responsiveness.
|
||||||
|
* Balance the load over many servers.
|
||||||
|
|
||||||
|
It's possible to run an ordinary Akka Cluster with default settings that spans multiple
|
||||||
|
data centers but that may result in problems like:
|
||||||
|
|
||||||
|
* Management of Cluster membership is stalled during network partitions as described in a
|
||||||
|
separate section below. This means that nodes would not be able to be added and removed
|
||||||
|
during network partitions between data centers.
|
||||||
|
* More frequent false positive failure detection for network connections across data centers.
|
||||||
|
It's not possible to have different settings for the failure detection within vs. across
|
||||||
|
data centers.
|
||||||
|
* Downing/removal of nodes in the case of network partitions should typically be treated
|
||||||
|
differently for failures within vs. across data centers. For network partitions between
|
||||||
|
data centers the system should typically not down the unreachable nodes, but instead wait until it heals or
|
||||||
|
a decision is made by a human or external monitoring system. For failures within same
|
||||||
|
data center automatic, more aggressive, downing mechanisms can be employed for quick fail over.
|
||||||
|
* Quick fail over of Cluster Singleton and Cluster Sharding from one data center to another
|
||||||
|
is difficult to do in a safe way. There is a risk that singletons or sharded entities become
|
||||||
|
active on both sides of a network partition.
|
||||||
|
* Lack of location information makes it difficult to optimize communication to prefer nodes
|
||||||
|
that are close over distant nodes. E.g. a cluster aware router would be more efficient
|
||||||
|
if it would prefer routing messages to nodes in the own data center.
|
||||||
|
|
||||||
|
To avoid some of these problems one can run a separate Akka Cluster per data center and use another
|
||||||
|
communication channel between the data centers, such as HTTP, an external message broker.
|
||||||
|
However, many of the nice tools that are built on top of the Cluster membership information are lost.
|
||||||
|
For example, it wouldn't be possible to use @ref[Distributed Data](distributed-data.md) across the separate clusters.
|
||||||
|
|
||||||
|
We often recommend implementing a micro-service as one Akka Cluster. The external API of the
|
||||||
|
service would be HTTP, gRPC or a message broker, and not Akka Remoting or Cluster (see additional discussion
|
||||||
|
in @ref:[When and where to use Akka Cluster](choosing-cluster.md)).
|
||||||
|
|
||||||
|
The internal communication within the service that is running on several nodes would use ordinary actor
|
||||||
|
messaging or the tools based on Akka Cluster. When deploying this service to multiple data
|
||||||
|
centers it would be inconvenient if the internal communication could not use ordinary actor
|
||||||
|
messaging because it was separated into several Akka Clusters. The benefit of using Akka
|
||||||
|
messaging internally is performance as well as ease of development and reasoning about
|
||||||
|
your domain in terms of Actors.
|
||||||
|
|
||||||
|
Therefore, it's possible to make the Akka Cluster aware of data centers so that one Akka
|
||||||
|
Cluster can span multiple data centers and still be tolerant to network partitions.
|
||||||
|
|
||||||
|
## Defining the data centers
|
||||||
|
|
||||||
|
The features are based on the idea that nodes can be assigned to a group of nodes
|
||||||
|
by setting the `akka.cluster.multi-data-center.self-data-center` configuration property.
|
||||||
|
A node can only belong to one data center and if nothing is specified a node will belong
|
||||||
|
to the `default` data center.
|
||||||
|
|
||||||
|
The grouping of nodes is not limited to the physical boundaries of data centers,
|
||||||
|
even though that is the primary use case. It could also be used as a logical grouping
|
||||||
|
for other reasons, such as isolation of certain nodes to improve stability or splitting
|
||||||
|
up a large cluster into smaller groups of nodes for better scalability.
|
||||||
|
|
||||||
|
## Membership
|
||||||
|
|
||||||
|
Some @ref[membership transitions](cluster-membership.md#membership-lifecycle) are managed by
|
||||||
|
one node called the @ref[leader](cluster-concepts.md#leader). There is one leader per data center
|
||||||
|
and it is responsible for these transitions for the members within the same data center. Members of
|
||||||
|
other data centers are managed independently by the leader of the respective data center. These actions
|
||||||
|
cannot be performed while there are any unreachability observations among the nodes in the data center,
|
||||||
|
but unreachability across different data centers don't influence the progress of membership management
|
||||||
|
within a data center. Nodes can be added and removed also when there are network partitions between
|
||||||
|
data centers, which is impossible if nodes are not grouped into data centers.
|
||||||
|
|
||||||
|

|
||||||
|
|
||||||
|
User actions like joining, leaving, and downing can be sent to any node in the cluster,
|
||||||
|
not only to the nodes in the data center of the node. Seed nodes are also global.
|
||||||
|
|
||||||
|
The data center membership is implemented by adding the data center name prefixed with `"dc-"` to the
|
||||||
|
@ref[roles](cluster.md#node-roles) of the member and thereby this information is known
|
||||||
|
by all other members in the cluster. This is an implementation detail, but it can be good to know
|
||||||
|
if you see this in log messages.
|
||||||
|
|
||||||
|
You can retrieve information about what data center a member belongs to:
|
||||||
|
|
||||||
|
Scala
|
||||||
|
: @@snip [BasicClusterExampleSpec.scala](/akka-cluster-typed/src/test/scala/docs/akka/cluster/typed/BasicClusterExampleSpec.scala) { #dcAccess }
|
||||||
|
|
||||||
|
Java
|
||||||
|
: @@snip [BasicClusterExampleTest.java](/akka-cluster-typed/src/test/java/jdocs/akka/cluster/typed/BasicClusterExampleTest.java) { #dcAccess }
|
||||||
|
|
||||||
|
## Failure Detection
|
||||||
|
|
||||||
|
@ref[Failure detection](cluster-concepts.md#failure-detector) is performed by sending heartbeat messages
|
||||||
|
to detect if a node is unreachable. This is done more frequently and with more certainty among
|
||||||
|
the nodes in the same data center than across data centers. The failure detection across different data centers should
|
||||||
|
be interpreted as an indication of problem with the network link between the data centers.
|
||||||
|
|
||||||
|
Two different failure detectors can be configured for these two purposes:
|
||||||
|
|
||||||
|
* `akka.cluster.failure-detector` for failure detection within own data center
|
||||||
|
* `akka.cluster.multi-data-center.failure-detector` for failure detection across different data centers
|
||||||
|
|
||||||
|
When @ref[subscribing to cluster events](cluster.md#cluster-subscriptions) the `UnreachableMember` and
|
||||||
|
`ReachableMember` events are for observations within the own data center. The same data center as where the
|
||||||
|
subscription was registered.
|
||||||
|
|
||||||
|
For cross data center unreachability notifications you can subscribe to `UnreachableDataCenter` and `ReachableDataCenter`
|
||||||
|
events.
|
||||||
|
|
||||||
|
Heartbeat messages for failure detection across data centers are only performed between a number of the
|
||||||
|
oldest nodes on each side. The number of nodes is configured with `akka.cluster.multi-data-center.cross-data-center-connections`.
|
||||||
|
The reason for only using a limited number of nodes is to keep the number of connections across data
|
||||||
|
centers low. The same nodes are also used for the gossip protocol when disseminating the membership
|
||||||
|
information across data centers. Within a data center all nodes are involved in gossip and failure detection.
|
||||||
|
|
||||||
|
This influences how rolling upgrades should be performed. Don't stop all of the oldest that are used for gossip
|
||||||
|
at the same time. Stop one or a few at a time so that new nodes can take over the responsibility.
|
||||||
|
It's best to leave the oldest nodes until last.
|
||||||
|
|
||||||
|
See the @ref:[failure detector](cluster.md#failure-detector) for more details.
|
||||||
|
|
||||||
|
## Cluster Singleton
|
||||||
|
|
||||||
|
The @ref[Cluster Singleton](cluster-singleton.md) is a singleton per data center. If you start the
|
||||||
|
`ClusterSingletonManager` on all nodes and you have defined 3 different data centers there will be
|
||||||
|
3 active singleton instances in the cluster, one in each data center. This is taken care of automatically,
|
||||||
|
but is important to be aware of. Designing the system for one singleton per data center makes it possible
|
||||||
|
for the system to be available also during network partitions between data centers.
|
||||||
|
|
||||||
|
The reason why the singleton is per data center and not global is that membership information is not
|
||||||
|
guaranteed to be consistent across data centers when using one leader per data center and that makes it
|
||||||
|
difficult to select a single global singleton.
|
||||||
|
|
||||||
|
If you need a global singleton you have to pick one data center to host that singleton and only start the
|
||||||
|
`ClusterSingletonManager` on nodes of that data center. If the data center is unreachable from another data center the
|
||||||
|
singleton is inaccessible, which is a reasonable trade-off when selecting consistency over availability.
|
||||||
|
|
||||||
|
The singleton proxy is by default routing messages to the singleton in the own data center, but
|
||||||
|
it can be started with a `dataCenter` parameter in the `ClusterSingletonProxySettings` to define that
|
||||||
|
it should route messages to a singleton located in another data center. That is useful for example when
|
||||||
|
having a global singleton in one data center and accessing it from other data centers.
|
||||||
|
|
||||||
|
This is how to create a singleton proxy for a specific data center:
|
||||||
|
|
||||||
|
Scala
|
||||||
|
: @@snip [SingletonCompileOnlySpec.scala](/akka-cluster-typed/src/test/scala/docs/akka/cluster/typed/SingletonCompileOnlySpec.scala) { #create-singleton-proxy-dc }
|
||||||
|
|
||||||
|
Java
|
||||||
|
: @@snip [SingletonCompileOnlyTest.java](/akka-cluster-typed/src/test/java/jdocs/akka/cluster/typed/SingletonCompileOnlyTest.java) { #create-singleton-proxy-dc }
|
||||||
|
|
||||||
|
If using the own data center as the `withDataCenter` parameter that would be a proxy for the singleton in the own data center, which
|
||||||
|
is also the default if `withDataCenter` is not given.
|
||||||
|
|
||||||
|
## Cluster Sharding
|
||||||
|
|
||||||
|
The coordinator in @ref[Cluster Sharding](cluster-sharding.md) is a Cluster Singleton and therefore,
|
||||||
|
as explained above, Cluster Sharding is also per data center. Each data center will have its own coordinator
|
||||||
|
and regions, isolated from other data centers. If you start an entity type with the same name on all
|
||||||
|
nodes and you have defined 3 different data centers and then send messages to the same entity id to
|
||||||
|
sharding regions in all data centers you will end up with 3 active entity instances for that entity id,
|
||||||
|
one in each data center. This is because the region/coordinator is only aware of its own data center
|
||||||
|
and will activate the entity there. It's unaware of the existence of corresponding entities in the
|
||||||
|
other data centers.
|
||||||
|
|
||||||
|
Especially when used together with Akka Persistence that is based on the single-writer principle
|
||||||
|
it is important to avoid running the same entity at multiple locations at the same time with a
|
||||||
|
shared data store. That would result in corrupt data since the events stored by different instances
|
||||||
|
may be interleaved and would be interpreted differently in a later replay. For active active persistent
|
||||||
|
entities see Lightbend's [Multi-DC Persistence](https://doc.akka.io/docs/akka-enhancements/current/persistence-dc/index.html)
|
||||||
|
|
||||||
|
If you need global entities you have to pick one data center to host that entity type and only start
|
||||||
|
`ClusterSharding` on nodes of that data center. If the data center is unreachable from another data center the
|
||||||
|
entities are inaccessible, which is a reasonable trade-off when selecting consistency over availability.
|
||||||
|
|
||||||
|
The Cluster Sharding proxy is by default routing messages to the shard regions in their own data center, but
|
||||||
|
it can be started with a `data-center` parameter to define that it should route messages to a shard region
|
||||||
|
located in another data center. That is useful for example when having global entities in one data center and
|
||||||
|
accessing them from other data centers.
|
||||||
|
|
||||||
|
This is how to create a sharding proxy for a specific data center:
|
||||||
|
|
||||||
|
Scala
|
||||||
|
: @@snip [MultiDcClusterShardingSpec.scala](/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/MultiDcClusterShardingSpec.scala) { #proxy-dc }
|
||||||
|
|
||||||
|
Java
|
||||||
|
: @@snip [ShardingCompileOnlyTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ShardingCompileOnlyTest.java) { #proxy-dc }
|
||||||
|
|
||||||
|
and it can also be used with an `EntityRef`:
|
||||||
|
|
||||||
|
Scala
|
||||||
|
: @@snip [MultiDcClusterShardingSpec.scala](/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/MultiDcClusterShardingSpec.scala) { #proxy-dc-entityref }
|
||||||
|
|
||||||
|
Java
|
||||||
|
: @@snip [ShardingCompileOnlyTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ShardingCompileOnlyTest.java) { #proxy-dc-entityref }
|
||||||
|
|
||||||
|
Another way to manage global entities is to make sure that certain entity ids are located in
|
||||||
|
only one data center by routing the messages to the right region. For example, the routing function
|
||||||
|
could be that odd entity ids are routed to data center A and even entity ids to data center B.
|
||||||
|
Before sending the message to the local region actor you make the decision of which data center it should
|
||||||
|
be routed to. Messages for another data center can be sent with a sharding proxy as explained above and
|
||||||
|
messages for the own data center are sent to the local region.
|
||||||
|
|
@ -462,4 +462,4 @@ Classic Pub Sub can be used by leveraging the `.toClassic` adapters.
|
||||||
See @ref:[Distributed Publish Subscribe in Cluster](../distributed-pub-sub.md). The API is @github[#26338](#26338).
|
See @ref:[Distributed Publish Subscribe in Cluster](../distributed-pub-sub.md). The API is @github[#26338](#26338).
|
||||||
|
|
||||||
@@include[cluster.md](../includes/cluster.md) { #cluster-multidc }
|
@@include[cluster.md](../includes/cluster.md) { #cluster-multidc }
|
||||||
See @ref:[Cluster Multi-DC](../cluster-dc.md). The API for multi-dc sharding is @github[#27705](#27705).
|
See @ref:[Cluster Multi-DC](cluster-dc.md).
|
||||||
|
|
|
||||||
|
|
@ -15,11 +15,11 @@ project.description: Akka Cluster concepts, node membership service, CRDT Distri
|
||||||
* [cluster-singleton](cluster-singleton.md)
|
* [cluster-singleton](cluster-singleton.md)
|
||||||
* [cluster-sharding](cluster-sharding.md)
|
* [cluster-sharding](cluster-sharding.md)
|
||||||
* [cluster-sharding-specification](cluster-sharding-concepts.md)
|
* [cluster-sharding-specification](cluster-sharding-concepts.md)
|
||||||
|
* [cluster-dc](cluster-dc.md)
|
||||||
* [serialization](../serialization.md)
|
* [serialization](../serialization.md)
|
||||||
* [serialization-jackson](../serialization-jackson.md)
|
* [serialization-jackson](../serialization-jackson.md)
|
||||||
* [multi-jvm-testing](../multi-jvm-testing.md)
|
* [multi-jvm-testing](../multi-jvm-testing.md)
|
||||||
* [multi-node-testing](../multi-node-testing.md)
|
* [multi-node-testing](../multi-node-testing.md)
|
||||||
* [cluster-dc](../cluster-dc.md)
|
|
||||||
* [remoting-artery](../remoting-artery.md)
|
* [remoting-artery](../remoting-artery.md)
|
||||||
* [remoting](../remoting.md)
|
* [remoting](../remoting.md)
|
||||||
* [coordination](../coordination.md)
|
* [coordination](../coordination.md)
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue