Docs for typed distributed data (#25747)

* Typed distributed data docs shaping up
This commit is contained in:
Johan Andrén 2018-10-12 09:14:01 +02:00 committed by Christopher Batey
parent 0c608321b9
commit b0de255d27
3 changed files with 203 additions and 71 deletions

View file

@ -4,16 +4,15 @@
package akka.cluster.ddata.typed.javadsl;
import akka.cluster.ddata.typed.javadsl.Replicator;
import akka.cluster.ddata.typed.javadsl.ReplicatorSettings;
import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory;
import java.util.Optional;
import org.junit.ClassRule;
import org.junit.Test;
import org.scalatest.junit.JUnitSuite;
import akka.actor.ActorSystem;
// #sample
import java.util.Optional;
import akka.actor.typed.ActorSystem;
import akka.cluster.Cluster;
import akka.cluster.ddata.GCounter;
import akka.cluster.ddata.GCounterKey;
@ -29,12 +28,15 @@ import akka.actor.typed.javadsl.Adapter;
import akka.actor.typed.javadsl.MutableBehavior;
import akka.actor.typed.javadsl.ActorContext;
// #sample
public class ReplicatorTest extends JUnitSuite {
interface ClientCommand {
}
// #sample
interface ClientCommand { }
static final class Increment implements ClientCommand {
enum Increment implements ClientCommand {
INSTANCE
}
static final class GetValue implements ClientCommand {
@ -53,36 +55,35 @@ public class ReplicatorTest extends JUnitSuite {
}
}
interface InternalMsg extends ClientCommand {
}
private interface InternalMsg extends ClientCommand { }
static final class InternalUpdateResponse<A extends ReplicatedData> implements InternalMsg {
final Replicator.UpdateResponse<A> rsp;
private static final class InternalUpdateResponse implements InternalMsg {
final Replicator.UpdateResponse<GCounter> rsp;
InternalUpdateResponse(Replicator.UpdateResponse<A> rsp) {
InternalUpdateResponse(Replicator.UpdateResponse<GCounter> rsp) {
this.rsp = rsp;
}
}
static final class InternalGetResponse<A extends ReplicatedData> implements InternalMsg {
final Replicator.GetResponse<A> rsp;
private static final class InternalGetResponse implements InternalMsg {
final Replicator.GetResponse<GCounter> rsp;
InternalGetResponse(Replicator.GetResponse<A> rsp) {
InternalGetResponse(Replicator.GetResponse<GCounter> rsp) {
this.rsp = rsp;
}
}
static final class InternalChanged<A extends ReplicatedData> implements InternalMsg {
final Replicator.Changed<A> chg;
private static final class InternalChanged implements InternalMsg {
final Replicator.Changed<GCounter> chg;
InternalChanged(Replicator.Changed<A> chg) {
InternalChanged(Replicator.Changed<GCounter> chg) {
this.chg = chg;
}
}
static final Key<GCounter> Key = GCounterKey.create("counter");
static class Client extends MutableBehavior<ClientCommand> {
static class Counter extends MutableBehavior<ClientCommand> {
private final ActorRef<Replicator.Command> replicator;
private final Cluster node;
final ActorRef<Replicator.UpdateResponse<GCounter>> updateResponseAdapter;
@ -91,10 +92,12 @@ public class ReplicatorTest extends JUnitSuite {
private int cachedValue = 0;
Client(ActorRef<Command> replicator, Cluster node, ActorContext<ClientCommand> ctx) {
Counter(ActorRef<Command> replicator, Cluster node, ActorContext<ClientCommand> ctx) {
this.replicator = replicator;
this.node = node;
// adapters turning the messages from the replicator
// into our own protocol
updateResponseAdapter = ctx.messageAdapter(
(Class<Replicator.UpdateResponse<GCounter>>) (Object) Replicator.UpdateResponse.class,
msg -> new InternalUpdateResponse(msg));
@ -110,50 +113,80 @@ public class ReplicatorTest extends JUnitSuite {
replicator.tell(new Replicator.Subscribe<>(Key, changedAdapter));
}
public static Behavior<ClientCommand> create(ActorRef<Command> replicator, Cluster node) {
return Behaviors.setup(ctx -> new Client(replicator, node, ctx));
public static Behavior<ClientCommand> create() {
return Behaviors.setup((ctx) -> {
// The distributed data types still need the implicit untyped Cluster.
// We will look into another solution for that.
Cluster node = Cluster.get(Adapter.toUntyped(ctx.getSystem()));
ActorRef<Replicator.Command> replicator = DistributedData.get(ctx.getSystem()).replicator();
return new Counter(replicator, node, ctx);
});
}
// #sample
// omitted from sample, needed for tests, factory above is for the docs sample
public static Behavior<ClientCommand> create(ActorRef<Command> replicator, Cluster node) {
return Behaviors.setup(ctx -> new Counter(replicator, node, ctx));
}
// #sample
@Override
public Behaviors.Receive<ClientCommand> createReceive() {
return receiveBuilder()
.onMessage(Increment.class, cmd -> {
replicator.tell(
new Replicator.Update<>(Key, GCounter.empty(), Replicator.writeLocal(), updateResponseAdapter,
curr -> curr.increment(node, 1)));
return this;
})
.onMessage(InternalUpdateResponse.class, msg -> {
return this;
})
.onMessage(GetValue.class, cmd -> {
replicator.tell(
new Replicator.Get<>(Key, Replicator.readLocal(), getResponseAdapter, Optional.of(cmd.replyTo)));
return this;
})
.onMessage(GetCachedValue.class, cmd -> {
cmd.replyTo.tell(cachedValue);
return this;
})
.onMessage(InternalGetResponse.class, msg -> {
if (msg.rsp instanceof Replicator.GetSuccess) {
int value = ((Replicator.GetSuccess<?>) msg.rsp).get(Key).getValue().intValue();
ActorRef<Integer> replyTo = (ActorRef<Integer>) msg.rsp.request().get();
replyTo.tell(value);
} else {
// not dealing with failures
}
return this;
})
.onMessage(InternalChanged.class, msg -> {
GCounter counter = (GCounter) msg.chg.get(Key);
cachedValue = counter.getValue().intValue();
return this;
})
.onMessage(Increment.class, this::onIncrement)
.onMessage(InternalUpdateResponse.class, msg -> Behaviors.same())
.onMessage(GetValue.class, this::onGetValue)
.onMessage(GetCachedValue.class, this::onGetCachedValue)
.onMessage(InternalGetResponse.class, this::onInternalGetResponse)
.onMessage(InternalChanged.class, this::onInternalChanged)
.build();
}
private Behavior<ClientCommand> onIncrement(Increment cmd) {
replicator.tell(
new Replicator.Update<>(
Key,
GCounter.empty(),
Replicator.writeLocal(),
updateResponseAdapter,
curr -> curr.increment(node, 1)));
return Behaviors.same();
}
private Behavior<ClientCommand> onGetValue(GetValue cmd) {
replicator.tell(
new Replicator.Get<>(Key, Replicator.readLocal(), getResponseAdapter, Optional.of(cmd.replyTo)));
return Behaviors.same();
}
private Behavior<ClientCommand> onGetCachedValue(GetCachedValue cmd) {
cmd.replyTo.tell(cachedValue);
return Behaviors.same();
}
private Behavior<ClientCommand> onInternalGetResponse(InternalGetResponse msg) {
if (msg.rsp instanceof Replicator.GetSuccess) {
int value = ((Replicator.GetSuccess<?>) msg.rsp).get(Key).getValue().intValue();
ActorRef<Integer> replyTo = (ActorRef<Integer>) msg.rsp.request().get();
replyTo.tell(value);
return Behaviors.same();
} else {
// not dealing with failures
return Behaviors.unhandled();
}
}
private Behavior<ClientCommand> onInternalChanged(InternalChanged msg) {
GCounter counter = msg.chg.get(Key);
cachedValue = counter.getValue().intValue();
return this;
}
}
// #sample
static Config config = ConfigFactory.parseString(
"akka.actor.provider = cluster \n" +
@ -165,14 +198,13 @@ public class ReplicatorTest extends JUnitSuite {
public static AkkaJUnitActorSystemResource actorSystemResource = new AkkaJUnitActorSystemResource("ReplicatorTest",
config);
private final ActorSystem system = actorSystemResource.getSystem();
private final akka.actor.ActorSystem system = actorSystemResource.getSystem();
akka.actor.typed.ActorSystem<?> typedSystem() {
ActorSystem<?> typedSystem() {
return Adapter.toTyped(system);
}
@Test
public void shouldHaveApiForUpdateAndGet() {
TestKit probe = new TestKit(system);
@ -180,9 +212,9 @@ public class ReplicatorTest extends JUnitSuite {
ActorRef<Replicator.Command> replicator =
Adapter.spawnAnonymous(system, Replicator.behavior(settings));
ActorRef<ClientCommand> client =
Adapter.spawnAnonymous(system, Client.create(replicator, Cluster.get(system)));
Adapter.spawnAnonymous(system, Counter.create(replicator, Cluster.get(system)));
client.tell(new Increment());
client.tell(Increment.INSTANCE);
client.tell(new GetValue(Adapter.toTyped(probe.getRef())));
probe.expectMsg(1);
}
@ -194,17 +226,17 @@ public class ReplicatorTest extends JUnitSuite {
ActorRef<Replicator.Command> replicator =
Adapter.spawnAnonymous(system, Replicator.behavior(settings));
ActorRef<ClientCommand> client =
Adapter.spawnAnonymous(system, Client.create(replicator, Cluster.get(system)));
Adapter.spawnAnonymous(system, Counter.create(replicator, Cluster.get(system)));
client.tell(new Increment());
client.tell(new Increment());
client.tell(Increment.INSTANCE);
client.tell(Increment.INSTANCE);
probe.awaitAssert(() -> {
client.tell(new GetCachedValue(Adapter.toTyped(probe.getRef())));
probe.expectMsg(2);
return null;
});
client.tell(new Increment());
client.tell(Increment.INSTANCE);
probe.awaitAssert(() -> {
client.tell(new GetCachedValue(Adapter.toTyped(probe.getRef())));
probe.expectMsg(3);

View file

@ -4,6 +4,10 @@
package akka.cluster.ddata.typed.scaladsl
import org.scalatest.WordSpecLike
import akka.actor.testkit.typed.TestKitSettings
// #sample
import akka.actor.Scheduler
import akka.actor.typed.{ ActorRef, Behavior }
import akka.actor.typed.scaladsl.AskPattern._
@ -13,14 +17,14 @@ import akka.cluster.Cluster
import akka.cluster.ddata.typed.scaladsl.Replicator._
import akka.cluster.ddata.{ GCounter, GCounterKey, ReplicatedData }
import akka.actor.testkit.typed.scaladsl._
import akka.actor.testkit.typed.TestKitSettings
import akka.util.Timeout
import com.typesafe.config.ConfigFactory
import org.scalatest.WordSpecLike
import scala.concurrent.Future
import scala.concurrent.duration._
// #sample
object ReplicatorSpec {
val config = ConfigFactory.parseString(
@ -31,20 +35,24 @@ object ReplicatorSpec {
akka.remote.artery.canonical.hostname = 127.0.0.1
""")
// #sample
sealed trait ClientCommand
final case object Increment extends ClientCommand
final case class GetValue(replyTo: ActorRef[Int]) extends ClientCommand
final case class GetCachedValue(replyTo: ActorRef[Int]) extends ClientCommand
private sealed trait InternalMsg extends ClientCommand
private case class InternalUpdateResponse[A <: ReplicatedData](rsp: Replicator.UpdateResponse[A]) extends InternalMsg
private case class InternalGetResponse[A <: ReplicatedData](rsp: Replicator.GetResponse[A]) extends InternalMsg
private case class InternalChanged[A <: ReplicatedData](chg: Replicator.Changed[A]) extends InternalMsg
private case class InternalUpdateResponse(rsp: Replicator.UpdateResponse[GCounter]) extends InternalMsg
private case class InternalGetResponse(rsp: Replicator.GetResponse[GCounter]) extends InternalMsg
private case class InternalChanged(chg: Replicator.Changed[GCounter]) extends InternalMsg
val Key = GCounterKey("counter")
def client(replicator: ActorRef[Replicator.Command])(implicit cluster: Cluster): Behavior[ClientCommand] =
Behaviors.setup[ClientCommand] { ctx
// The distributed data types still need the implicit untyped Cluster.
// We will look into another solution for that.
val updateResponseAdapter: ActorRef[Replicator.UpdateResponse[GCounter]] =
ctx.messageAdapter(InternalUpdateResponse.apply)
@ -92,6 +100,7 @@ object ReplicatorSpec {
behavior(cachedValue = 0)
}
// #sample
object CompileOnlyTest {
def shouldHaveConvenienceForAsk(): Unit = {