* 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
|
|
@ -41,7 +41,7 @@ abstract class MultiDcClusterSingletonSpec
|
|||
extends MultiNodeSpec(MultiDcClusterSingletonSpecConfig)
|
||||
with MultiNodeTypedClusterSpec {
|
||||
|
||||
import MultiDcClusterActors._
|
||||
import MultiDcPinger._
|
||||
import MultiDcClusterSingletonSpecConfig._
|
||||
|
||||
"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 {
|
||||
runOn(first) {
|
||||
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]
|
||||
pinger ! Ping(probe.ref)
|
||||
probe.expectMessage(Pong("dc1"))
|
||||
|
|
@ -79,7 +79,7 @@ abstract class MultiDcClusterSingletonSpec
|
|||
runOn(second) {
|
||||
val singleton = ClusterSingleton(system.toTyped)
|
||||
val pinger = singleton.init(
|
||||
SingletonActor(multiDcPinger, "ping")
|
||||
SingletonActor(MultiDcPinger(), "ping")
|
||||
.withStopMessage(NoMore)
|
||||
.withSettings(ClusterSingletonSettings(typedSystem).withDataCenter("dc1")))
|
||||
val probe = TestProbe[Pong]
|
||||
|
|
@ -93,7 +93,7 @@ abstract class MultiDcClusterSingletonSpec
|
|||
"be able to target singleton with the same name in own dc " in {
|
||||
runOn(second, third) {
|
||||
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]
|
||||
pinger ! Ping(probe.ref)
|
||||
probe.expectMessage(Pong("dc2"))
|
||||
|
|
|
|||
|
|
@ -5,18 +5,20 @@
|
|||
package akka.cluster.typed
|
||||
|
||||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.serialization.jackson.CborSerializable
|
||||
|
||||
object MultiDcClusterActors {
|
||||
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
|
||||
object MultiDcPinger {
|
||||
|
||||
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)
|
||||
Behaviors.receiveMessage[PingProtocol] {
|
||||
Behaviors.receiveMessage[Command] {
|
||||
case Ping(ref) =>
|
||||
ref ! Pong(cluster.selfMember.dataCenter)
|
||||
Behaviors.same
|
||||
|
|
@ -24,6 +24,7 @@ import com.typesafe.config.ConfigFactory;
|
|||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
// FIXME use awaitAssert to await cluster forming like in BasicClusterExampleSpec
|
||||
public class BasicClusterExampleTest { // extends JUnitSuite {
|
||||
|
|
@ -146,4 +147,19 @@ public class BasicClusterExampleTest { // extends JUnitSuite {
|
|||
}
|
||||
// #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 akka.cluster.typed.ClusterSingleton;
|
||||
import akka.cluster.typed.ClusterSingletonSettings;
|
||||
import akka.cluster.typed.SingletonActor;
|
||||
|
||||
// #import
|
||||
|
|
@ -117,4 +118,15 @@ public interface SingletonCompileOnlyTest {
|
|||
// #backoff
|
||||
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.testkit.SocketUtil
|
||||
import com.github.ghik.silencer.silent
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.{ Matchers, WordSpec }
|
||||
//#cluster-imports
|
||||
|
|
@ -83,6 +84,22 @@ akka {
|
|||
}
|
||||
//#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 {
|
||||
|
|
|
|||
|
|
@ -6,9 +6,10 @@ package docs.akka.cluster.typed
|
|||
|
||||
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, SupervisorStrategy }
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import akka.cluster.typed.ClusterSingletonSettings
|
||||
|
||||
object SingletonCompileOnlySpec {
|
||||
|
||||
val system = ActorSystem(Behaviors.empty, "Singleton")
|
||||
|
|
@ -64,4 +65,9 @@ object SingletonCompileOnlySpec {
|
|||
.onFailure[Exception](SupervisorStrategy.restartWithBackoff(1.second, 10.seconds, 0.2)),
|
||||
"GlobalCounter"))
|
||||
//#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
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue