Add API for multi-dc Sharding/Singleton in Typed, #27705 (#27974)

* 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:
Patrik Nordwall 2019-10-15 12:20:41 +02:00 committed by GitHub
parent c83d04c1f8
commit 46fcca5f39
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
18 changed files with 480 additions and 207 deletions

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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