Remove docs for ActorPub/Sub and write persistence query docs as stages (#26844)
* use separate db columns * Use io dispatcher for sample stage
This commit is contained in:
parent
2bbf13f707
commit
c65bf2d276
22 changed files with 272 additions and 2194 deletions
|
|
@ -6,6 +6,7 @@ package jdocs.persistence;
|
|||
|
||||
import static akka.pattern.Patterns.ask;
|
||||
|
||||
import java.sql.Connection;
|
||||
import java.time.Duration;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
|
@ -21,9 +22,8 @@ import akka.stream.ActorMaterializer;
|
|||
import akka.stream.javadsl.Sink;
|
||||
import akka.stream.javadsl.Source;
|
||||
|
||||
import docs.persistence.query.MyEventsByTagPublisher;
|
||||
import jdocs.persistence.query.MyEventsByTagSource;
|
||||
import org.reactivestreams.Subscriber;
|
||||
import scala.concurrent.duration.FiniteDuration;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
|
@ -92,12 +92,11 @@ public class PersistenceQueryDocTest {
|
|||
akka.persistence.query.javadsl.PersistenceIdsQuery,
|
||||
akka.persistence.query.javadsl.CurrentPersistenceIdsQuery {
|
||||
|
||||
private final FiniteDuration refreshInterval;
|
||||
private final Duration refreshInterval;
|
||||
private Connection conn;
|
||||
|
||||
public MyJavadslReadJournal(ExtendedActorSystem system, Config config) {
|
||||
refreshInterval =
|
||||
FiniteDuration.create(
|
||||
config.getDuration("refresh-interval", TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
|
||||
refreshInterval = config.getDuration("refresh-interval");
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -115,10 +114,8 @@ public class PersistenceQueryDocTest {
|
|||
public Source<EventEnvelope, NotUsed> eventsByTag(String tag, Offset offset) {
|
||||
if (offset instanceof Sequence) {
|
||||
Sequence sequenceOffset = (Sequence) offset;
|
||||
final Props props =
|
||||
MyEventsByTagPublisher.props(tag, sequenceOffset.value(), refreshInterval);
|
||||
return Source.<EventEnvelope>actorPublisher(props)
|
||||
.mapMaterializedValue(m -> NotUsed.getInstance());
|
||||
return Source.fromGraph(
|
||||
new MyEventsByTagSource(conn, tag, sequenceOffset.value(), refreshInterval));
|
||||
} else if (offset == NoOffset.getInstance())
|
||||
return eventsByTag(tag, Offset.sequence(0L)); // recursive
|
||||
else
|
||||
|
|
|
|||
|
|
@ -1,135 +0,0 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.persistence.query;
|
||||
|
||||
import akka.actor.Cancellable;
|
||||
import akka.actor.Scheduler;
|
||||
import akka.japi.Pair;
|
||||
import akka.persistence.PersistentRepr;
|
||||
import akka.persistence.query.Offset;
|
||||
import akka.serialization.Serialization;
|
||||
import akka.serialization.SerializationExtension;
|
||||
import akka.stream.actor.AbstractActorPublisher;
|
||||
|
||||
import akka.actor.Props;
|
||||
import akka.persistence.query.EventEnvelope;
|
||||
import akka.stream.actor.ActorPublisherMessage.Cancel;
|
||||
|
||||
import java.sql.Connection;
|
||||
import java.sql.PreparedStatement;
|
||||
import java.sql.ResultSet;
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.time.Duration;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
// #events-by-tag-publisher
|
||||
class MyEventsByTagJavaPublisher extends AbstractActorPublisher<EventEnvelope> {
|
||||
private final Serialization serialization = SerializationExtension.get(getContext().getSystem());
|
||||
|
||||
private final Connection connection;
|
||||
|
||||
private final String tag;
|
||||
|
||||
private final String CONTINUE = "CONTINUE";
|
||||
private final int LIMIT = 1000;
|
||||
private long currentOffset;
|
||||
private List<EventEnvelope> buf = new LinkedList<>();
|
||||
|
||||
private Cancellable continueTask;
|
||||
|
||||
public MyEventsByTagJavaPublisher(
|
||||
Connection connection, String tag, Long offset, Duration refreshInterval) {
|
||||
this.connection = connection;
|
||||
this.tag = tag;
|
||||
this.currentOffset = offset;
|
||||
|
||||
final Scheduler scheduler = getContext().getSystem().scheduler();
|
||||
this.continueTask =
|
||||
scheduler.schedule(
|
||||
refreshInterval,
|
||||
refreshInterval,
|
||||
getSelf(),
|
||||
CONTINUE,
|
||||
getContext().getDispatcher(),
|
||||
getSelf());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Receive createReceive() {
|
||||
return receiveBuilder()
|
||||
.matchEquals(
|
||||
CONTINUE,
|
||||
(in) -> {
|
||||
query();
|
||||
deliverBuf();
|
||||
})
|
||||
.match(
|
||||
Cancel.class,
|
||||
(in) -> {
|
||||
getContext().stop(getSelf());
|
||||
})
|
||||
.build();
|
||||
}
|
||||
|
||||
public static Props props(Connection conn, String tag, Long offset, Duration refreshInterval) {
|
||||
return Props.create(
|
||||
MyEventsByTagJavaPublisher.class,
|
||||
() -> new MyEventsByTagJavaPublisher(conn, tag, offset, refreshInterval));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postStop() {
|
||||
continueTask.cancel();
|
||||
}
|
||||
|
||||
private void query() {
|
||||
if (buf.isEmpty()) {
|
||||
final String query =
|
||||
"SELECT id, persistent_repr "
|
||||
+ "FROM journal WHERE tag = ? AND id > ? "
|
||||
+ "ORDER BY id LIMIT ?";
|
||||
|
||||
try (PreparedStatement s = connection.prepareStatement(query)) {
|
||||
s.setString(1, tag);
|
||||
s.setLong(2, currentOffset);
|
||||
s.setLong(3, LIMIT);
|
||||
try (ResultSet rs = s.executeQuery()) {
|
||||
|
||||
final List<Pair<Long, byte[]>> res = new ArrayList<>(LIMIT);
|
||||
while (rs.next()) res.add(Pair.create(rs.getLong(1), rs.getBytes(2)));
|
||||
|
||||
if (!res.isEmpty()) {
|
||||
currentOffset = res.get(res.size() - 1).first();
|
||||
}
|
||||
|
||||
buf =
|
||||
res.stream()
|
||||
.map(
|
||||
in -> {
|
||||
final Long id = in.first();
|
||||
final byte[] bytes = in.second();
|
||||
|
||||
final PersistentRepr p =
|
||||
serialization.deserialize(bytes, PersistentRepr.class).get();
|
||||
|
||||
return new EventEnvelope(
|
||||
Offset.sequence(id), p.persistenceId(), p.sequenceNr(), p.payload());
|
||||
})
|
||||
.collect(toList());
|
||||
}
|
||||
} catch (Exception e) {
|
||||
onErrorThenStop(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void deliverBuf() {
|
||||
while (totalDemand() > 0 && !buf.isEmpty()) onNext(buf.remove(0));
|
||||
}
|
||||
}
|
||||
// #events-by-tag-publisher
|
||||
|
|
@ -0,0 +1,136 @@
|
|||
/*
|
||||
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.persistence.query;
|
||||
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.japi.Pair;
|
||||
import akka.persistence.PersistentRepr;
|
||||
import akka.persistence.query.EventEnvelope;
|
||||
import akka.persistence.query.Offset;
|
||||
import akka.serialization.Serialization;
|
||||
import akka.serialization.SerializationExtension;
|
||||
import akka.stream.*;
|
||||
import akka.stream.stage.*;
|
||||
|
||||
import java.sql.Connection;
|
||||
import java.sql.PreparedStatement;
|
||||
import java.sql.ResultSet;
|
||||
import java.time.Duration;
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
// #events-by-tag-publisher
|
||||
public class MyEventsByTagSource extends GraphStage<SourceShape<EventEnvelope>> {
|
||||
public Outlet<EventEnvelope> out = Outlet.create("MyEventByTagSource.out");
|
||||
private static final String QUERY =
|
||||
"SELECT id, persistence_id, seq_nr, serializer_id, serializer_manifest, payload "
|
||||
+ "FROM journal WHERE tag = ? AND id > ? "
|
||||
+ "ORDER BY id LIMIT ?";
|
||||
|
||||
enum Continue {
|
||||
INSTANCE;
|
||||
}
|
||||
|
||||
private static final int LIMIT = 1000;
|
||||
private final Connection connection;
|
||||
private final String tag;
|
||||
private final long initialOffset;
|
||||
private final Duration refreshInterval;
|
||||
|
||||
// assumes a shared connection, could also be a factory for creating connections/pool
|
||||
public MyEventsByTagSource(
|
||||
Connection connection, String tag, long initialOffset, Duration refreshInterval) {
|
||||
this.connection = connection;
|
||||
this.tag = tag;
|
||||
this.initialOffset = initialOffset;
|
||||
this.refreshInterval = refreshInterval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Attributes initialAttributes() {
|
||||
return Attributes.apply(ActorAttributes.IODispatcher());
|
||||
}
|
||||
|
||||
@Override
|
||||
public SourceShape<EventEnvelope> shape() {
|
||||
return SourceShape.of(out);
|
||||
}
|
||||
|
||||
@Override
|
||||
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
|
||||
return new TimerGraphStageLogic(shape()) {
|
||||
private ActorSystem system = ((ActorMaterializer) materializer()).system();
|
||||
private long currentOffset = initialOffset;
|
||||
private List<EventEnvelope> buf = new LinkedList<>();
|
||||
private final Serialization serialization = SerializationExtension.get(system);
|
||||
|
||||
@Override
|
||||
public void preStart() {
|
||||
schedulePeriodically(Continue.INSTANCE, refreshInterval);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTimer(Object timerKey) {
|
||||
query();
|
||||
deliver();
|
||||
}
|
||||
|
||||
private void deliver() {
|
||||
if (isAvailable(out) && !buf.isEmpty()) {
|
||||
push(out, buf.remove(0));
|
||||
}
|
||||
}
|
||||
|
||||
private void query() {
|
||||
if (buf.isEmpty()) {
|
||||
|
||||
try (PreparedStatement s = connection.prepareStatement(QUERY)) {
|
||||
s.setString(1, tag);
|
||||
s.setLong(2, currentOffset);
|
||||
s.setLong(3, LIMIT);
|
||||
try (ResultSet rs = s.executeQuery()) {
|
||||
final List<EventEnvelope> res = new ArrayList<>(LIMIT);
|
||||
while (rs.next()) {
|
||||
Object deserialized =
|
||||
serialization
|
||||
.deserialize(
|
||||
rs.getBytes("payload"),
|
||||
rs.getInt("serializer_id"),
|
||||
rs.getString("serializer_manifest"))
|
||||
.get();
|
||||
currentOffset = rs.getLong("id");
|
||||
res.add(
|
||||
new EventEnvelope(
|
||||
Offset.sequence(currentOffset),
|
||||
rs.getString("persistence_id"),
|
||||
rs.getLong("seq_nr"),
|
||||
deserialized));
|
||||
}
|
||||
buf = res;
|
||||
}
|
||||
} catch (Exception e) {
|
||||
failStage(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
setHandler(
|
||||
out,
|
||||
new AbstractOutHandler() {
|
||||
@Override
|
||||
public void onPull() {
|
||||
query();
|
||||
deliver();
|
||||
}
|
||||
});
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
// #events-by-tag-publisher
|
||||
|
|
@ -1,161 +0,0 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.actor.Props;
|
||||
import akka.stream.ActorMaterializer;
|
||||
import akka.stream.Materializer;
|
||||
import akka.stream.actor.AbstractActorPublisher;
|
||||
import akka.stream.actor.ActorPublisherMessage;
|
||||
import akka.stream.javadsl.Sink;
|
||||
import akka.stream.javadsl.Source;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import akka.testkit.javadsl.TestKit;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class ActorPublisherDocTest extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
static Materializer mat;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("ActorPublisherDocTest");
|
||||
mat = ActorMaterializer.create(system);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
mat = null;
|
||||
}
|
||||
|
||||
// #job-manager
|
||||
public static class JobManagerProtocol {
|
||||
public static final class Job {
|
||||
public final String payload;
|
||||
|
||||
public Job(String payload) {
|
||||
this.payload = payload;
|
||||
}
|
||||
}
|
||||
|
||||
public static class JobAcceptedMessage {
|
||||
@Override
|
||||
public String toString() {
|
||||
return "JobAccepted";
|
||||
}
|
||||
}
|
||||
|
||||
public static final JobAcceptedMessage JobAccepted = new JobAcceptedMessage();
|
||||
|
||||
public static class JobDeniedMessage {
|
||||
@Override
|
||||
public String toString() {
|
||||
return "JobDenied";
|
||||
}
|
||||
}
|
||||
|
||||
public static final JobDeniedMessage JobDenied = new JobDeniedMessage();
|
||||
}
|
||||
|
||||
public static class JobManager extends AbstractActorPublisher<JobManagerProtocol.Job> {
|
||||
|
||||
public static Props props() {
|
||||
return Props.create(JobManager.class);
|
||||
}
|
||||
|
||||
private final int MAX_BUFFER_SIZE = 100;
|
||||
private final List<JobManagerProtocol.Job> buf = new ArrayList<>();
|
||||
|
||||
@Override
|
||||
public Receive createReceive() {
|
||||
return receiveBuilder()
|
||||
.match(
|
||||
JobManagerProtocol.Job.class,
|
||||
job -> buf.size() == MAX_BUFFER_SIZE,
|
||||
job -> {
|
||||
getSender().tell(JobManagerProtocol.JobDenied, getSelf());
|
||||
})
|
||||
.match(
|
||||
JobManagerProtocol.Job.class,
|
||||
job -> {
|
||||
getSender().tell(JobManagerProtocol.JobAccepted, getSelf());
|
||||
|
||||
if (buf.isEmpty() && totalDemand() > 0) onNext(job);
|
||||
else {
|
||||
buf.add(job);
|
||||
deliverBuf();
|
||||
}
|
||||
})
|
||||
.match(ActorPublisherMessage.Request.class, request -> deliverBuf())
|
||||
.match(ActorPublisherMessage.Cancel.class, cancel -> getContext().stop(getSelf()))
|
||||
.build();
|
||||
}
|
||||
|
||||
void deliverBuf() {
|
||||
while (totalDemand() > 0) {
|
||||
/*
|
||||
* totalDemand is a Long and could be larger than
|
||||
* what buf.splitAt can accept
|
||||
*/
|
||||
if (totalDemand() <= Integer.MAX_VALUE) {
|
||||
final List<JobManagerProtocol.Job> took =
|
||||
buf.subList(0, Math.min(buf.size(), (int) totalDemand()));
|
||||
took.forEach(this::onNext);
|
||||
buf.removeAll(took);
|
||||
break;
|
||||
} else {
|
||||
final List<JobManagerProtocol.Job> took =
|
||||
buf.subList(0, Math.min(buf.size(), Integer.MAX_VALUE));
|
||||
took.forEach(this::onNext);
|
||||
buf.removeAll(took);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
// #job-manager
|
||||
|
||||
@Test
|
||||
public void demonstrateActorPublisherUsage() {
|
||||
new TestKit(system) {
|
||||
private final SilenceSystemOut.System System = SilenceSystemOut.get(getTestActor());
|
||||
|
||||
{
|
||||
// #actor-publisher-usage
|
||||
final Source<JobManagerProtocol.Job, ActorRef> jobManagerSource =
|
||||
Source.actorPublisher(JobManager.props());
|
||||
|
||||
final ActorRef ref =
|
||||
jobManagerSource
|
||||
.map(job -> job.payload.toUpperCase())
|
||||
.map(
|
||||
elem -> {
|
||||
System.out.println(elem);
|
||||
return elem;
|
||||
})
|
||||
.to(Sink.ignore())
|
||||
.run(mat);
|
||||
|
||||
ref.tell(new JobManagerProtocol.Job("a"), ActorRef.noSender());
|
||||
ref.tell(new JobManagerProtocol.Job("b"), ActorRef.noSender());
|
||||
ref.tell(new JobManagerProtocol.Job("c"), ActorRef.noSender());
|
||||
// #actor-publisher-usage
|
||||
|
||||
expectMsgEquals("A");
|
||||
expectMsgEquals("B");
|
||||
expectMsgEquals("C");
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -1,274 +0,0 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import akka.actor.AbstractActor;
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.actor.Props;
|
||||
import akka.routing.ActorRefRoutee;
|
||||
import akka.routing.RoundRobinRoutingLogic;
|
||||
import akka.routing.Routee;
|
||||
import akka.routing.Router;
|
||||
import akka.stream.ActorMaterializer;
|
||||
import akka.stream.Materializer;
|
||||
import akka.stream.actor.AbstractActorSubscriber;
|
||||
import akka.stream.actor.ActorSubscriberMessage;
|
||||
import akka.stream.actor.MaxInFlightRequestStrategy;
|
||||
import akka.stream.actor.RequestStrategy;
|
||||
import akka.stream.javadsl.Sink;
|
||||
import akka.stream.javadsl.Source;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import akka.testkit.javadsl.TestKit;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.*;
|
||||
import java.time.Duration;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
public class ActorSubscriberDocTest extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
static Materializer mat;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("ActorSubscriberDocTest");
|
||||
mat = ActorMaterializer.create(system);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
mat = null;
|
||||
}
|
||||
|
||||
// #worker-pool
|
||||
public static class WorkerPoolProtocol {
|
||||
|
||||
public static class Msg {
|
||||
public final int id;
|
||||
public final ActorRef replyTo;
|
||||
|
||||
public Msg(int id, ActorRef replyTo) {
|
||||
this.id = id;
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("Msg(%s, %s)", id, replyTo);
|
||||
}
|
||||
}
|
||||
|
||||
public static Msg msg(int id, ActorRef replyTo) {
|
||||
return new Msg(id, replyTo);
|
||||
}
|
||||
|
||||
public static class Work {
|
||||
public final int id;
|
||||
|
||||
public Work(int id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("Work(%s)", id);
|
||||
}
|
||||
}
|
||||
|
||||
public static Work work(int id) {
|
||||
return new Work(id);
|
||||
}
|
||||
|
||||
public static class Reply {
|
||||
public final int id;
|
||||
|
||||
public Reply(int id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("Reply(%s)", id);
|
||||
}
|
||||
}
|
||||
|
||||
public static Reply reply(int id) {
|
||||
return new Reply(id);
|
||||
}
|
||||
|
||||
public static class Done {
|
||||
public final int id;
|
||||
|
||||
public Done(int id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("Done(%s)", id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
Done done = (Done) o;
|
||||
|
||||
if (id != done.id) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return id;
|
||||
}
|
||||
}
|
||||
|
||||
public static Done done(int id) {
|
||||
return new Done(id);
|
||||
}
|
||||
}
|
||||
|
||||
public static class WorkerPool extends AbstractActorSubscriber {
|
||||
|
||||
public static Props props() {
|
||||
return Props.create(WorkerPool.class);
|
||||
}
|
||||
|
||||
final int MAX_QUEUE_SIZE = 10;
|
||||
final Map<Integer, ActorRef> queue = new HashMap<>();
|
||||
|
||||
final Router router;
|
||||
|
||||
@Override
|
||||
public RequestStrategy requestStrategy() {
|
||||
return new MaxInFlightRequestStrategy(MAX_QUEUE_SIZE) {
|
||||
@Override
|
||||
public int inFlightInternally() {
|
||||
return queue.size();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public WorkerPool() {
|
||||
final List<Routee> routees = new ArrayList<>();
|
||||
for (int i = 0; i < 3; i++)
|
||||
routees.add(new ActorRefRoutee(getContext().actorOf(Props.create(Worker.class))));
|
||||
router = new Router(new RoundRobinRoutingLogic(), routees);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Receive createReceive() {
|
||||
return receiveBuilder()
|
||||
.match(
|
||||
ActorSubscriberMessage.OnNext.class,
|
||||
on -> on.element() instanceof WorkerPoolProtocol.Msg,
|
||||
onNext -> {
|
||||
WorkerPoolProtocol.Msg msg = (WorkerPoolProtocol.Msg) onNext.element();
|
||||
queue.put(msg.id, msg.replyTo);
|
||||
|
||||
if (queue.size() > MAX_QUEUE_SIZE)
|
||||
throw new RuntimeException("queued too many: " + queue.size());
|
||||
|
||||
router.route(WorkerPoolProtocol.work(msg.id), getSelf());
|
||||
})
|
||||
.match(
|
||||
ActorSubscriberMessage.onCompleteInstance().getClass(),
|
||||
complete -> {
|
||||
if (queue.isEmpty()) {
|
||||
getContext().stop(getSelf());
|
||||
}
|
||||
})
|
||||
.match(
|
||||
WorkerPoolProtocol.Reply.class,
|
||||
reply -> {
|
||||
int id = reply.id;
|
||||
queue.get(id).tell(WorkerPoolProtocol.done(id), getSelf());
|
||||
queue.remove(id);
|
||||
if (canceled() && queue.isEmpty()) {
|
||||
getContext().stop(getSelf());
|
||||
}
|
||||
})
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
||||
static class Worker extends AbstractActor {
|
||||
@Override
|
||||
public Receive createReceive() {
|
||||
return receiveBuilder()
|
||||
.match(
|
||||
WorkerPoolProtocol.Work.class,
|
||||
work -> {
|
||||
// ...
|
||||
getSender().tell(WorkerPoolProtocol.reply(work.id), getSelf());
|
||||
})
|
||||
.build();
|
||||
}
|
||||
}
|
||||
// #worker-pool
|
||||
|
||||
@Test
|
||||
public void demonstrateActorPublisherUsage() {
|
||||
new TestKit(system) {
|
||||
|
||||
{
|
||||
final ActorRef replyTo = getTestActor();
|
||||
|
||||
// #actor-subscriber-usage
|
||||
final int N = 117;
|
||||
final List<Integer> data = new ArrayList<>(N);
|
||||
for (int i = 0; i < N; i++) {
|
||||
data.add(i);
|
||||
}
|
||||
|
||||
final ActorRef worker =
|
||||
Source.from(data)
|
||||
.map(i -> WorkerPoolProtocol.msg(i, replyTo))
|
||||
.runWith(Sink.<WorkerPoolProtocol.Msg>actorSubscriber(WorkerPool.props()), mat);
|
||||
// #actor-subscriber-usage
|
||||
|
||||
watch(worker);
|
||||
|
||||
List<Object> got = new ArrayList<>(receiveN(N));
|
||||
Collections.sort(
|
||||
got,
|
||||
new Comparator<Object>() {
|
||||
@Override
|
||||
public int compare(Object o1, Object o2) {
|
||||
if (o1 instanceof WorkerPoolProtocol.Done
|
||||
&& o2 instanceof WorkerPoolProtocol.Done) {
|
||||
return ((WorkerPoolProtocol.Done) o1).id - ((WorkerPoolProtocol.Done) o2).id;
|
||||
} else return 0;
|
||||
}
|
||||
});
|
||||
int i = 0;
|
||||
for (; i < N; i++) {
|
||||
assertEquals(
|
||||
String.format("Expected %d, but got %s", i, got.get(i)),
|
||||
WorkerPoolProtocol.done(i),
|
||||
got.get(i));
|
||||
}
|
||||
assertEquals(String.format("Expected 117 messages but got %d", i), i, 117);
|
||||
expectTerminated(Duration.ofSeconds(10), worker);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue