=clt #18343 Improve structure of DistPubSub docs

This commit is contained in:
Patrik Nordwall 2015-09-11 15:48:49 +02:00
parent 02d0452f23
commit a3ced0a38a
4 changed files with 330 additions and 138 deletions

View file

@ -82,16 +82,41 @@ object DistributedPubSubMediatorSpec extends MultiNodeConfig {
mediator ! Subscribe("content", self)
def receive = {
case s: String
log.info("Got {}", s)
case SubscribeAck(Subscribe("content", None, `self`))
context become ready
log.info("subscribing");
}
}
//#subscriber
def ready: Actor.Receive = {
//#sender
class Sender extends Actor {
import DistributedPubSubMediator.Send
// activate the extension
val mediator = DistributedPubSub(context.system).mediator
def receive = {
case in: String
val out = in.toUpperCase
mediator ! Send(path = "/user/destination", msg = out, localAffinity = true)
}
}
//#sender
//#send-destination
class Destination extends Actor with ActorLogging {
import DistributedPubSubMediator.Put
val mediator = DistributedPubSub(context.system).mediator
// register to the path
mediator ! Put(self)
def receive = {
case s: String
log.info("Got {}", s)
}
}
//#subscriber
//#send-destination
}
@ -301,7 +326,7 @@ class DistributedPubSubMediatorSpec extends MultiNodeSpec(DistributedPubSubMedia
enterBarrier("after-7")
}
"demonstrate usage" in within(15 seconds) {
"demonstrate usage of Publish" in within(15 seconds) {
def later(): Unit = {
awaitCount(10)
}
@ -328,12 +353,38 @@ class DistributedPubSubMediatorSpec extends MultiNodeSpec(DistributedPubSubMedia
enterBarrier("after-8")
}
"demonstrate usage of Send" in within(15 seconds) {
def later(): Unit = {
awaitCount(12)
}
//#start-send-destinations
runOn(first) {
system.actorOf(Props[Destination], "destination")
}
runOn(second) {
system.actorOf(Props[Destination], "destination")
}
//#start-send-destinations
//#send-message
runOn(third) {
val sender = system.actorOf(Props[Sender], "sender")
later()
// after a while the destinations are replicated
sender ! "hello"
}
//#send-message
enterBarrier("after-8")
}
"send-all to all other nodes" in within(15 seconds) {
runOn(first, second, third) { // create the user on all nodes
val u11 = createChatUser("u11")
mediator ! Put(u11)
}
awaitCount(13)
awaitCount(15)
enterBarrier("11-registered")
runOn(third) {
@ -366,7 +417,7 @@ class DistributedPubSubMediatorSpec extends MultiNodeSpec(DistributedPubSubMedia
u13 ! JoinGroup("topic2", "group2")
expectMsg(SubscribeAck(Subscribe("topic2", Some("group2"), u13)))
}
awaitCount(17)
awaitCount(19)
enterBarrier("12-registered")
runOn(first) {
@ -406,8 +457,8 @@ class DistributedPubSubMediatorSpec extends MultiNodeSpec(DistributedPubSubMedia
mediator ! Status(versions = Map.empty)
val deltaBuckets = expectMsgType[Delta].buckets
deltaBuckets.size should ===(3)
deltaBuckets.find(_.owner == firstAddress).get.content.size should ===(9)
deltaBuckets.find(_.owner == secondAddress).get.content.size should ===(8)
deltaBuckets.find(_.owner == firstAddress).get.content.size should ===(10)
deltaBuckets.find(_.owner == secondAddress).get.content.size should ===(9)
deltaBuckets.find(_.owner == thirdAddress).get.content.size should ===(2)
}
enterBarrier("verified-initial-delta")
@ -429,12 +480,12 @@ class DistributedPubSubMediatorSpec extends MultiNodeSpec(DistributedPubSubMedia
mediator ! Status(versions = deltaBuckets2.map(b b.owner -> b.version).toMap)
val deltaBuckets3 = expectMsgType[Delta].buckets
deltaBuckets3.map(_.content.size).sum should ===(9 + 8 + 2 + many - 500 - 500)
deltaBuckets3.map(_.content.size).sum should ===(10 + 9 + 2 + many - 500 - 500)
}
enterBarrier("verified-delta-with-many")
within(10.seconds) {
awaitCount(17 + many)
awaitCount(19 + many)
}
enterBarrier("after-13")

View file

@ -31,7 +31,7 @@ public class DistributedPubSubMediatorTest {
@Test
public void demonstrateUsage() {
public void demonstratePublishUsage() {
//#start-subscribers
system.actorOf(Props.create(Subscriber.class), "subscriber1");
//another node
@ -47,6 +47,21 @@ public class DistributedPubSubMediatorTest {
//#publish-message
}
public void demonstrateSendUsage() {
//#start-send-destinations
system.actorOf(Props.create(Destination.class), "destination");
//another node
system.actorOf(Props.create(Destination.class), "destination");
//#start-send-destinations
//#send-message
//somewhere else
ActorRef sender = system.actorOf(Props.create(Publisher.class), "sender");
// after a while the destinations are replicated
sender.tell("hello", null);
//#send-message
}
static//#subscriber
public class Subscriber extends UntypedActor {
LoggingAdapter log = Logging.getLogger(getContext().system(), this);
@ -90,4 +105,48 @@ public class DistributedPubSubMediatorTest {
}
}
//#publisher
static//#send-destination
public class Destination extends UntypedActor {
LoggingAdapter log = Logging.getLogger(getContext().system(), this);
public Destination() {
ActorRef mediator =
DistributedPubSub.get(getContext().system()).mediator();
// register to the path
mediator.tell(new DistributedPubSubMediator.Put(getSelf()), getSelf());
}
public void onReceive(Object msg) {
if (msg instanceof String)
log.info("Got: {}", msg);
else if (msg instanceof DistributedPubSubMediator.SubscribeAck)
log.info("subscribing");
else
unhandled(msg);
}
}
//#send-destination
static//#sender
public class Sender extends UntypedActor {
// activate the extension
ActorRef mediator =
DistributedPubSub.get(getContext().system()).mediator();
public void onReceive(Object msg) {
if (msg instanceof String) {
String in = (String) msg;
String out = in.toUpperCase();
boolean localAffinity = true;
mediator.tell(new DistributedPubSubMediator.Send("/user/destination", out,
localAffinity), getSelf());
} else {
unhandled(msg);
}
}
}
//#sender
}

View file

@ -22,72 +22,44 @@ a few seconds. Changes are only performed in the own part of the registry and th
changes are versioned. Deltas are disseminated in a scalable way to other nodes with
a gossip protocol.
You can send messages via the mediator on any node to registered actors on
any other node. There are four modes of message delivery.
**1. DistributedPubSubMediator.Send**
The message will be delivered to one recipient with a matching path, if any such
exists in the registry. If several entries match the path the message will be sent
via the supplied ``RoutingLogic`` (default random) to one destination. The sender() of the
message can specify that local affinity is preferred, i.e. the message is sent to an actor
in the same local actor system as the used mediator actor, if any such exists, otherwise
route to any other matching entry. A typical usage of this mode is private chat to one
other user in an instant messaging application. It can also be used for distributing
tasks to registered workers, like a cluster aware router where the routees dynamically
can register themselves.
**2. DistributedPubSubMediator.SendToAll**
The message will be delivered to all recipients with a matching path. Actors with
the same path, without address information, can be registered on different nodes.
On each node there can only be one such actor, since the path is unique within one
local actor system. Typical usage of this mode is to broadcast messages to all replicas
with the same path, e.g. 3 actors on different nodes that all perform the same actions,
for redundancy. You can also optionally specify a property (``allButSelf``) deciding
if the message should be sent to a matching path on the self node or not.
**3. DistributedPubSubMediator.Publish**
Actors may be registered to a named topic instead of path. This enables many subscribers
on each node. The message will be delivered to all subscribers of the topic. For
efficiency the message is sent over the wire only once per node (that has a matching topic),
and then delivered to all subscribers of the local topic representation. This is the
true pub/sub mode. A typical usage of this mode is a chat room in an instant messaging
application.
**4. DistributedPubSubMediator.Publish with sendOneMessageToEachGroup**
Actors may be subscribed to a named topic with an optional property (``group``).
If subscribing with a group name, each message published to a topic with the
(``sendOneMessageToEachGroup``) flag is delivered via the supplied ``RoutingLogic``
(default random) to one actor within each subscribing group.
If all the subscribed actors have the same group name, then this works just like
``Send`` and all messages are delivered to one subscriber.
If all the subscribed actors have different group names, then this works like
normal ``Publish`` and all messages are broadcast to all subscribers.
You register actors to the local mediator with ``DistributedPubSubMediator.Put`` or
``DistributedPubSubMediator.Subscribe``. ``Put`` is used together with ``Send`` and
``SendToAll`` message delivery modes. The ``ActorRef`` in ``Put`` must belong to the same
local actor system as the mediator. ``Subscribe`` is used together with ``Publish``.
Actors are automatically removed from the registry when they are terminated, or you
can explicitly remove entries with ``DistributedPubSubMediator.Remove`` or
``DistributedPubSubMediator.Unsubscribe``.
Successful ``Subscribe`` and ``Unsubscribe`` is acknowledged with
``DistributedPubSubMediator.SubscribeAck`` and ``DistributedPubSubMediator.UnsubscribeAck``
replies.
Cluster members with status :ref:`WeaklyUp <weakly_up_java>`, if that feature is enabled,
will participate in Distributed Publish Subscribe, i.e. subscribers on nodes with
``WeaklyUp`` status will receive published messages if the publisher and subscriber are on
same side of a network partition.
A Small Example
---------------
You can send messages via the mediator on any node to registered actors on
any other node.
A subscriber actor:
There a two different modes of message delivery, explained in the sections
:ref:`distributed-pub-sub-publish-java` and :ref:`distributed-pub-sub-send-java` below.
.. _distributed-pub-sub-publish-java:
Publish
-------
This is the true pub/sub mode. A typical usage of this mode is a chat room in an instant
messaging application.
Actors are registered to a named topic. This enables many subscribers on each node.
The message will be delivered to all subscribers of the topic.
For efficiency the message is sent over the wire only once per node (that has a matching topic),
and then delivered to all subscribers of the local topic representation.
You register actors to the local mediator with ``DistributedPubSubMediator.Subscribe``.
Successful ``Subscribe`` and ``Unsubscribe`` is acknowledged with
``DistributedPubSubMediator.SubscribeAck`` and ``DistributedPubSubMediator.UnsubscribeAck``
replies. The acknowledgment means that the subscription is registered, but it can still
take some time until it is replicated to other nodes.
You publish messages by sending ``DistributedPubSubMediator.Publish`` message to the
local mediator.
Actors are automatically removed from the registry when they are terminated, or you
can explicitly remove entries with ``DistributedPubSubMediator.Unsubscribe``.
An example of a subscriber actor:
.. includecode:: ../../../akka-cluster-tools/src/test/java/akka/cluster/pubsub/DistributedPubSubMediatorTest.java#subscriber
@ -104,6 +76,75 @@ It can publish messages to the topic from anywhere in the cluster:
.. includecode:: ../../../akka-cluster-tools/src/test/java/akka/cluster/pubsub/DistributedPubSubMediatorTest.java#publish-message
Actors may also be subscribed to a named topic with an optional property (``group``).
If subscribing with a group name, each message published to a topic with the
(``sendOneMessageToEachGroup``) flag is delivered via the supplied ``RoutingLogic``
(default random) to one actor within each subscribing group.
If all the subscribed actors have the same group name, then this works just like
``Send`` and all messages are delivered to one subscriber.
If all the subscribed actors have different group names, then this works like
normal ``Publish`` and all messages are broadcast to all subscribers.
.. _distributed-pub-sub-send-java:
Send
----
This is a point-to-point mode where each message is delivered to one destination,
but you still does not have to know where the destination is located.
A typical usage of this mode is private chat to one other user in an instant messaging
application. It can also be used for distributing tasks to registered workers, like a
cluster aware router where the routees dynamically can register themselves.
The message will be delivered to one recipient with a matching path, if any such
exists in the registry. If several entries match the path because it has been registered
on several nodes the message will be sent via the supplied ``RoutingLogic`` (default random)
to one destination. The sender() of the message can specify that local affinity is preferred,
i.e. the message is sent to an actor in the same local actor system as the used mediator actor,
if any such exists, otherwise route to any other matching entry.
You register actors to the local mediator with ``DistributedPubSubMediator.Put``.
The ``ActorRef`` in ``Put`` must belong to the same local actor system as the mediator.
The path without address information is the key to which you send messages.
On each node there can only be one actor for a given path, since the path is unique
within one local actor system.
You send messages by sending ``DistributedPubSubMediator.Send`` message to the
local mediator with the path (without address information) of the destination
actors.
Actors are automatically removed from the registry when they are terminated, or you
can explicitly remove entries with ``DistributedPubSubMediator.Remove``.
An example of a destination actor:
.. includecode:: ../../../akka-cluster-tools/src/test/java/akka/cluster/pubsub/DistributedPubSubMediatorTest.java#send-destination
Subscriber actors can be started on several nodes in the cluster, and all will receive
messages published to the "content" topic.
.. includecode:: ../../../akka-cluster-tools/src/test/java/akka/cluster/pubsub/DistributedPubSubMediatorTest.java#start-send-destinations
A simple actor that publishes to this "content" topic:
.. includecode:: ../../../akka-cluster-tools/src/test/java/akka/cluster/pubsub/DistributedPubSubMediatorTest.java#sender
It can publish messages to the topic from anywhere in the cluster:
.. includecode:: ../../../akka-cluster-tools/src/test/java/akka/cluster/pubsub/DistributedPubSubMediatorTest.java#send-message
It is also possible to broadcast messages to the actors that have been registered with
``Put``. Send ``DistributedPubSubMediator.SendToAll`` message to the local mediator and the wrapped message
will then be delivered to all recipients with a matching path. Actors with
the same path, without address information, can be registered on different nodes.
On each node there can only be one such actor, since the path is unique within one
local actor system.
Typical usage of this mode is to broadcast messages to all replicas
with the same path, e.g. 3 actors on different nodes that all perform the same actions,
for redundancy. You can also optionally specify a property (``allButSelf``) deciding
if the message should be sent to a matching path on the self node or not.
DistributedPubSub Extension
---------------------------

View file

@ -22,72 +22,47 @@ a few seconds. Changes are only performed in the own part of the registry and th
changes are versioned. Deltas are disseminated in a scalable way to other nodes with
a gossip protocol.
You can send messages via the mediator on any node to registered actors on
any other node. There are four modes of message delivery.
**1. DistributedPubSubMediator.Send**
The message will be delivered to one recipient with a matching path, if any such
exists in the registry. If several entries match the path the message will be sent
via the supplied ``RoutingLogic`` (default random) to one destination. The sender() of the
message can specify that local affinity is preferred, i.e. the message is sent to an actor
in the same local actor system as the used mediator actor, if any such exists, otherwise
route to any other matching entry. A typical usage of this mode is private chat to one
other user in an instant messaging application. It can also be used for distributing
tasks to registered workers, like a cluster aware router where the routees dynamically
can register themselves.
**2. DistributedPubSubMediator.SendToAll**
The message will be delivered to all recipients with a matching path. Actors with
the same path, without address information, can be registered on different nodes.
On each node there can only be one such actor, since the path is unique within one
local actor system. Typical usage of this mode is to broadcast messages to all replicas
with the same path, e.g. 3 actors on different nodes that all perform the same actions,
for redundancy. You can also optionally specify a property (``allButSelf``) deciding
if the message should be sent to a matching path on the self node or not.
**3. DistributedPubSubMediator.Publish**
Actors may be registered to a named topic instead of path. This enables many subscribers
on each node. The message will be delivered to all subscribers of the topic. For
efficiency the message is sent over the wire only once per node (that has a matching topic),
and then delivered to all subscribers of the local topic representation. This is the
true pub/sub mode. A typical usage of this mode is a chat room in an instant messaging
application.
**4. DistributedPubSubMediator.Publish with sendOneMessageToEachGroup**
Actors may be subscribed to a named topic with an optional property (``group``).
If subscribing with a group name, each message published to a topic with the
(``sendOneMessageToEachGroup``) flag is delivered via the supplied ``RoutingLogic``
(default random) to one actor within each subscribing group.
If all the subscribed actors have the same group name, then this works just like
``Send`` and all messages are delivered to one subscriber.
If all the subscribed actors have different group names, then this works like
normal ``Publish`` and all messages are broadcast to all subscribers.
You register actors to the local mediator with ``DistributedPubSubMediator.Put`` or
``DistributedPubSubMediator.Subscribe``. ``Put`` is used together with ``Send`` and
``SendToAll`` message delivery modes. The ``ActorRef`` in ``Put`` must belong to the same
local actor system as the mediator. ``Subscribe`` is used together with ``Publish``.
Actors are automatically removed from the registry when they are terminated, or you
can explicitly remove entries with ``DistributedPubSubMediator.Remove`` or
``DistributedPubSubMediator.Unsubscribe``.
Successful ``Subscribe`` and ``Unsubscribe`` is acknowledged with
``DistributedPubSubMediator.SubscribeAck`` and ``DistributedPubSubMediator.UnsubscribeAck``
replies.
Cluster members with status :ref:`WeaklyUp <weakly_up_scala>`, if that feature is enabled,
will participate in Distributed Publish Subscribe, i.e. subscribers on nodes with
``WeaklyUp`` status will receive published messages if the publisher and subscriber are on
same side of a network partition.
A Small Example
---------------
You can send messages via the mediator on any node to registered actors on
any other node.
A subscriber actor:
There a two different modes of message delivery, explained in the sections
:ref:`distributed-pub-sub-publish-scala` and :ref:`distributed-pub-sub-send-scala` below.
A more comprehensive sample is available in the `Typesafe Activator <http://www.typesafe.com/platform/getstarted>`_
tutorial named `Akka Clustered PubSub with Scala! <http://www.typesafe.com/activator/template/akka-clustering>`_.
.. _distributed-pub-sub-publish-scala:
Publish
-------
This is the true pub/sub mode. A typical usage of this mode is a chat room in an instant
messaging application.
Actors are registered to a named topic. This enables many subscribers on each node.
The message will be delivered to all subscribers of the topic.
For efficiency the message is sent over the wire only once per node (that has a matching topic),
and then delivered to all subscribers of the local topic representation.
You register actors to the local mediator with ``DistributedPubSubMediator.Subscribe``.
Successful ``Subscribe`` and ``Unsubscribe`` is acknowledged with
``DistributedPubSubMediator.SubscribeAck`` and ``DistributedPubSubMediator.UnsubscribeAck``
replies. The acknowledgment means that the subscription is registered, but it can still
take some time until it is replicated to other nodes.
You publish messages by sending ``DistributedPubSubMediator.Publish`` message to the
local mediator.
Actors are automatically removed from the registry when they are terminated, or you
can explicitly remove entries with ``DistributedPubSubMediator.Unsubscribe``.
An example of a subscriber actor:
.. includecode:: ../../../akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubMediatorSpec.scala#subscriber
@ -104,8 +79,74 @@ It can publish messages to the topic from anywhere in the cluster:
.. includecode:: ../../../akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubMediatorSpec.scala#publish-message
A more comprehensive sample is available in the `Typesafe Activator <http://www.typesafe.com/platform/getstarted>`_
tutorial named `Akka Clustered PubSub with Scala! <http://www.typesafe.com/activator/template/akka-clustering>`_.
Actors may also be subscribed to a named topic with an optional property (``group``).
If subscribing with a group name, each message published to a topic with the
(``sendOneMessageToEachGroup``) flag is delivered via the supplied ``RoutingLogic``
(default random) to one actor within each subscribing group.
If all the subscribed actors have the same group name, then this works just like
``Send`` and all messages are delivered to one subscriber.
If all the subscribed actors have different group names, then this works like
normal ``Publish`` and all messages are broadcast to all subscribers.
.. _distributed-pub-sub-send-scala:
Send
----
This is a point-to-point mode where each message is delivered to one destination,
but you still does not have to know where the destination is located.
A typical usage of this mode is private chat to one other user in an instant messaging
application. It can also be used for distributing tasks to registered workers, like a
cluster aware router where the routees dynamically can register themselves.
The message will be delivered to one recipient with a matching path, if any such
exists in the registry. If several entries match the path because it has been registered
on several nodes the message will be sent via the supplied ``RoutingLogic`` (default random)
to one destination. The sender() of the message can specify that local affinity is preferred,
i.e. the message is sent to an actor in the same local actor system as the used mediator actor,
if any such exists, otherwise route to any other matching entry.
You register actors to the local mediator with ``DistributedPubSubMediator.Put``.
The ``ActorRef`` in ``Put`` must belong to the same local actor system as the mediator.
The path without address information is the key to which you send messages.
On each node there can only be one actor for a given path, since the path is unique
within one local actor system.
You send messages by sending ``DistributedPubSubMediator.Send`` message to the
local mediator with the path (without address information) of the destination
actors.
Actors are automatically removed from the registry when they are terminated, or you
can explicitly remove entries with ``DistributedPubSubMediator.Remove``.
An example of a destination actor:
.. includecode:: ../../../akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubMediatorSpec.scala#send-destination
Destination actors can be started on several nodes in the cluster, and all will receive
messages sent to the path (without address information).
.. includecode:: ../../../akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubMediatorSpec.scala#start-send-destinations
A simple actor that sends to the path:
.. includecode:: ../../../akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubMediatorSpec.scala#sender
It can send messages to the path from anywhere in the cluster:
.. includecode:: ../../../akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubMediatorSpec.scala#send-message
It is also possible to broadcast messages to the actors that have been registered with
``Put``. Send ``DistributedPubSubMediator.SendToAll`` message to the local mediator and the wrapped message
will then be delivered to all recipients with a matching path. Actors with
the same path, without address information, can be registered on different nodes.
On each node there can only be one such actor, since the path is unique within one
local actor system.
Typical usage of this mode is to broadcast messages to all replicas
with the same path, e.g. 3 actors on different nodes that all perform the same actions,
for redundancy. You can also optionally specify a property (``allButSelf``) deciding
if the message should be sent to a matching path on the self node or not.
DistributedPubSub Extension
---------------------------