move code to src/test

* so that it compiles and tests pass
* fix some additional snip references in getting started
This commit is contained in:
Patrik Nordwall 2017-05-11 15:11:25 +02:00
parent 413df8e0f4
commit 59f53e1a22
289 changed files with 45 additions and 45 deletions

View file

@ -0,0 +1,152 @@
/*
* Copyright (C) 2015-2017 Lightbend Inc. <http://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 {
final public static 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");
}
};
}
}

View file

@ -0,0 +1,251 @@
/*
* Copyright (C) 2015-2017 Lightbend Inc. <http://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 scala.concurrent.duration.Duration;
import java.util.*;
import java.util.concurrent.TimeUnit;
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.create(10, TimeUnit.SECONDS), worker);
}
};
}
}

View file

@ -0,0 +1,256 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream;
import java.nio.ByteOrder;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
import akka.NotUsed;
import akka.stream.javadsl.GraphDSL;
import jdocs.AbstractJavaTest;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import akka.actor.ActorSystem;
import akka.japi.pf.PFBuilder;
import akka.stream.*;
import akka.stream.javadsl.*;
import akka.stream.stage.*;
import akka.util.ByteIterator;
import akka.util.ByteString;
import akka.util.ByteStringBuilder;
import static org.junit.Assert.assertArrayEquals;
public class BidiFlowDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("FlowDocTest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
//#codec
static interface Message {}
static class Ping implements Message {
final int id;
public Ping(int id) { this.id = id; }
@Override
public boolean equals(Object o) {
if (o instanceof Ping) {
return ((Ping) o).id == id;
} else return false;
}
@Override
public int hashCode() {
return id;
}
}
static class Pong implements Message {
final int id;
public Pong(int id) { this.id = id; }
@Override
public boolean equals(Object o) {
if (o instanceof Pong) {
return ((Pong) o).id == id;
} else return false;
}
@Override
public int hashCode() {
return id;
}
}
//#codec-impl
public static ByteString toBytes(Message msg) {
//#implementation-details-elided
if (msg instanceof Ping) {
final int id = ((Ping) msg).id;
return new ByteStringBuilder().putByte((byte) 1)
.putInt(id, ByteOrder.LITTLE_ENDIAN).result();
} else {
final int id = ((Pong) msg).id;
return new ByteStringBuilder().putByte((byte) 2)
.putInt(id, ByteOrder.LITTLE_ENDIAN).result();
}
//#implementation-details-elided
}
public static Message fromBytes(ByteString bytes) {
//#implementation-details-elided
final ByteIterator it = bytes.iterator();
switch(it.getByte()) {
case 1:
return new Ping(it.getInt(ByteOrder.LITTLE_ENDIAN));
case 2:
return new Pong(it.getInt(ByteOrder.LITTLE_ENDIAN));
default:
throw new RuntimeException("message format error");
}
//#implementation-details-elided
}
//#codec-impl
//#codec
@SuppressWarnings("unused")
//#codec
public final BidiFlow<Message, ByteString, ByteString, Message, NotUsed> codecVerbose =
BidiFlow.fromGraph(GraphDSL.create(b -> {
final FlowShape<Message, ByteString> top =
b.add(Flow.of(Message.class).map(BidiFlowDocTest::toBytes));
final FlowShape<ByteString, Message> bottom =
b.add(Flow.of(ByteString.class).map(BidiFlowDocTest::fromBytes));
return BidiShape.fromFlows(top, bottom);
}));
public final BidiFlow<Message, ByteString, ByteString, Message, NotUsed> codec =
BidiFlow.fromFunctions(BidiFlowDocTest::toBytes, BidiFlowDocTest::fromBytes);
//#codec
//#framing
public static ByteString addLengthHeader(ByteString bytes) {
final int len = bytes.size();
return new ByteStringBuilder()
.putInt(len, ByteOrder.LITTLE_ENDIAN)
.append(bytes)
.result();
}
public static class FrameParser extends GraphStage<FlowShape<ByteString, ByteString>> {
public Inlet<ByteString> in = Inlet.create("FrameParser.in");
public Outlet<ByteString> out = Outlet.create("FrameParser.out");
private FlowShape<ByteString, ByteString> shape = FlowShape.of(in, out);
@Override
public FlowShape<ByteString, ByteString> shape() {
return shape;
}
@Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new GraphStageLogic(shape) {
// this holds the received but not yet parsed bytes
private ByteString stash = ByteString.empty();
// this holds the current message length or -1 if at a boundary
private int needed = -1;
{
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() throws Exception {
ByteString bytes = grab(in);
stash = stash.concat(bytes);
run();
}
@Override
public void onUpstreamFinish() throws Exception {
// either we are done
if (stash.isEmpty()) completeStage();
// or we still have bytes to emit
// wait with completion and let run() complete when the
// rest of the stash has been sent downstream
else if (isAvailable(out)) run();
}
});
setHandler(out, new AbstractOutHandler() {
@Override
public void onPull() throws Exception {
if (isClosed(in)) run();
else pull(in);
}
});
}
private void run() {
if (needed == -1) {
// are we at a boundary? then figure out next length
if (stash.size() < 4) {
if (isClosed(in)) completeStage();
else pull(in);
} else {
needed = stash.iterator().getInt(ByteOrder.LITTLE_ENDIAN);
stash = stash.drop(4);
run(); // cycle back to possibly already emit the next chunk
}
} else if (stash.size() < needed) {
// we are in the middle of a message, need more bytes
// or in is already closed and we cannot pull any more
if (isClosed(in)) completeStage();
else pull(in);
} else {
// we have enough to emit at least one message, so do it
final ByteString emit = stash.take(needed);
stash = stash.drop(needed);
needed = -1;
push(out, emit);
}
}
};
}
}
public final BidiFlow<ByteString, ByteString, ByteString, ByteString, NotUsed> framing =
BidiFlow.fromGraph(GraphDSL.create(b -> {
final FlowShape<ByteString, ByteString> top =
b.add(Flow.of(ByteString.class).map(BidiFlowDocTest::addLengthHeader));
final FlowShape<ByteString, ByteString> bottom =
b.add(Flow.of(ByteString.class).via(new FrameParser()));
return BidiShape.fromFlows(top, bottom);
}));
//#framing
@Test
public void mustCompose() throws Exception {
//#compose
/* construct protocol stack
* +------------------------------------+
* | stack |
* | |
* | +-------+ +---------+ |
* ~> O~~o | ~> | o~~O ~>
* Message | | codec | ByteString | framing | | ByteString
* <~ O~~o | <~ | o~~O <~
* | +-------+ +---------+ |
* +------------------------------------+
*/
final BidiFlow<Message, ByteString, ByteString, Message, NotUsed> stack =
codec.atop(framing);
// test it by plugging it into its own inverse and closing the right end
final Flow<Message, Message, NotUsed> pingpong =
Flow.of(Message.class).collect(new PFBuilder<Message, Message>()
.match(Ping.class, p -> new Pong(p.id))
.build()
);
final Flow<Message, Message, NotUsed> flow =
stack.atop(stack.reversed()).join(pingpong);
final CompletionStage<List<Message>> result = Source
.from(Arrays.asList(0, 1, 2))
.<Message> map(id -> new Ping(id))
.via(flow)
.grouped(10)
.runWith(Sink.<List<Message>> head(), mat);
assertArrayEquals(
new Message[] { new Pong(0), new Pong(1), new Pong(2) },
result.toCompletableFuture().get(1, TimeUnit.SECONDS).toArray(new Message[0]));
//#compose
}
}

View file

@ -0,0 +1,306 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream;
import java.util.Arrays;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage;
import akka.NotUsed;
import akka.stream.ClosedShape;
import jdocs.AbstractJavaTest;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import akka.actor.ActorSystem;
import akka.japi.Pair;
import akka.stream.*;
import akka.stream.javadsl.*;
import akka.stream.javadsl.Tcp.OutgoingConnection;
import akka.util.ByteString;
public class CompositionDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("CompositionDocTest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
@Test
public void nonNestedFlow() throws Exception {
//#non-nested-flow
Source.single(0)
.map(i -> i + 1)
.filter(i -> i != 0)
.map(i -> i - 2)
.to(Sink.fold(0, (acc, i) -> acc + i));
// ... where is the nesting?
//#non-nested-flow
}
@Test
public void nestedFlow() throws Exception {
//#nested-flow
final Source<Integer, NotUsed> nestedSource =
Source.single(0) // An atomic source
.map(i -> i + 1) // an atomic processing stage
.named("nestedSource"); // wraps up the current Source and gives it a name
final Flow<Integer, Integer, NotUsed> nestedFlow =
Flow.of(Integer.class).filter(i -> i != 0) // an atomic processing stage
.map(i -> i - 2) // another atomic processing stage
.named("nestedFlow"); // wraps up the Flow, and gives it a name
final Sink<Integer, NotUsed> nestedSink =
nestedFlow.to(Sink.fold(0, (acc, i) -> acc + i)) // wire an atomic sink to the nestedFlow
.named("nestedSink"); // wrap it up
// Create a RunnableGraph
final RunnableGraph<NotUsed> runnableGraph = nestedSource.to(nestedSink);
//#nested-flow
}
@Test
public void reusingComponents() throws Exception {
final Source<Integer, NotUsed> nestedSource =
Source.single(0) // An atomic source
.map(i -> i + 1) // an atomic processing stage
.named("nestedSource"); // wraps up the current Source and gives it a name
final Flow<Integer, Integer, NotUsed> nestedFlow =
Flow.of(Integer.class).filter(i -> i != 0) // an atomic processing stage
.map(i -> i - 2) // another atomic processing stage
.named("nestedFlow"); // wraps up the Flow, and gives it a name
final Sink<Integer, NotUsed> nestedSink =
nestedFlow.to(Sink.fold(0, (acc, i) -> acc + i)) // wire an atomic sink to the nestedFlow
.named("nestedSink"); // wrap it up
//#reuse
// Create a RunnableGraph from our components
final RunnableGraph<NotUsed> runnableGraph = nestedSource.to(nestedSink);
// Usage is uniform, no matter if modules are composite or atomic
final RunnableGraph<NotUsed> runnableGraph2 =
Source.single(0).to(Sink.fold(0, (acc, i) -> acc + i));
//#reuse
}
@Test
public void complexGraph() throws Exception {
//#complex-graph
RunnableGraph.fromGraph(
GraphDSL.create(builder -> {
final Outlet<Integer> A = builder.add(Source.single(0)).out();
final UniformFanOutShape<Integer, Integer> B = builder.add(Broadcast.create(2));
final UniformFanInShape<Integer, Integer> C = builder.add(Merge.create(2));
final FlowShape<Integer, Integer> D =
builder.add(Flow.of(Integer.class).map(i -> i + 1));
final UniformFanOutShape<Integer, Integer> E = builder.add(Balance.create(2));
final UniformFanInShape<Integer, Integer> F = builder.add(Merge.create(2));
final Inlet<Integer> G = builder.add(Sink.<Integer> foreach(System.out::println)).in();
builder.from(F).toFanIn(C);
builder.from(A).viaFanOut(B).viaFanIn(C).toFanIn(F);
builder.from(B).via(D).viaFanOut(E).toFanIn(F);
builder.from(E).toInlet(G);
return ClosedShape.getInstance();
}));
//#complex-graph
//#complex-graph-alt
RunnableGraph.fromGraph(
GraphDSL.create(builder -> {
final SourceShape<Integer> A = builder.add(Source.single(0));
final UniformFanOutShape<Integer, Integer> B = builder.add(Broadcast.create(2));
final UniformFanInShape<Integer, Integer> C = builder.add(Merge.create(2));
final FlowShape<Integer, Integer> D =
builder.add(Flow.of(Integer.class).map(i -> i + 1));
final UniformFanOutShape<Integer, Integer> E = builder.add(Balance.create(2));
final UniformFanInShape<Integer, Integer> F = builder.add(Merge.create(2));
final SinkShape<Integer> G = builder.add(Sink.foreach(System.out::println));
builder.from(F.out()).toInlet(C.in(0));
builder.from(A).toInlet(B.in());
builder.from(B.out(0)).toInlet(C.in(1));
builder.from(C.out()).toInlet(F.in(0));
builder.from(B.out(1)).via(D).toInlet(E.in());
builder.from(E.out(0)).toInlet(F.in(1));
builder.from(E.out(1)).to(G);
return ClosedShape.getInstance();
}));
//#complex-graph-alt
}
@Test
public void partialGraph() throws Exception {
//#partial-graph
final Graph<FlowShape<Integer, Integer>, NotUsed> partial =
GraphDSL.create(builder -> {
final UniformFanOutShape<Integer, Integer> B = builder.add(Broadcast.create(2));
final UniformFanInShape<Integer, Integer> C = builder.add(Merge.create(2));
final UniformFanOutShape<Integer, Integer> E = builder.add(Balance.create(2));
final UniformFanInShape<Integer, Integer> F = builder.add(Merge.create(2));
builder.from(F.out()).toInlet(C.in(0));
builder.from(B).viaFanIn(C).toFanIn(F);
builder.from(B).via(builder.add(Flow.of(Integer.class).map(i -> i + 1))).viaFanOut(E).toFanIn(F);
return new FlowShape<Integer, Integer>(B.in(), E.out(1));
});
//#partial-graph
//#partial-use
Source.single(0).via(partial).to(Sink.ignore());
//#partial-use
//#partial-flow-dsl
// Convert the partial graph of FlowShape to a Flow to get
// access to the fluid DSL (for example to be able to call .filter())
final Flow<Integer, Integer, NotUsed> flow = Flow.fromGraph(partial);
// Simple way to create a graph backed Source
final Source<Integer, NotUsed> source = Source.fromGraph(
GraphDSL.create(builder -> {
final UniformFanInShape<Integer, Integer> merge = builder.add(Merge.create(2));
builder.from(builder.add(Source.single(0))).toFanIn(merge);
builder.from(builder.add(Source.from(Arrays.asList(2, 3, 4)))).toFanIn(merge);
// Exposing exactly one output port
return new SourceShape<Integer>(merge.out());
})
);
// Building a Sink with a nested Flow, using the fluid DSL
final Sink<Integer, NotUsed> sink = Flow.of(Integer.class)
.map(i -> i * 2)
.drop(10)
.named("nestedFlow")
.to(Sink.head());
// Putting all together
final RunnableGraph<NotUsed> closed = source.via(flow.filter(i -> i > 1)).to(sink);
//#partial-flow-dsl
}
@Test
public void closedGraph() throws Exception {
//#embed-closed
final RunnableGraph<NotUsed> closed1 =
Source.single(0).to(Sink.foreach(System.out::println));
final RunnableGraph<NotUsed> closed2 =
RunnableGraph.fromGraph(
GraphDSL.create(builder -> {
final ClosedShape embeddedClosed = builder.add(closed1);
return embeddedClosed; // Could return ClosedShape.getInstance()
}));
//#embed-closed
}
//#mat-combine-4a
static class MyClass {
private CompletableFuture<Optional<Integer>> p;
private OutgoingConnection conn;
public MyClass(CompletableFuture<Optional<Integer>> p, OutgoingConnection conn) {
this.p = p;
this.conn = conn;
}
public void close() {
p.complete(Optional.empty());
}
}
static class Combiner {
static CompletionStage<MyClass> f(CompletableFuture<Optional<Integer>> p,
Pair<CompletionStage<OutgoingConnection>, CompletionStage<String>> rest) {
return rest.first().thenApply(c -> new MyClass(p, c));
}
}
//#mat-combine-4a
@Test
public void materializedValues() throws Exception {
//#mat-combine-1
// Materializes to CompletableFuture<Optional<Integer>> (red)
final Source<Integer, CompletableFuture<Optional<Integer>>> source = Source.<Integer>maybe();
// Materializes to NotUsed (black)
final Flow<Integer, Integer, NotUsed> flow1 = Flow.of(Integer.class).take(100);
// Materializes to CompletableFuture<Optional<Integer>> (red)
final Source<Integer, CompletableFuture<Optional<Integer>>> nestedSource =
source.viaMat(flow1, Keep.left()).named("nestedSource");
//#mat-combine-1
//#mat-combine-2
// Materializes to NotUsed (orange)
final Flow<Integer, ByteString, NotUsed> flow2 = Flow.of(Integer.class)
.map(i -> ByteString.fromString(i.toString()));
// Materializes to Future<OutgoingConnection> (yellow)
final Flow<ByteString, ByteString, CompletionStage<OutgoingConnection>> flow3 =
Tcp.get(system).outgoingConnection("localhost", 8080);
// Materializes to Future<OutgoingConnection> (yellow)
final Flow<Integer, ByteString, CompletionStage<OutgoingConnection>> nestedFlow =
flow2.viaMat(flow3, Keep.right()).named("nestedFlow");
//#mat-combine-2
//#mat-combine-3
// Materializes to Future<String> (green)
final Sink<ByteString, CompletionStage<String>> sink =
Sink.<String, ByteString> fold("", (acc, i) -> acc + i.utf8String());
// Materializes to Pair<Future<OutgoingConnection>, Future<String>> (blue)
final Sink<Integer, Pair<CompletionStage<OutgoingConnection>, CompletionStage<String>>> nestedSink =
nestedFlow.toMat(sink, Keep.both());
//#mat-combine-3
//#mat-combine-4b
// Materializes to Future<MyClass> (purple)
final RunnableGraph<CompletionStage<MyClass>> runnableGraph =
nestedSource.toMat(nestedSink, Combiner::f);
//#mat-combine-4b
}
@Test
public void attributes() throws Exception {
//#attributes-inheritance
final Source<Integer, NotUsed> nestedSource =
Source.single(0)
.map(i -> i + 1)
.named("nestedSource"); // Wrap, no inputBuffer set
final Flow<Integer, Integer, NotUsed> nestedFlow =
Flow.of(Integer.class).filter(i -> i != 0)
.via(Flow.of(Integer.class)
.map(i -> i - 2)
.withAttributes(Attributes.inputBuffer(4, 4))) // override
.named("nestedFlow"); // Wrap, no inputBuffer set
final Sink<Integer, NotUsed> nestedSink =
nestedFlow.to(Sink.fold(0, (acc, i) -> acc + i)) // wire an atomic sink to the nestedFlow
.withAttributes(Attributes.name("nestedSink")
.and(Attributes.inputBuffer(3, 3))); // override
//#attributes-inheritance
}
}

View file

@ -0,0 +1,282 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream;
import static org.junit.Assert.assertEquals;
import java.util.*;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
import akka.NotUsed;
import akka.japi.Pair;
import jdocs.AbstractJavaTest;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import akka.actor.ActorSystem;
import akka.actor.Cancellable;
import akka.dispatch.Futures;
import akka.stream.*;
import akka.stream.javadsl.*;
public class FlowDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("FlowDocTest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
@Test
public void sourceIsImmutable() throws Exception {
//#source-immutable
final Source<Integer, NotUsed> source =
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
source.map(x -> 0); // has no effect on source, since it's immutable
source.runWith(Sink.fold(0, (agg, next) -> agg + next), mat); // 55
// returns new Source<Integer>, with `map()` appended
final Source<Integer, NotUsed> zeroes = source.map(x -> 0);
final Sink<Integer, CompletionStage<Integer>> fold =
Sink.<Integer, Integer> fold(0, (agg, next) -> agg + next);
zeroes.runWith(fold, mat); // 0
//#source-immutable
int result = zeroes.runWith(fold, mat).toCompletableFuture().get(3, TimeUnit.SECONDS);
assertEquals(0, result);
}
@Test
public void materializationInSteps() throws Exception {
//#materialization-in-steps
final Source<Integer, NotUsed> source =
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
// note that the Future is scala.concurrent.Future
final Sink<Integer, CompletionStage<Integer>> sink =
Sink.<Integer, Integer> fold(0, (aggr, next) -> aggr + next);
// connect the Source to the Sink, obtaining a RunnableFlow
final RunnableGraph<CompletionStage<Integer>> runnable =
source.toMat(sink, Keep.right());
// materialize the flow
final CompletionStage<Integer> sum = runnable.run(mat);
//#materialization-in-steps
int result = sum.toCompletableFuture().get(3, TimeUnit.SECONDS);
assertEquals(55, result);
}
@Test
public void materializationRunWith() throws Exception {
//#materialization-runWith
final Source<Integer, NotUsed> source =
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
final Sink<Integer, CompletionStage<Integer>> sink =
Sink.<Integer, Integer> fold(0, (aggr, next) -> aggr + next);
// materialize the flow, getting the Sinks materialized value
final CompletionStage<Integer> sum = source.runWith(sink, mat);
//#materialization-runWith
int result = sum.toCompletableFuture().get(3, TimeUnit.SECONDS);
assertEquals(55, result);
}
@Test
public void materializedMapUnique() throws Exception {
//#stream-reuse
// connect the Source to the Sink, obtaining a RunnableGraph
final Sink<Integer, CompletionStage<Integer>> sink =
Sink.<Integer, Integer> fold(0, (aggr, next) -> aggr + next);
final RunnableGraph<CompletionStage<Integer>> runnable =
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)).toMat(sink, Keep.right());
// get the materialized value of the FoldSink
final CompletionStage<Integer> sum1 = runnable.run(mat);
final CompletionStage<Integer> sum2 = runnable.run(mat);
// sum1 and sum2 are different Futures!
//#stream-reuse
int result1 = sum1.toCompletableFuture().get(3, TimeUnit.SECONDS);
assertEquals(55, result1);
int result2 = sum2.toCompletableFuture().get(3, TimeUnit.SECONDS);
assertEquals(55, result2);
}
@Test
@SuppressWarnings("unused")
public void compoundSourceCannotBeUsedAsKey() throws Exception {
//#compound-source-is-not-keyed-runWith
final Object tick = new Object();
final FiniteDuration oneSecond = Duration.create(1, TimeUnit.SECONDS);
//akka.actor.Cancellable
final Source<Object, Cancellable> timer =
Source.tick(oneSecond, oneSecond, tick);
Sink.ignore().runWith(timer, mat);
final Source<String, Cancellable> timerMap = timer.map(t -> "tick");
// WRONG: returned type is not the timers Cancellable!
// Cancellable timerCancellable = Sink.ignore().runWith(timerMap, mat);
//#compound-source-is-not-keyed-runWith
//#compound-source-is-not-keyed-run
// retain the materialized map, in order to retrieve the timer's Cancellable
final Cancellable timerCancellable = timer.to(Sink.ignore()).run(mat);
timerCancellable.cancel();
//#compound-source-is-not-keyed-run
}
@Test
public void creatingSourcesSinks() throws Exception {
//#source-sink
// Create a source from an Iterable
List<Integer> list = new LinkedList<Integer>();
list.add(1);
list.add(2);
list.add(3);
Source.from(list);
// Create a source form a Future
Source.fromFuture(Futures.successful("Hello Streams!"));
// Create a source from a single element
Source.single("only one element");
// an empty source
Source.empty();
// Sink that folds over the stream and returns a Future
// of the final result in the MaterializedMap
Sink.fold(0, (Integer aggr, Integer next) -> aggr + next);
// Sink that returns a Future in the MaterializedMap,
// containing the first element of the stream
Sink.head();
// A Sink that consumes a stream without doing anything with the elements
Sink.ignore();
// A Sink that executes a side-effecting call for every element of the stream
Sink.foreach(System.out::println);
//#source-sink
}
@Test
public void variousWaysOfConnecting() throws Exception {
//#flow-connecting
// Explicitly creating and wiring up a Source, Sink and Flow
Source.from(Arrays.asList(1, 2, 3, 4))
.via(Flow.of(Integer.class).map(elem -> elem * 2))
.to(Sink.foreach(System.out::println));
// Starting from a Source
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(1, 2, 3, 4))
.map(elem -> elem * 2);
source.to(Sink.foreach(System.out::println));
// Starting from a Sink
final Sink<Integer, NotUsed> sink = Flow.of(Integer.class)
.map(elem -> elem * 2).to(Sink.foreach(System.out::println));
Source.from(Arrays.asList(1, 2, 3, 4)).to(sink);
//#flow-connecting
}
@Test
public void transformingMaterialized() throws Exception {
FiniteDuration oneSecond = FiniteDuration.apply(1, TimeUnit.SECONDS);
Flow<Integer, Integer, Cancellable> throttler =
Flow.fromGraph(GraphDSL.create(
Source.tick(oneSecond, oneSecond, ""),
(b, tickSource) -> {
FanInShape2<String, Integer, Integer> zip = b.add(ZipWith.create(Keep.right()));
b.from(tickSource).toInlet(zip.in0());
return FlowShape.of(zip.in1(), zip.out());
}));
//#flow-mat-combine
// An empty source that can be shut down explicitly from the outside
Source<Integer, CompletableFuture<Optional<Integer>>> source = Source.<Integer>maybe();
// A flow that internally throttles elements to 1/second, and returns a Cancellable
// which can be used to shut down the stream
Flow<Integer, Integer, Cancellable> flow = throttler;
// A sink that returns the first element of a stream in the returned Future
Sink<Integer, CompletionStage<Integer>> sink = Sink.head();
// By default, the materialized value of the leftmost stage is preserved
RunnableGraph<CompletableFuture<Optional<Integer>>> r1 = source.via(flow).to(sink);
// Simple selection of materialized values by using Keep.right
RunnableGraph<Cancellable> r2 = source.viaMat(flow, Keep.right()).to(sink);
RunnableGraph<CompletionStage<Integer>> r3 = source.via(flow).toMat(sink, Keep.right());
// Using runWith will always give the materialized values of the stages added
// by runWith() itself
CompletionStage<Integer> r4 = source.via(flow).runWith(sink, mat);
CompletableFuture<Optional<Integer>> r5 = flow.to(sink).runWith(source, mat);
Pair<CompletableFuture<Optional<Integer>>, CompletionStage<Integer>> r6 = flow.runWith(source, sink, mat);
// Using more complex combinations
RunnableGraph<Pair<CompletableFuture<Optional<Integer>>, Cancellable>> r7 =
source.viaMat(flow, Keep.both()).to(sink);
RunnableGraph<Pair<CompletableFuture<Optional<Integer>>, CompletionStage<Integer>>> r8 =
source.via(flow).toMat(sink, Keep.both());
RunnableGraph<Pair<Pair<CompletableFuture<Optional<Integer>>, Cancellable>, CompletionStage<Integer>>> r9 =
source.viaMat(flow, Keep.both()).toMat(sink, Keep.both());
RunnableGraph<Pair<Cancellable, CompletionStage<Integer>>> r10 =
source.viaMat(flow, Keep.right()).toMat(sink, Keep.both());
// It is also possible to map over the materialized values. In r9 we had a
// doubly nested pair, but we want to flatten it out
RunnableGraph<Cancellable> r11 =
r9.mapMaterializedValue( (nestedTuple) -> {
CompletableFuture<Optional<Integer>> p = nestedTuple.first().first();
Cancellable c = nestedTuple.first().second();
CompletionStage<Integer> f = nestedTuple.second();
// Picking the Cancellable, but we could also construct a domain class here
return c;
});
//#flow-mat-combine
}
public void fusingAndAsync() {
//#flow-async
Source.range(1, 3)
.map(x -> x + 1).async()
.map(x -> x * 2)
.to(Sink.ignore());
//#flow-async
}
}

View file

@ -0,0 +1,141 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream;
import static org.junit.Assert.assertEquals;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import akka.NotUsed;
import jdocs.AbstractJavaTest;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import akka.actor.ActorSystem;
import akka.stream.ActorMaterializer;
import akka.stream.ActorMaterializerSettings;
import akka.stream.Materializer;
import akka.stream.Supervision;
import akka.stream.javadsl.Flow;
import akka.stream.ActorAttributes;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.japi.function.Function;
public class FlowErrorDocTest extends AbstractJavaTest {
private static ActorSystem system;
@BeforeClass
public static void setup() {
system = ActorSystem.create("FlowDocTest");
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
}
@Test(expected = ExecutionException.class)
public void demonstrateFailStream() throws Exception {
//#stop
final Materializer mat = ActorMaterializer.create(system);
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(0, 1, 2, 3, 4, 5))
.map(elem -> 100 / elem);
final Sink<Integer, CompletionStage<Integer>> fold =
Sink.<Integer, Integer> fold(0, (acc, elem) -> acc + elem);
final CompletionStage<Integer> result = source.runWith(fold, mat);
// division by zero will fail the stream and the
// result here will be a Future completed with Failure(ArithmeticException)
//#stop
result.toCompletableFuture().get(3, TimeUnit.SECONDS);
}
@Test
public void demonstrateResumeStream() throws Exception {
//#resume
final Function<Throwable, Supervision.Directive> decider = exc -> {
if (exc instanceof ArithmeticException)
return Supervision.resume();
else
return Supervision.stop();
};
final Materializer mat = ActorMaterializer.create(
ActorMaterializerSettings.create(system).withSupervisionStrategy(decider),
system);
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(0, 1, 2, 3, 4, 5))
.map(elem -> 100 / elem);
final Sink<Integer, CompletionStage<Integer>> fold =
Sink.fold(0, (acc, elem) -> acc + elem);
final CompletionStage<Integer> result = source.runWith(fold, mat);
// the element causing division by zero will be dropped
// result here will be a Future completed with Success(228)
//#resume
assertEquals(Integer.valueOf(228), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
}
@Test
public void demonstrateResumeSectionStream() throws Exception {
//#resume-section
final Materializer mat = ActorMaterializer.create(system);
final Function<Throwable, Supervision.Directive> decider = exc -> {
if (exc instanceof ArithmeticException)
return Supervision.resume();
else
return Supervision.stop();
};
final Flow<Integer, Integer, NotUsed> flow =
Flow.of(Integer.class).filter(elem -> 100 / elem < 50).map(elem -> 100 / (5 - elem))
.withAttributes(ActorAttributes.withSupervisionStrategy(decider));
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(0, 1, 2, 3, 4, 5))
.via(flow);
final Sink<Integer, CompletionStage<Integer>> fold =
Sink.<Integer, Integer> fold(0, (acc, elem) -> acc + elem);
final CompletionStage<Integer> result = source.runWith(fold, mat);
// the elements causing division by zero will be dropped
// result here will be a Future completed with Success(150)
//#resume-section
assertEquals(Integer.valueOf(150), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
}
@Test
public void demonstrateRestartSectionStream() throws Exception {
//#restart-section
final Materializer mat = ActorMaterializer.create(system);
final Function<Throwable, Supervision.Directive> decider = exc -> {
if (exc instanceof IllegalArgumentException)
return Supervision.restart();
else
return Supervision.stop();
};
final Flow<Integer, Integer, NotUsed> flow =
Flow.of(Integer.class).scan(0, (acc, elem) -> {
if (elem < 0) throw new IllegalArgumentException("negative not allowed");
else return acc + elem;
})
.withAttributes(ActorAttributes.withSupervisionStrategy(decider));
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(1, 3, -1, 5, 7))
.via(flow);
final CompletionStage<List<Integer>> result = source.grouped(1000)
.runWith(Sink.<List<Integer>>head(), mat);
// the negative element cause the scan stage to be restarted,
// i.e. start from 0 again
// result here will be a Future completed with Success(List(0, 1, 4, 0, 5, 12))
//#restart-section
assertEquals(
Arrays.asList(0, 1, 4, 0, 5, 12),
result.toCompletableFuture().get(3, TimeUnit.SECONDS));
}
}

View file

@ -0,0 +1,149 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream;
import static org.junit.Assert.assertEquals;
import akka.NotUsed;
import jdocs.AbstractJavaTest;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import akka.actor.ActorSystem;
import akka.stream.*;
import akka.stream.javadsl.*;
public class FlowParallelismDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("FlowParallellismDocTest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
static class ScoopOfBatter {}
static class HalfCookedPancake {}
static class Pancake {}
//#pipelining
Flow<ScoopOfBatter, HalfCookedPancake, NotUsed> fryingPan1 =
Flow.of(ScoopOfBatter.class).map(batter -> new HalfCookedPancake());
Flow<HalfCookedPancake, Pancake, NotUsed> fryingPan2 =
Flow.of(HalfCookedPancake.class).map(halfCooked -> new Pancake());
//#pipelining
@Test
public void demonstratePipelining() {
//#pipelining
// With the two frying pans we can fully cook pancakes
Flow<ScoopOfBatter, Pancake, NotUsed> pancakeChef =
fryingPan1.async().via(fryingPan2.async());
//#pipelining
}
@Test
public void demonstrateParallelism() {
//#parallelism
Flow<ScoopOfBatter, Pancake, NotUsed> fryingPan =
Flow.of(ScoopOfBatter.class).map(batter -> new Pancake());
Flow<ScoopOfBatter, Pancake, NotUsed> pancakeChef =
Flow.fromGraph(GraphDSL.create(b -> {
final UniformFanInShape<Pancake, Pancake> mergePancakes =
b.add(Merge.create(2));
final UniformFanOutShape<ScoopOfBatter, ScoopOfBatter> dispatchBatter =
b.add(Balance.create(2));
// Using two frying pans in parallel, both fully cooking a pancake from the batter.
// We always put the next scoop of batter to the first frying pan that becomes available.
b.from(dispatchBatter.out(0)).via(b.add(fryingPan.async())).toInlet(mergePancakes.in(0));
// Notice that we used the "fryingPan" flow without importing it via builder.add().
// Flows used this way are auto-imported, which in this case means that the two
// uses of "fryingPan" mean actually different stages in the graph.
b.from(dispatchBatter.out(1)).via(b.add(fryingPan.async())).toInlet(mergePancakes.in(1));
return FlowShape.of(dispatchBatter.in(), mergePancakes.out());
}));
//#parallelism
}
@Test
public void parallelPipeline() {
//#parallel-pipeline
Flow<ScoopOfBatter, Pancake, NotUsed> pancakeChef =
Flow.fromGraph(GraphDSL.create(b -> {
final UniformFanInShape<Pancake, Pancake> mergePancakes =
b.add(Merge.create(2));
final UniformFanOutShape<ScoopOfBatter, ScoopOfBatter> dispatchBatter =
b.add(Balance.create(2));
// Using two pipelines, having two frying pans each, in total using
// four frying pans
b.from(dispatchBatter.out(0))
.via(b.add(fryingPan1.async()))
.via(b.add(fryingPan2.async()))
.toInlet(mergePancakes.in(0));
b.from(dispatchBatter.out(1))
.via(b.add(fryingPan1.async()))
.via(b.add(fryingPan2.async()))
.toInlet(mergePancakes.in(1));
return FlowShape.of(dispatchBatter.in(), mergePancakes.out());
}));
//#parallel-pipeline
}
@Test
public void pipelinedParallel() {
//#pipelined-parallel
Flow<ScoopOfBatter, HalfCookedPancake, NotUsed> pancakeChefs1 =
Flow.fromGraph(GraphDSL.create(b -> {
final UniformFanInShape<HalfCookedPancake, HalfCookedPancake> mergeHalfCooked =
b.add(Merge.create(2));
final UniformFanOutShape<ScoopOfBatter, ScoopOfBatter> dispatchBatter =
b.add(Balance.create(2));
// Two chefs work with one frying pan for each, half-frying the pancakes then putting
// them into a common pool
b.from(dispatchBatter.out(0)).via(b.add(fryingPan1.async())).toInlet(mergeHalfCooked.in(0));
b.from(dispatchBatter.out(1)).via(b.add(fryingPan1.async())).toInlet(mergeHalfCooked.in(1));
return FlowShape.of(dispatchBatter.in(), mergeHalfCooked.out());
}));
Flow<HalfCookedPancake, Pancake, NotUsed> pancakeChefs2 =
Flow.fromGraph(GraphDSL.create(b -> {
final UniformFanInShape<Pancake, Pancake> mergePancakes =
b.add(Merge.create(2));
final UniformFanOutShape<HalfCookedPancake, HalfCookedPancake> dispatchHalfCooked =
b.add(Balance.create(2));
// Two chefs work with one frying pan for each, finishing the pancakes then putting
// them into a common pool
b.from(dispatchHalfCooked.out(0)).via(b.add(fryingPan2.async())).toInlet(mergePancakes.in(0));
b.from(dispatchHalfCooked.out(1)).via(b.add(fryingPan2.async())).toInlet(mergePancakes.in(1));
return FlowShape.of(dispatchHalfCooked.in(), mergePancakes.out());
}));
Flow<ScoopOfBatter, Pancake, NotUsed> kitchen =
pancakeChefs1.via(pancakeChefs2);
//#pipelined-parallel
}
}

View file

@ -0,0 +1,160 @@
package jdocs.stream;
import java.util.Arrays;
import akka.NotUsed;
import jdocs.AbstractJavaTest;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import akka.actor.ActorSystem;
import akka.stream.*;
import akka.stream.javadsl.*;
import akka.stream.scaladsl.MergePreferred.MergePreferredShape;
public class GraphCyclesDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("GraphCyclesDocTest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
final static SilenceSystemOut.System System = SilenceSystemOut.get();
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(1, 2, 3, 4, 5));
@Test
public void demonstrateDeadlockedCycle() {
//#deadlocked
// WARNING! The graph below deadlocks!
final Flow<Integer, Integer, NotUsed> printFlow =
Flow.of(Integer.class).map(s -> {
System.out.println(s);
return s;
});
RunnableGraph.fromGraph(GraphDSL.create(b -> {
final UniformFanInShape<Integer, Integer> merge = b.add(Merge.create(2));
final UniformFanOutShape<Integer, Integer> bcast = b.add(Broadcast.create(2));
final Outlet<Integer> src = b.add(source).out();
final FlowShape<Integer, Integer> printer = b.add(printFlow);
final SinkShape<Integer> ignore = b.add(Sink.ignore());
b.from(src).viaFanIn(merge).via(printer).viaFanOut(bcast).to(ignore);
b.to(merge) .fromFanOut(bcast);
return ClosedShape.getInstance();
}));
//#deadlocked
}
@Test
public void demonstrateUnfairCycle() {
final Flow<Integer, Integer, NotUsed> printFlow =
Flow.of(Integer.class).map(s -> {
System.out.println(s);
return s;
});
//#unfair
// WARNING! The graph below stops consuming from "source" after a few steps
RunnableGraph.fromGraph(GraphDSL.create(b -> {
final MergePreferredShape<Integer> merge = b.add(MergePreferred.create(1));
final UniformFanOutShape<Integer, Integer> bcast = b.add(Broadcast.create(2));
final Outlet<Integer> src = b.add(source).out();
final FlowShape<Integer, Integer> printer = b.add(printFlow);
final SinkShape<Integer> ignore = b.add(Sink.ignore());
b.from(src).viaFanIn(merge).via(printer).viaFanOut(bcast).to(ignore);
b.to(merge.preferred()).fromFanOut(bcast);
return ClosedShape.getInstance();
}));
//#unfair
}
@Test
public void demonstrateDroppingCycle() {
final Flow<Integer, Integer, NotUsed> printFlow =
Flow.of(Integer.class).map(s -> {
System.out.println(s);
return s;
});
//#dropping
RunnableGraph.fromGraph(GraphDSL.create(b -> {
final UniformFanInShape<Integer, Integer> merge = b.add(Merge.create(2));
final UniformFanOutShape<Integer, Integer> bcast = b.add(Broadcast.create(2));
final FlowShape<Integer, Integer> droppyFlow = b.add(
Flow.of(Integer.class).buffer(10, OverflowStrategy.dropHead()));
final Outlet<Integer> src = b.add(source).out();
final FlowShape<Integer, Integer> printer = b.add(printFlow);
final SinkShape<Integer> ignore = b.add(Sink.ignore());
b.from(src).viaFanIn(merge).via(printer).viaFanOut(bcast).to(ignore);
b.to(merge).via(droppyFlow).fromFanOut(bcast);
return ClosedShape.getInstance();
}));
//#dropping
}
@Test
public void demonstrateZippingCycle() {
final Flow<Integer, Integer, NotUsed> printFlow =
Flow.of(Integer.class).map(s -> {
System.out.println(s);
return s;
});
//#zipping-dead
// WARNING! The graph below never processes any elements
RunnableGraph.fromGraph(GraphDSL.create(b -> {
final FanInShape2<Integer, Integer, Integer> zip =
b.add(ZipWith.create((Integer left, Integer right) -> left));
final UniformFanOutShape<Integer, Integer> bcast = b.add(Broadcast.create(2));
final FlowShape<Integer, Integer> printer = b.add(printFlow);
final SinkShape<Integer> ignore = b.add(Sink.ignore());
b.from(b.add(source)).toInlet(zip.in0());
b.from(zip.out()).via(printer).viaFanOut(bcast).to(ignore);
b.to(zip.in1()) .fromFanOut(bcast);
return ClosedShape.getInstance();
}));
//#zipping-dead
}
@Test
public void demonstrateLiveZippingCycle() {
final Flow<Integer, Integer, NotUsed> printFlow =
Flow.of(Integer.class).map(s -> {
System.out.println(s);
return s;
});
//#zipping-live
RunnableGraph.fromGraph(GraphDSL.create(b -> {
final FanInShape2<Integer, Integer, Integer> zip =
b.add(ZipWith.create((Integer left, Integer right) -> left));
final UniformFanOutShape<Integer, Integer> bcast = b.add(Broadcast.create(2));
final UniformFanInShape<Integer, Integer> concat = b.add(Concat.create());
final FlowShape<Integer, Integer> printer = b.add(printFlow);
final SinkShape<Integer> ignore = b.add(Sink.ignore());
b.from(b.add(source)).toInlet(zip.in0());
b.from(zip.out()).via(printer).viaFanOut(bcast).to(ignore);
b.to(zip.in1()).viaFanIn(concat).from(b.add(Source.single(1)));
b.to(concat).fromFanOut(bcast);
return ClosedShape.getInstance();
}));
//#zipping-live
}
}

View file

@ -0,0 +1,169 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream;
import static org.junit.Assert.*;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
import akka.NotUsed;
import akka.stream.ClosedShape;
import akka.stream.SourceShape;
import jdocs.AbstractJavaTest;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import akka.actor.ActorSystem;
import akka.japi.Pair;
import akka.stream.*;
import akka.stream.javadsl.*;
public class GraphDSLDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("GraphDSLDocTest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
@Test
public void demonstrateBuildSimpleGraph() throws Exception {
//#simple-graph-dsl
final Source<Integer, NotUsed> in = Source.from(Arrays.asList(1, 2, 3, 4, 5));
final Sink<List<String>, CompletionStage<List<String>>> sink = Sink.head();
final Flow<Integer, Integer, NotUsed> f1 = Flow.of(Integer.class).map(elem -> elem + 10);
final Flow<Integer, Integer, NotUsed> f2 = Flow.of(Integer.class).map(elem -> elem + 20);
final Flow<Integer, String, NotUsed> f3 = Flow.of(Integer.class).map(elem -> elem.toString());
final Flow<Integer, Integer, NotUsed> f4 = Flow.of(Integer.class).map(elem -> elem + 30);
final RunnableGraph<CompletionStage<List<String>>> result =
RunnableGraph.fromGraph(
GraphDSL // create() function binds sink, out which is sink's out port and builder DSL
.create( // we need to reference out's shape in the builder DSL below (in to() function)
sink, // previously created sink (Sink)
(builder, out) -> { // variables: builder (GraphDSL.Builder) and out (SinkShape)
final UniformFanOutShape<Integer, Integer> bcast = builder.add(Broadcast.create(2));
final UniformFanInShape<Integer, Integer> merge = builder.add(Merge.create(2));
final Outlet<Integer> source = builder.add(in).out();
builder.from(source).via(builder.add(f1))
.viaFanOut(bcast).via(builder.add(f2)).viaFanIn(merge)
.via(builder.add(f3.grouped(1000))).to(out); // to() expects a SinkShape
builder.from(bcast).via(builder.add(f4)).toFanIn(merge);
return ClosedShape.getInstance();
}));
//#simple-graph-dsl
final List<String> list = result.run(mat).toCompletableFuture().get(3, TimeUnit.SECONDS);
final String[] res = list.toArray(new String[] {});
Arrays.sort(res, null);
assertArrayEquals(new String[] { "31", "32", "33", "34", "35", "41", "42", "43", "44", "45" }, res);
}
@Test
@SuppressWarnings("unused")
public void demonstrateConnectErrors() {
try {
//#simple-graph
final RunnableGraph<NotUsed> g =
RunnableGraph.<NotUsed>fromGraph(
GraphDSL
.create((b) -> {
final SourceShape<Integer> source1 = b.add(Source.from(Arrays.asList(1, 2, 3, 4, 5)));
final SourceShape<Integer> source2 = b.add(Source.from(Arrays.asList(1, 2, 3, 4, 5)));
final FanInShape2<Integer, Integer, Pair<Integer, Integer>> zip = b.add(Zip.create());
b.from(source1).toInlet(zip.in0());
b.from(source2).toInlet(zip.in1());
return ClosedShape.getInstance();
}
)
);
// unconnected zip.out (!) => "The inlets [] and outlets [] must correspond to the inlets [] and outlets [ZipWith2.out]"
//#simple-graph
org.junit.Assert.fail("expected IllegalArgumentException");
} catch (IllegalStateException e) {
assertTrue(e != null && e.getMessage() != null && e.getMessage().contains("ZipWith2.out"));
}
}
@Test
public void demonstrateReusingFlowInGraph() throws Exception {
//#graph-dsl-reusing-a-flow
final Sink<Integer, CompletionStage<Integer>> topHeadSink = Sink.head();
final Sink<Integer, CompletionStage<Integer>> bottomHeadSink = Sink.head();
final Flow<Integer, Integer, NotUsed> sharedDoubler = Flow.of(Integer.class).map(elem -> elem * 2);
final RunnableGraph<Pair<CompletionStage<Integer>, CompletionStage<Integer>>> g =
RunnableGraph.<Pair<CompletionStage<Integer>, CompletionStage<Integer>>>fromGraph(
GraphDSL.create(
topHeadSink, // import this sink into the graph
bottomHeadSink, // and this as well
Keep.both(),
(b, top, bottom) -> {
final UniformFanOutShape<Integer, Integer> bcast =
b.add(Broadcast.create(2));
b.from(b.add(Source.single(1))).viaFanOut(bcast)
.via(b.add(sharedDoubler)).to(top);
b.from(bcast).via(b.add(sharedDoubler)).to(bottom);
return ClosedShape.getInstance();
}
)
);
//#graph-dsl-reusing-a-flow
final Pair<CompletionStage<Integer>, CompletionStage<Integer>> pair = g.run(mat);
assertEquals(Integer.valueOf(2), pair.first().toCompletableFuture().get(3, TimeUnit.SECONDS));
assertEquals(Integer.valueOf(2), pair.second().toCompletableFuture().get(3, TimeUnit.SECONDS));
}
@Test
public void demonstrateMatValue() throws Exception {
//#graph-dsl-matvalue
final Sink<Integer, CompletionStage<Integer>> foldSink = Sink.<Integer, Integer> fold(0, (a, b) -> {
return a + b;
});
final Flow<CompletionStage<Integer>, Integer, NotUsed> flatten =
Flow.<CompletionStage<Integer>>create().mapAsync(4, x -> x);
final Flow<Integer, Integer, CompletionStage<Integer>> foldingFlow = Flow.fromGraph(
GraphDSL.create(foldSink,
(b, fold) -> {
return FlowShape.of(
fold.in(),
b.from(b.materializedValue()).via(b.add(flatten)).out());
}));
//#graph-dsl-matvalue
//#graph-dsl-matvalue-cycle
// This cannot produce any value:
final Source<Integer, CompletionStage<Integer>> cyclicSource = Source.fromGraph(
GraphDSL.create(foldSink,
(b, fold) -> {
// - Fold cannot complete until its upstream mapAsync completes
// - mapAsync cannot complete until the materialized Future produced by
// fold completes
// As a result this Source will never emit anything, and its materialited
// Future will never complete
b.from(b.materializedValue()).via(b.add(flatten)).to(fold);
return SourceShape.of(b.from(b.materializedValue()).via(b.add(flatten)).out());
}));
//#graph-dsl-matvalue-cycle
}
}

View file

@ -0,0 +1,739 @@
package jdocs.stream;
import akka.Done;
import akka.NotUsed;
import akka.actor.ActorSystem;
//#imports
import akka.dispatch.Futures;
import akka.japi.Option;
import akka.japi.Predicate;
import akka.japi.function.Procedure;
import akka.stream.*;
import akka.stream.javadsl.*;
import akka.stream.stage.*;
//#imports
import akka.stream.testkit.TestPublisher;
import akka.stream.testkit.TestSubscriber;
import akka.japi.Function;
import jdocs.AbstractJavaTest;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.reactivestreams.Subscription;
import scala.Tuple2;
import scala.concurrent.ExecutionContext;
import scala.concurrent.Promise;
import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import java.util.*;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.assertEquals;
public class GraphStageDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("GraphStageDocTest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
//#simple-source
public class NumbersSource extends GraphStage<SourceShape<Integer>> {
// Define the (sole) output port of this stage
public final Outlet<Integer> out = Outlet.create("NumbersSource.out");
// Define the shape of this stage, which is SourceShape with the port we defined above
private final SourceShape<Integer> shape = SourceShape.of(out);
@Override
public SourceShape<Integer> shape() {
return shape;
}
// This is where the actual (possibly stateful) logic is created
@Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new GraphStageLogic(shape()) {
// All state MUST be inside the GraphStageLogic,
// never inside the enclosing GraphStage.
// This state is safe to access and modify from all the
// callbacks that are provided by GraphStageLogic and the
// registered handlers.
private int counter = 1;
{
setHandler(out, new AbstractOutHandler() {
@Override
public void onPull() throws Exception {
push(out, counter);
counter += 1;
}
});
}
};
}
}
//#simple-source
//#simple-sink
public class StdoutSink extends GraphStage<SinkShape<Integer>> {
public final Inlet<Integer> in = Inlet.create("StdoutSink.in");
private final SinkShape<Integer> shape = SinkShape.of(in);
@Override
public SinkShape<Integer> shape() {
return shape;
}
@Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new GraphStageLogic(shape()) {
// This requests one element at the Sink startup.
@Override
public void preStart() {
pull(in);
}
{
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() throws Exception {
Integer element = grab(in);
System.out.println(element);
pull(in);
}
});
}
};
}
}
//#simple-sink
@Test
public void demonstrateCustomSourceUsage() throws Exception {
//#simple-source-usage
// A GraphStage is a proper Graph, just like what GraphDSL.create would return
Graph<SourceShape<Integer>, NotUsed> sourceGraph = new NumbersSource();
// Create a Source from the Graph to access the DSL
Source<Integer, NotUsed> mySource = Source.fromGraph(sourceGraph);
// Returns 55
CompletionStage<Integer> result1 = mySource.take(10).runFold(0, (sum, next) -> sum + next, mat);
// The source is reusable. This returns 5050
CompletionStage<Integer> result2 = mySource.take(100).runFold(0, (sum, next) -> sum + next, mat);
//#simple-source-usage
assertEquals(result1.toCompletableFuture().get(3, TimeUnit.SECONDS), (Integer) 55);
assertEquals(result2.toCompletableFuture().get(3, TimeUnit.SECONDS), (Integer) 5050);
}
@Test
public void demonstrateCustomSinkUsage() throws Exception {
Graph<SinkShape<Integer>, NotUsed> sinkGraph = new StdoutSink();
Sink<Integer, NotUsed> mySink = Sink.fromGraph(sinkGraph);
Source.from(Arrays.asList(1, 2, 3)).runWith(mySink, mat);
}
//#one-to-one
public class Map<A, B> extends GraphStage<FlowShape<A, B>> {
private final Function<A, B> f;
public Map(Function<A, B> f) {
this.f = f;
}
public final Inlet<A> in = Inlet.create("Map.in");
public final Outlet<B> out = Outlet.create("Map.out");
private final FlowShape<A, B> shape = FlowShape.of(in, out);
@Override
public FlowShape<A,B> shape() {
return shape;
}
@Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new GraphStageLogic(shape) {
{
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() throws Exception {
push(out, f.apply(grab(in)));
}
});
setHandler(out, new AbstractOutHandler() {
@Override
public void onPull() throws Exception {
pull(in);
}
});
}
};
}
}
//#one-to-one
@Test
public void demonstrateOneToOne() throws Exception {
// tests:
final Graph<FlowShape<String, Integer>, NotUsed> stringLength =
Flow.fromGraph(new Map<String, Integer>(new Function<String, Integer>() {
@Override
public Integer apply(String str) {
return str.length();
}
}));
CompletionStage<Integer> result =
Source.from(Arrays.asList("one", "two", "three"))
.via(stringLength)
.runFold(0, (sum, n) -> sum + n, mat);
assertEquals(new Integer(11), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
}
//#many-to-one
public final class Filter<A> extends GraphStage<FlowShape<A, A>> {
private final Predicate<A> p;
public Filter(Predicate<A> p) {
this.p = p;
}
public final Inlet<A> in = Inlet.create("Filter.in");
public final Outlet<A> out = Outlet.create("Filter.out");
private final FlowShape<A, A> shape = FlowShape.of(in, out);
@Override
public FlowShape<A, A> shape() {
return shape;
}
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new GraphStageLogic(shape) {
{
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() {
A elem = grab(in);
if (p.test(elem)) {
push(out, elem);
} else {
pull(in);
}
}
});
setHandler(out, new AbstractOutHandler() {
@Override
public void onPull() throws Exception {
pull(in);
}
});
}
};
}
}
//#many-to-one
@Test
public void demonstrateAManyToOneElementGraphStage() throws Exception {
// tests:
Graph<FlowShape<Integer, Integer>, NotUsed> evenFilter =
Flow.fromGraph(new Filter<Integer>(n -> n % 2 == 0));
CompletionStage<Integer> result =
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6))
.via(evenFilter)
.runFold(0, (elem, sum) -> sum + elem, mat);
assertEquals(new Integer(12), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
}
//#one-to-many
public class Duplicator<A> extends GraphStage<FlowShape<A, A>> {
public final Inlet<A> in = Inlet.create("Duplicator.in");
public final Outlet<A> out = Outlet.create("Duplicator.out");
private final FlowShape<A, A> shape = FlowShape.of(in, out);
@Override
public FlowShape<A, A> shape() {
return shape;
}
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new GraphStageLogic(shape) {
// Again: note that all mutable state
// MUST be inside the GraphStageLogic
Option<A> lastElem = Option.none();
{
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() {
A elem = grab(in);
lastElem = Option.some(elem);
push(out, elem);
}
@Override
public void onUpstreamFinish() {
if (lastElem.isDefined()) {
emit(out, lastElem.get());
}
complete(out);
}
});
setHandler(out, new AbstractOutHandler() {
@Override
public void onPull() throws Exception {
if (lastElem.isDefined()) {
push(out, lastElem.get());
lastElem = Option.none();
} else {
pull(in);
}
}
});
}
};
}
}
//#one-to-many
@Test
public void demonstrateAOneToManyElementGraphStage() throws Exception {
// tests:
Graph<FlowShape<Integer, Integer>, NotUsed> duplicator =
Flow.fromGraph(new Duplicator<Integer>());
CompletionStage<Integer> result =
Source.from(Arrays.asList(1, 2, 3))
.via(duplicator)
.runFold(0, (n, sum) -> n + sum, mat);
assertEquals(new Integer(12), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
}
//#simpler-one-to-many
public class Duplicator2<A> extends GraphStage<FlowShape<A, A>> {
public final Inlet<A> in = Inlet.create("Duplicator.in");
public final Outlet<A> out = Outlet.create("Duplicator.out");
private final FlowShape<A, A> shape = FlowShape.of(in, out);
@Override
public FlowShape<A, A> shape() {
return shape;
}
@Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new GraphStageLogic(shape) {
{
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() {
A elem = grab(in);
// this will temporarily suspend this handler until the two elems
// are emitted and then reinstates it
emitMultiple(out, Arrays.asList(elem, elem).iterator());
}
});
setHandler(out, new AbstractOutHandler() {
@Override
public void onPull() throws Exception {
pull(in);
}
});
}
};
}
}
//#simpler-one-to-many
@Test
public void demonstrateASimplerOneToManyStage() throws Exception {
// tests:
Graph<FlowShape<Integer, Integer>, NotUsed> duplicator =
Flow.fromGraph(new Duplicator2<Integer>());
CompletionStage<Integer> result =
Source.from(Arrays.asList(1, 2, 3))
.via(duplicator)
.runFold(0, (n, sum) -> n + sum, mat);
assertEquals(new Integer(12), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
}
@Test
public void demonstrateChainingOfGraphStages() throws Exception {
Graph<SinkShape<Integer>, CompletionStage<String>> sink = Sink.fold("", (acc, n) -> acc + n.toString());
//#graph-stage-chain
CompletionStage<String> resultFuture = Source.from(Arrays.asList(1,2,3,4,5))
.via(new Filter<Integer>((n) -> n % 2 == 0))
.via(new Duplicator<Integer>())
.via(new Map<Integer, Integer>((n) -> n / 2))
.runWith(sink, mat);
//#graph-stage-chain
assertEquals("1122", resultFuture.toCompletableFuture().get(3, TimeUnit.SECONDS));
}
//#async-side-channel
// will close upstream in all materializations of the stage instance
// when the completion stage completes
public class KillSwitch<A> extends GraphStage<FlowShape<A, A>> {
private final CompletionStage<Done> switchF;
public KillSwitch(CompletionStage<Done> switchF) {
this.switchF = switchF;
}
public final Inlet<A> in = Inlet.create("KillSwitch.in");
public final Outlet<A> out = Outlet.create("KillSwitch.out");
private final FlowShape<A, A> shape = FlowShape.of(in, out);
@Override
public FlowShape<A, A> shape() {
return shape;
}
@Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new GraphStageLogic(shape) {
{
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() {
push(out, grab(in));
}
});
setHandler(out, new AbstractOutHandler() {
@Override
public void onPull() throws Exception {
pull(in);
}
});
}
@Override
public void preStart() {
AsyncCallback<Done> callback = createAsyncCallback(new Procedure<Done>() {
@Override
public void apply(Done param) throws Exception {
completeStage();
}
});
ExecutionContext ec = system.dispatcher();
switchF.thenAccept(callback::invoke);
}
};
}
}
//#async-side-channel
@Test
public void demonstrateAnAsynchronousSideChannel() throws Exception{
// tests:
TestSubscriber.Probe<Integer> out = TestSubscriber.probe(system);
TestPublisher.Probe<Integer> in = TestPublisher.probe(0, system);
CompletableFuture<Done> switchF = new CompletableFuture<>();
Graph<FlowShape<Integer, Integer>, NotUsed> killSwitch =
Flow.fromGraph(new KillSwitch<>(switchF));
Source.fromPublisher(in).via(killSwitch).to(Sink.fromSubscriber(out)).run(mat);
out.request(1);
in.sendNext(1);
out.expectNext(1);
switchF.complete(Done.getInstance());
out.expectComplete();
}
//#timed
// each time an event is pushed through it will trigger a period of silence
public class TimedGate<A> extends GraphStage<FlowShape<A, A>> {
private final FiniteDuration silencePeriod;
public TimedGate(FiniteDuration silencePeriod) {
this.silencePeriod = silencePeriod;
}
public final Inlet<A> in = Inlet.create("TimedGate.in");
public final Outlet<A> out = Outlet.create("TimedGate.out");
private final FlowShape<A, A> shape = FlowShape.of(in, out);
@Override
public FlowShape<A, A> shape() {
return shape;
}
@Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new TimerGraphStageLogic(shape) {
private boolean open = false;
{
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() throws Exception {
A elem = grab(in);
if (open) pull(in);
else {
push(out, elem);
open = true;
scheduleOnce("key", silencePeriod);
}
}
});
setHandler(out, new AbstractOutHandler() {
@Override
public void onPull() throws Exception {
pull(in);
}
});
}
@Override
public void onTimer(Object key) {
if (key.equals("key")) {
open = false;
}
}
};
}
}
//#timed
public void demonstrateAGraphStageWithATimer() throws Exception {
// tests:
CompletionStage<Integer> result =
Source.from(Arrays.asList(1, 2, 3))
.via(new TimedGate<>(Duration.create(2, "seconds")))
.takeWithin(Duration.create(250, "millis"))
.runFold(0, (n, sum) -> n + sum, mat);
assertEquals(new Integer(1), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
}
//#materialized
public class FirstValue<A> extends GraphStageWithMaterializedValue<FlowShape<A, A>, CompletionStage<A>> {
public final Inlet<A> in = Inlet.create("FirstValue.in");
public final Outlet<A> out = Outlet.create("FirstValue.out");
private final FlowShape<A, A> shape = FlowShape.of(in, out);
@Override
public FlowShape<A, A> shape() {
return shape;
}
@Override
public Tuple2<GraphStageLogic, CompletionStage<A>> createLogicAndMaterializedValue(Attributes inheritedAttributes) {
Promise<A> promise = Futures.promise();
GraphStageLogic logic = new GraphStageLogic(shape) {
{
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() {
A elem = grab(in);
promise.success(elem);
push(out, elem);
// replace handler with one just forwarding
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() {
push(out, grab(in));
}
});
}
});
setHandler(out, new AbstractOutHandler() {
@Override
public void onPull() throws Exception {
pull(in);
}
});
}
};
return new Tuple2(logic, promise.future());
}
}
//#materialized
public void demonstrateACustomMaterializedValue() throws Exception {
// tests:
RunnableGraph<CompletionStage<Integer>> flow = Source.from(Arrays.asList(1, 2, 3))
.viaMat(new FirstValue(), Keep.right())
.to(Sink.ignore());
CompletionStage<Integer> result = flow.run(mat);
assertEquals(new Integer(1), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
}
//#detached
public class TwoBuffer<A> extends GraphStage<FlowShape<A, A>> {
public final Inlet<A> in = Inlet.create("TwoBuffer.in");
public final Outlet<A> out = Outlet.create("TwoBuffer.out");
private final FlowShape<A, A> shape = FlowShape.of(in, out);
@Override
public FlowShape<A, A> shape() {
return shape;
}
@Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new GraphStageLogic(shape) {
private final int SIZE = 2;
private Queue<A> buffer = new ArrayDeque<>(SIZE);
private boolean downstreamWaiting = false;
private boolean isBufferFull() {
return buffer.size() == SIZE;
}
@Override
public void preStart() {
// a detached stage needs to start upstream demand
// itself as it is not triggered by downstream demand
pull(in);
}
{
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() {
A elem = grab(in);
buffer.add(elem);
if (downstreamWaiting) {
downstreamWaiting = false;
A bufferedElem = buffer.poll();
push(out, bufferedElem);
}
if (!isBufferFull()) {
pull(in);
}
}
@Override
public void onUpstreamFinish() {
if (!buffer.isEmpty()) {
// emit the rest if possible
emitMultiple(out, buffer.iterator());
}
completeStage();
}
});
setHandler(out, new AbstractOutHandler() {
@Override
public void onPull() throws Exception {
if (buffer.isEmpty()) {
downstreamWaiting = true;
} else {
A elem = buffer.poll();
push(out, elem);
}
if (!isBufferFull() && !hasBeenPulled(in)) {
pull(in);
}
}
});
}
};
}
}
//#detached
public void demonstrateADetachedGraphStage() throws Exception {
// tests:
CompletionStage<Integer> result1 = Source.from(Arrays.asList(1, 2, 3))
.via(new TwoBuffer<>())
.runFold(0, (acc, n) -> acc + n, mat);
assertEquals(new Integer(6), result1.toCompletableFuture().get(3, TimeUnit.SECONDS));
TestSubscriber.ManualProbe<Integer> subscriber = TestSubscriber.manualProbe(system);
TestPublisher.Probe<Integer> publisher = TestPublisher.probe(0, system);
RunnableGraph<NotUsed> flow2 =
Source.fromPublisher(publisher)
.via(new TwoBuffer<>())
.to(Sink.fromSubscriber(subscriber));
flow2.run(mat);
Subscription sub = subscriber.expectSubscription();
// this happens even though the subscriber has not signalled any demand
publisher.sendNext(1);
publisher.sendNext(2);
sub.cancel();
}
}

View file

@ -0,0 +1,64 @@
/*
* Copyright (C) 2016-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream;
import akka.actor.ActorSystem;
import akka.stream.Attributes;
import akka.stream.Materializer;
import akka.stream.Outlet;
import akka.stream.SourceShape;
import akka.stream.stage.*;
import jdocs.AbstractJavaTest;
import org.junit.Test;
import java.util.concurrent.ThreadLocalRandom;
public class GraphStageLoggingDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer mat;
@Test
public void compileOnlyTestClass() throws Exception { }
//#stage-with-logging
public class RandomLettersSource extends GraphStage<SourceShape<String>> {
public final Outlet<String> out = Outlet.create("RandomLettersSource.in");
private final SourceShape<String> shape = SourceShape.of(out);
@Override
public SourceShape<String> shape() {
return shape;
}
@Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new GraphStageLogicWithLogging(shape()) {
{
setHandler(out, new AbstractOutHandler() {
@Override
public void onPull() throws Exception {
final String s = nextChar();// ASCII lower case letters
// `log` is obtained from materializer automatically (via StageLogging)
log().debug("Randomly generated: [{}]", s);
push(out, s);
}
private String nextChar() {
final char i = (char) ThreadLocalRandom.current().nextInt('a', 'z' + 1);
return String.valueOf(i);
}
});
}
};
}
}
//#stage-with-logging
}

View file

@ -0,0 +1,140 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream;
import akka.Done;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.actor.Cancellable;
import akka.japi.Pair;
import akka.stream.ActorMaterializer;
import akka.stream.KillSwitches;
import akka.stream.Materializer;
import akka.stream.UniqueKillSwitch;
import akka.stream.javadsl.*;
import jdocs.AbstractJavaTest;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.duration.FiniteDuration;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
public class HubDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer materializer;
@BeforeClass
public static void setup() {
system = ActorSystem.create("GraphDSLDocTest");
materializer = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
materializer = null;
}
@Test
public void dynamicMerge() {
//#merge-hub
// A simple consumer that will print to the console for now
Sink<String, CompletionStage<Done>> consumer = Sink.foreach(System.out::println);
// Attach a MergeHub Source to the consumer. This will materialize to a
// corresponding Sink.
RunnableGraph<Sink<String, NotUsed>> runnableGraph =
MergeHub.of(String.class, 16).to(consumer);
// By running/materializing the consumer we get back a Sink, and hence
// now have access to feed elements into it. This Sink can be materialized
// any number of times, and every element that enters the Sink will
// be consumed by our consumer.
Sink<String, NotUsed> toConsumer = runnableGraph.run(materializer);
Source.single("Hello!").runWith(toConsumer, materializer);
Source.single("Hub!").runWith(toConsumer, materializer);
//#merge-hub
}
@Test
public void dynamicBroadcast() {
// Used to be able to clean up the running stream
ActorMaterializer materializer = ActorMaterializer.create(system);
//#broadcast-hub
// A simple producer that publishes a new "message" every second
Source<String, Cancellable> producer = Source.tick(
FiniteDuration.create(1, TimeUnit.SECONDS),
FiniteDuration.create(1, TimeUnit.SECONDS),
"New message"
);
// Attach a BroadcastHub Sink to the producer. This will materialize to a
// corresponding Source.
// (We need to use toMat and Keep.right since by default the materialized
// value to the left is used)
RunnableGraph<Source<String, NotUsed>> runnableGraph =
producer.toMat(BroadcastHub.of(String.class, 256), Keep.right());
// By running/materializing the producer, we get back a Source, which
// gives us access to the elements published by the producer.
Source<String, NotUsed> fromProducer = runnableGraph.run(materializer);
// Print out messages from the producer in two independent consumers
fromProducer.runForeach(msg -> System.out.println("consumer1: " + msg), materializer);
fromProducer.runForeach(msg -> System.out.println("consumer2: " + msg), materializer);
//#broadcast-hub
// Cleanup
materializer.shutdown();
}
@Test
public void mergeBroadcastCombination() {
//#pub-sub-1
// Obtain a Sink and Source which will publish and receive from the "bus" respectively.
Pair<Sink<String, NotUsed>, Source<String, NotUsed>> sinkAndSource =
MergeHub.of(String.class, 16)
.toMat(BroadcastHub.of(String.class, 256), Keep.both())
.run(materializer);
Sink<String, NotUsed> sink = sinkAndSource.first();
Source<String, NotUsed> source = sinkAndSource.second();
//#pub-sub-1
//#pub-sub-2
// Ensure that the Broadcast output is dropped if there are no listening parties.
// If this dropping Sink is not attached, then the broadcast hub will not drop any
// elements itself when there are no subscribers, backpressuring the producer instead.
source.runWith(Sink.ignore(), materializer);
//#pub-sub-2
//#pub-sub-3
// We create now a Flow that represents a publish-subscribe channel using the above
// started stream as its "topic". We add two more features, external cancellation of
// the registration and automatic cleanup for very slow subscribers.
Flow<String, String, UniqueKillSwitch> busFlow =
Flow.fromSinkAndSource(sink, source)
.joinMat(KillSwitches.singleBidi(), Keep.right())
.backpressureTimeout(FiniteDuration.create(1, TimeUnit.SECONDS));
//#pub-sub-3
//#pub-sub-4
UniqueKillSwitch killSwitch =
Source.repeat("Hello World!")
.viaMat(busFlow, Keep.right())
.to(Sink.foreach(System.out::println))
.run(materializer);
// Shut down externally
killSwitch.shutdown();
//#pub-sub-4
}
}

View file

@ -0,0 +1,639 @@
/*
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream;
import akka.NotUsed;
import akka.actor.*;
import akka.stream.*;
import akka.stream.javadsl.*;
import akka.testkit.TestProbe;
import akka.testkit.javadsl.TestKit;
import akka.util.Timeout;
import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory;
import jdocs.AbstractJavaTest;
import jdocs.stream.TwitterStreamQuickstartDocTest.Model.Author;
import jdocs.stream.TwitterStreamQuickstartDocTest.Model.Tweet;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import static akka.pattern.PatternsCS.ask;
import static jdocs.stream.TwitterStreamQuickstartDocTest.Model.AKKA;
import static jdocs.stream.TwitterStreamQuickstartDocTest.Model.tweets;
import static junit.framework.TestCase.assertTrue;
public class IntegrationDocTest extends AbstractJavaTest {
private static final SilenceSystemOut.System System = SilenceSystemOut.get();
static ActorSystem system;
static Materializer mat;
static ActorRef ref;
@BeforeClass
public static void setup() {
final Config config = ConfigFactory.parseString("" +
"blocking-dispatcher { \n" +
" executor = thread-pool-executor \n" +
" thread-pool-executor { \n" +
" core-pool-size-min = 10 \n" +
" core-pool-size-max = 10 \n" +
" } \n" +
"} \n" +
"akka.actor.default-mailbox.mailbox-type = akka.dispatch.UnboundedMailbox\n");
system = ActorSystem.create("ActorPublisherDocTest", config);
mat = ActorMaterializer.create(system);
ref = system.actorOf(Props.create(Translator.class));
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
ref = null;
}
class AddressSystem {
//#email-address-lookup
public CompletionStage<Optional<String>> lookupEmail(String handle)
//#email-address-lookup
{
return CompletableFuture.completedFuture(Optional.of(handle + "@somewhere.com"));
}
//#phone-lookup
public CompletionStage<Optional<String>> lookupPhoneNumber(String handle)
//#phone-lookup
{
return CompletableFuture.completedFuture(Optional.of("" + handle.hashCode()));
}
}
class AddressSystem2 {
//#email-address-lookup2
public CompletionStage<String> lookupEmail(String handle)
//#email-address-lookup2
{
return CompletableFuture.completedFuture(handle + "@somewhere.com");
}
}
static class Email {
public final String to;
public final String title;
public final String body;
public Email(String to, String title, String body) {
this.to = to;
this.title = title;
this.body = body;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Email email = (Email) o;
if (body != null ? !body.equals(email.body) : email.body != null) {
return false;
}
if (title != null ? !title.equals(email.title) : email.title != null) {
return false;
}
if (to != null ? !to.equals(email.to) : email.to != null) {
return false;
}
return true;
}
@Override
public int hashCode() {
int result = to != null ? to.hashCode() : 0;
result = 31 * result + (title != null ? title.hashCode() : 0);
result = 31 * result + (body != null ? body.hashCode() : 0);
return result;
}
}
static class TextMessage {
public final String to;
public final String body;
TextMessage(String to, String body) {
this.to = to;
this.body = body;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
TextMessage that = (TextMessage) o;
if (body != null ? !body.equals(that.body) : that.body != null) {
return false;
}
if (to != null ? !to.equals(that.to) : that.to != null) {
return false;
}
return true;
}
@Override
public int hashCode() {
int result = to != null ? to.hashCode() : 0;
result = 31 * result + (body != null ? body.hashCode() : 0);
return result;
}
}
static class EmailServer {
public final ActorRef probe;
public EmailServer(ActorRef probe) {
this.probe = probe;
}
//#email-server-send
public CompletionStage<Email> send(Email email) {
// ...
//#email-server-send
probe.tell(email.to, ActorRef.noSender());
return CompletableFuture.completedFuture(email);
//#email-server-send
}
//#email-server-send
}
static class SmsServer {
public final ActorRef probe;
public SmsServer(ActorRef probe) {
this.probe = probe;
}
//#sms-server-send
public boolean send(TextMessage text) {
// ...
//#sms-server-send
probe.tell(text.to, ActorRef.noSender());
//#sms-server-send
return true;
}
//#sms-server-send
}
static class Save {
public final Tweet tweet;
Save(Tweet tweet) {
this.tweet = tweet;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Save save = (Save) o;
if (tweet != null ? !tweet.equals(save.tweet) : save.tweet != null) {
return false;
}
return true;
}
@Override
public int hashCode() {
return tweet != null ? tweet.hashCode() : 0;
}
}
static class SaveDone {
public static SaveDone INSTANCE = new SaveDone();
private SaveDone() {
}
}
static class DatabaseService extends AbstractActor {
public final ActorRef probe;
public DatabaseService(ActorRef probe) {
this.probe = probe;
}
@Override
public Receive createReceive() {
return receiveBuilder()
.match(Save.class, s -> {
probe.tell(s.tweet.author.handle, ActorRef.noSender());
getSender().tell(SaveDone.INSTANCE, getSelf());
})
.build();
}
}
//#sometimes-slow-service
static class SometimesSlowService {
private final Executor ec;
public SometimesSlowService(Executor ec) {
this.ec = ec;
}
private final AtomicInteger runningCount = new AtomicInteger();
public CompletionStage<String> convert(String s) {
System.out.println("running: " + s + "(" + runningCount.incrementAndGet() + ")");
return CompletableFuture.supplyAsync(() -> {
if (!s.isEmpty() && Character.isLowerCase(s.charAt(0)))
try { Thread.sleep(500); } catch (InterruptedException e) {}
else
try { Thread.sleep(20); } catch (InterruptedException e) {}
System.out.println("completed: " + s + "(" + runningCount.decrementAndGet() + ")");
return s.toUpperCase();
}, ec);
}
}
//#sometimes-slow-service
//#ask-actor
static class Translator extends AbstractActor {
@Override
public Receive createReceive() {
return receiveBuilder()
.match(String.class, word -> {
// ... process message
String reply = word.toUpperCase();
// reply to the ask
getSender().tell(reply, getSelf());
})
.build();
}
}
//#ask-actor
@SuppressWarnings("unchecked")
@Test
public void mapAsyncPlusAsk() throws Exception {
//#mapAsync-ask
Source<String, NotUsed> words =
Source.from(Arrays.asList("hello", "hi"));
Timeout askTimeout = Timeout.apply(5, TimeUnit.SECONDS);
words
.mapAsync(5, elem -> ask(ref, elem, askTimeout))
.map(elem -> (String) elem)
// continue processing of the replies from the actor
.map(elem -> elem.toLowerCase())
.runWith(Sink.ignore(), mat);
//#mapAsync-ask
}
@Test
public void callingExternalServiceWithMapAsync() throws Exception {
new TestKit(system) {
final TestProbe probe = new TestProbe(system);
final AddressSystem addressSystem = new AddressSystem();
final EmailServer emailServer = new EmailServer(probe.ref());
{
//#tweet-authors
final Source<Author, NotUsed> authors = tweets
.filter(t -> t.hashtags().contains(AKKA))
.map(t -> t.author);
//#tweet-authors
//#email-addresses-mapAsync
final Source<String, NotUsed> emailAddresses = authors
.mapAsync(4, author -> addressSystem.lookupEmail(author.handle))
.filter(o -> o.isPresent())
.map(o -> o.get());
//#email-addresses-mapAsync
//#send-emails
final RunnableGraph<NotUsed> sendEmails = emailAddresses
.mapAsync(4, address ->
emailServer.send(new Email(address, "Akka", "I like your tweet")))
.to(Sink.ignore());
sendEmails.run(mat);
//#send-emails
probe.expectMsg("rolandkuhn@somewhere.com");
probe.expectMsg("patriknw@somewhere.com");
probe.expectMsg("bantonsson@somewhere.com");
probe.expectMsg("drewhk@somewhere.com");
probe.expectMsg("ktosopl@somewhere.com");
probe.expectMsg("mmartynas@somewhere.com");
probe.expectMsg("akkateam@somewhere.com");
}
};
}
@Test
@SuppressWarnings("unused")
public void callingExternalServiceWithMapAsyncAndSupervision() throws Exception {
new TestKit(system) {
final AddressSystem2 addressSystem = new AddressSystem2();
{
final Source<Author, NotUsed> authors = tweets
.filter(t -> t.hashtags().contains(AKKA))
.map(t -> t.author);
//#email-addresses-mapAsync-supervision
final Attributes resumeAttrib =
ActorAttributes.withSupervisionStrategy(Supervision.getResumingDecider());
final Flow<Author, String, NotUsed> lookupEmail =
Flow.of(Author.class)
.mapAsync(4, author -> addressSystem.lookupEmail(author.handle))
.withAttributes(resumeAttrib);
final Source<String, NotUsed> emailAddresses = authors.via(lookupEmail);
//#email-addresses-mapAsync-supervision
}
};
}
@Test
public void callingExternalServiceWithMapAsyncUnordered() throws Exception {
new TestKit(system) {
final TestProbe probe = new TestProbe(system);
final AddressSystem addressSystem = new AddressSystem();
final EmailServer emailServer = new EmailServer(probe.ref());
{
//#external-service-mapAsyncUnordered
final Source<Author, NotUsed> authors =
tweets
.filter(t -> t.hashtags().contains(AKKA))
.map(t -> t.author);
final Source<String, NotUsed> emailAddresses =
authors
.mapAsyncUnordered(4, author -> addressSystem.lookupEmail(author.handle))
.filter(o -> o.isPresent())
.map(o -> o.get());
final RunnableGraph<NotUsed> sendEmails =
emailAddresses
.mapAsyncUnordered(4, address ->
emailServer.send(new Email(address, "Akka", "I like your tweet")))
.to(Sink.ignore());
sendEmails.run(mat);
//#external-service-mapAsyncUnordered
}
};
}
@Test
public void carefulManagedBlockingWithMapAsync() throws Exception {
new TestKit(system) {
final AddressSystem addressSystem = new AddressSystem();
final EmailServer emailServer = new EmailServer(getRef());
final SmsServer smsServer = new SmsServer(getRef());
{
final Source<Author, NotUsed> authors =
tweets
.filter(t -> t.hashtags().contains(AKKA))
.map(t -> t.author);
final Source<String, NotUsed> phoneNumbers = authors.mapAsync(4, author -> addressSystem.lookupPhoneNumber(author.handle))
.filter(o -> o.isPresent())
.map(o -> o.get());
//#blocking-mapAsync
final Executor blockingEc = system.dispatchers().lookup("blocking-dispatcher");
final RunnableGraph<NotUsed> sendTextMessages =
phoneNumbers
.mapAsync(4, phoneNo -> CompletableFuture.supplyAsync(() ->
smsServer.send(new TextMessage(phoneNo, "I like your tweet")), blockingEc))
.to(Sink.ignore());
sendTextMessages.run(mat);
//#blocking-mapAsync
final List<Object> got = receiveN(7);
final Set<Object> set = new HashSet<>(got);
assertTrue(set.contains(String.valueOf("rolandkuhn".hashCode())));
assertTrue(set.contains(String.valueOf("patriknw".hashCode())));
assertTrue(set.contains(String.valueOf("bantonsson".hashCode())));
assertTrue(set.contains(String.valueOf("drewhk".hashCode())));
assertTrue(set.contains(String.valueOf("ktosopl".hashCode())));
assertTrue(set.contains(String.valueOf("mmartynas".hashCode())));
assertTrue(set.contains(String.valueOf("akkateam".hashCode())));
}
};
}
@Test
public void carefulManagedBlockingWithMap() throws Exception {
new TestKit(system) {
final TestProbe probe = new TestProbe(system);
final AddressSystem addressSystem = new AddressSystem();
final EmailServer emailServer = new EmailServer(probe.ref());
final SmsServer smsServer = new SmsServer(probe.ref());
{
final Source<Author, NotUsed> authors =
tweets
.filter(t -> t.hashtags().contains(AKKA))
.map(t -> t.author);
final Source<String, NotUsed> phoneNumbers = authors.mapAsync(4, author -> addressSystem.lookupPhoneNumber(author.handle))
.filter(o -> o.isPresent())
.map(o -> o.get());
//#blocking-map
final Flow<String, Boolean, NotUsed> send =
Flow.of(String.class)
.map(phoneNo -> smsServer.send(new TextMessage(phoneNo, "I like your tweet")))
.withAttributes(ActorAttributes.dispatcher("blocking-dispatcher"));
final RunnableGraph<?> sendTextMessages =
phoneNumbers.via(send).to(Sink.ignore());
sendTextMessages.run(mat);
//#blocking-map
probe.expectMsg(String.valueOf("rolandkuhn".hashCode()));
probe.expectMsg(String.valueOf("patriknw".hashCode()));
probe.expectMsg(String.valueOf("bantonsson".hashCode()));
probe.expectMsg(String.valueOf("drewhk".hashCode()));
probe.expectMsg(String.valueOf("ktosopl".hashCode()));
probe.expectMsg(String.valueOf("mmartynas".hashCode()));
probe.expectMsg(String.valueOf("akkateam".hashCode()));
}
};
}
@Test
public void callingActorServiceWithMapAsync() throws Exception {
new TestKit(system) {
final TestProbe probe = new TestProbe(system);
final EmailServer emailServer = new EmailServer(probe.ref());
final ActorRef database = system.actorOf(Props.create(DatabaseService.class, probe.ref()), "db");
{
//#save-tweets
final Source<Tweet, NotUsed> akkaTweets = tweets.filter(t -> t.hashtags().contains(AKKA));
final RunnableGraph<NotUsed> saveTweets =
akkaTweets
.mapAsync(4, tweet -> ask(database, new Save(tweet), 300))
.to(Sink.ignore());
//#save-tweets
saveTweets.run(mat);
probe.expectMsg("rolandkuhn");
probe.expectMsg("patriknw");
probe.expectMsg("bantonsson");
probe.expectMsg("drewhk");
probe.expectMsg("ktosopl");
probe.expectMsg("mmartynas");
probe.expectMsg("akkateam");
}
};
}
@Test
public void illustrateOrderingAndParallelismOfMapAsync() throws Exception {
new TestKit(system) {
final TestProbe probe = new TestProbe(system);
final EmailServer emailServer = new EmailServer(probe.ref());
class MockSystem {
class Println {
public <T> void println(T s) {
if (s.toString().startsWith("after:"))
probe.ref().tell(s, ActorRef.noSender());
}
}
public final Println out = new Println();
}
private final MockSystem System = new MockSystem();
{
//#sometimes-slow-mapAsync
final Executor blockingEc = system.dispatchers().lookup("blocking-dispatcher");
final SometimesSlowService service = new SometimesSlowService(blockingEc);
final ActorMaterializer mat = ActorMaterializer.create(
ActorMaterializerSettings.create(system).withInputBuffer(4, 4), system);
Source.from(Arrays.asList("a", "B", "C", "D", "e", "F", "g", "H", "i", "J"))
.map(elem -> { System.out.println("before: " + elem); return elem; })
.mapAsync(4, service::convert)
.runForeach(elem -> System.out.println("after: " + elem), mat);
//#sometimes-slow-mapAsync
probe.expectMsg("after: A");
probe.expectMsg("after: B");
probe.expectMsg("after: C");
probe.expectMsg("after: D");
probe.expectMsg("after: E");
probe.expectMsg("after: F");
probe.expectMsg("after: G");
probe.expectMsg("after: H");
probe.expectMsg("after: I");
probe.expectMsg("after: J");
}
};
}
@Test
public void illustrateOrderingAndParallelismOfMapAsyncUnordered() throws Exception {
new TestKit(system) {
final EmailServer emailServer = new EmailServer(getRef());
class MockSystem {
class Println {
public <T> void println(T s) {
if (s.toString().startsWith("after:"))
getRef().tell(s, ActorRef.noSender());
}
}
public final Println out = new Println();
}
private final MockSystem System = new MockSystem();
{
//#sometimes-slow-mapAsyncUnordered
final Executor blockingEc = system.dispatchers().lookup("blocking-dispatcher");
final SometimesSlowService service = new SometimesSlowService(blockingEc);
final ActorMaterializer mat = ActorMaterializer.create(
ActorMaterializerSettings.create(system).withInputBuffer(4, 4), system);
Source.from(Arrays.asList("a", "B", "C", "D", "e", "F", "g", "H", "i", "J"))
.map(elem -> { System.out.println("before: " + elem); return elem; })
.mapAsyncUnordered(4, service::convert)
.runForeach(elem -> System.out.println("after: " + elem), mat);
//#sometimes-slow-mapAsyncUnordered
final List<Object> got = receiveN(10);
final Set<Object> set = new HashSet<>(got);
assertTrue(set.contains("after: A"));
assertTrue(set.contains("after: B"));
assertTrue(set.contains("after: C"));
assertTrue(set.contains("after: D"));
assertTrue(set.contains("after: E"));
assertTrue(set.contains("after: F"));
assertTrue(set.contains("after: G"));
assertTrue(set.contains("after: H"));
assertTrue(set.contains("after: I"));
assertTrue(set.contains("after: J"));
}
};
}
}

View file

@ -0,0 +1,152 @@
package jdocs.stream;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.japi.Pair;
import akka.stream.*;
import akka.stream.javadsl.Keep;
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 scala.concurrent.duration.FiniteDuration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.assertEquals;
class KillSwitchDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("GraphDSLDocTest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
@Test
public void compileOnlyTest() {
}
public void uniqueKillSwitchShutdownExample() throws Exception {
//#unique-shutdown
final Source<Integer, NotUsed> countingSrc =
Source.from(new ArrayList<>(Arrays.asList(1, 2, 3, 4)))
.delay(FiniteDuration.apply(1, TimeUnit.SECONDS), DelayOverflowStrategy.backpressure());
final Sink<Integer, CompletionStage<Integer>> lastSnk = Sink.last();
final Pair<UniqueKillSwitch, CompletionStage<Integer>> stream = countingSrc
.viaMat(KillSwitches.single(), Keep.right())
.toMat(lastSnk, Keep.both()).run(mat);
final UniqueKillSwitch killSwitch = stream.first();
final CompletionStage<Integer> completionStage = stream.second();
doSomethingElse();
killSwitch.shutdown();
final int finalCount =
completionStage.toCompletableFuture().get(1, TimeUnit.SECONDS);
assertEquals(2, finalCount);
//#unique-shutdown
}
public static void uniqueKillSwitchAbortExample() throws Exception {
//#unique-abort
final Source<Integer, NotUsed> countingSrc =
Source.from(new ArrayList<>(Arrays.asList(1, 2, 3, 4)))
.delay(FiniteDuration.apply(1, TimeUnit.SECONDS), DelayOverflowStrategy.backpressure());
final Sink<Integer, CompletionStage<Integer>> lastSnk = Sink.last();
final Pair<UniqueKillSwitch, CompletionStage<Integer>> stream = countingSrc
.viaMat(KillSwitches.single(), Keep.right())
.toMat(lastSnk, Keep.both()).run(mat);
final UniqueKillSwitch killSwitch = stream.first();
final CompletionStage<Integer> completionStage = stream.second();
final Exception error = new Exception("boom!");
killSwitch.abort(error);
final int result =
completionStage.toCompletableFuture().exceptionally(e -> -1).get(1, TimeUnit.SECONDS);
assertEquals(-1, result);
//#unique-abort
}
public void sharedKillSwitchShutdownExample() throws Exception {
//#shared-shutdown
final Source<Integer, NotUsed> countingSrc =
Source.from(new ArrayList<>(Arrays.asList(1, 2, 3, 4)))
.delay(FiniteDuration.apply(1, TimeUnit.SECONDS), DelayOverflowStrategy.backpressure());
final Sink<Integer, CompletionStage<Integer>> lastSnk = Sink.last();
final SharedKillSwitch killSwitch = KillSwitches.shared("my-kill-switch");
final CompletionStage<Integer> completionStage = countingSrc
.viaMat(killSwitch.flow(), Keep.right())
.toMat(lastSnk, Keep.right()).run(mat);
final CompletionStage<Integer> completionStageDelayed = countingSrc
.delay(FiniteDuration.apply(1, TimeUnit.SECONDS), DelayOverflowStrategy.backpressure())
.viaMat(killSwitch.flow(), Keep.right())
.toMat(lastSnk, Keep.right()).run(mat);
doSomethingElse();
killSwitch.shutdown();
final int finalCount =
completionStage.toCompletableFuture().get(1, TimeUnit.SECONDS);
final int finalCountDelayed =
completionStageDelayed.toCompletableFuture().get(1, TimeUnit.SECONDS);
assertEquals(2, finalCount);
assertEquals(1, finalCountDelayed);
//#shared-shutdown
}
public static void sharedKillSwitchAbortExample() throws Exception {
//#shared-abort
final Source<Integer, NotUsed> countingSrc =
Source.from(new ArrayList<>(Arrays.asList(1, 2, 3, 4)))
.delay(FiniteDuration.apply(1, TimeUnit.SECONDS), DelayOverflowStrategy.backpressure());
final Sink<Integer, CompletionStage<Integer>> lastSnk = Sink.last();
final SharedKillSwitch killSwitch = KillSwitches.shared("my-kill-switch");
final CompletionStage<Integer> completionStage1 = countingSrc
.viaMat(killSwitch.flow(), Keep.right())
.toMat(lastSnk, Keep.right()).run(mat);
final CompletionStage<Integer> completionStage2 = countingSrc
.viaMat(killSwitch.flow(), Keep.right())
.toMat(lastSnk, Keep.right()).run(mat);
final Exception error = new Exception("boom!");
killSwitch.abort(error);
final int result1 =
completionStage1.toCompletableFuture().exceptionally(e -> -1).get(1, TimeUnit.SECONDS);
final int result2 =
completionStage2.toCompletableFuture().exceptionally(e -> -1).get(1, TimeUnit.SECONDS);
assertEquals(-1, result1);
assertEquals(-1, result2);
//#shared-abort
}
private static void doSomethingElse(){
}
}

View file

@ -0,0 +1,9 @@
package jdocs.stream;
//#main-app
public class Main {
public static void main(String[] argv) {
// Code here
}
}
//#main-app

View file

@ -0,0 +1,38 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream;
import java.util.stream.Stream;
import akka.NotUsed;
import akka.japi.Pair;
import akka.stream.javadsl.*;
//#asPublisher-import
import static akka.stream.javadsl.AsPublisher.*;
//#asPublisher-import
public class MigrationsJava {
public static void main(String[] args) {
//#expand-continually
Flow.of(Integer.class).expand(in -> Stream.iterate(in, i -> i).iterator());
//#expand-continually
//#expand-state
Flow.of(Integer.class).expand(in ->
Stream.iterate(new Pair<>(in, 0),
p -> new Pair<>(in, p.second() + 1)).iterator());
//#expand-state
//#asPublisher
Sink.asPublisher(WITH_FANOUT); // instead of Sink.asPublisher(true)
Sink.asPublisher(WITHOUT_FANOUT); // instead of Sink.asPublisher(false)
//#asPublisher
//#async
Flow<Integer, Integer, NotUsed> flow = Flow.of(Integer.class).map(n -> n + 1);
Source.range(1, 10).via(flow.async());
//#async
}
}

View file

@ -0,0 +1,93 @@
/**
* Copyright (C) 2016-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream;
//#stream-imports
import akka.stream.*;
import akka.stream.javadsl.*;
//#stream-imports
//#other-imports
import akka.Done;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.util.ByteString;
import java.nio.file.Paths;
import java.math.BigInteger;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import jdocs.AbstractJavaTest;
import scala.concurrent.duration.Duration;
//#other-imports
import org.junit.*;
/**
* This class is not meant to be run as a test in the test suite, but it
* is set up such that it can be run interactively from within an IDE.
*/
public class QuickStartDocTest extends AbstractJavaTest {
@Test
public void demonstrateSource() throws InterruptedException, ExecutionException {
//#create-materializer
final ActorSystem system = ActorSystem.create("QuickStart");
final Materializer materializer = ActorMaterializer.create(system);
//#create-materializer
//#create-source
final Source<Integer, NotUsed> source = Source.range(1, 100);
//#create-source
//#run-source
source.runForeach(i -> System.out.println(i), materializer);
//#run-source
//#transform-source
final Source<BigInteger, NotUsed> factorials =
source
.scan(BigInteger.ONE, (acc, next) -> acc.multiply(BigInteger.valueOf(next)));
final CompletionStage<IOResult> result =
factorials
.map(num -> ByteString.fromString(num.toString() + "\n"))
.runWith(FileIO.toPath(Paths.get("factorials.txt")), materializer);
//#transform-source
//#use-transformed-sink
factorials.map(BigInteger::toString).runWith(lineSink("factorial2.txt"), materializer);
//#use-transformed-sink
//#add-streams
factorials
.zipWith(Source.range(0, 99), (num, idx) -> String.format("%d! = %s", idx, num))
.throttle(1, Duration.create(1, TimeUnit.SECONDS), 1, ThrottleMode.shaping())
//#add-streams
.take(2)
//#add-streams
.runForeach(s -> System.out.println(s), materializer);
//#add-streams
//#run-source-and-terminate
final CompletionStage<Done> done =
source.runForeach(i -> System.out.println(i), materializer);
done.thenRun(() -> system.terminate());
//#run-source-and-terminate
done.toCompletableFuture().get();
}
//#transform-sink
public Sink<String, CompletionStage<IOResult>> lineSink(String filename) {
return Flow.of(String.class)
.map(s -> ByteString.fromString(s.toString() + "\n"))
.toMat(FileIO.toPath(Paths.get(filename)), Keep.right());
}
//#transform-sink
}

View file

@ -0,0 +1,159 @@
/*
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.DoubleStream;
import java.util.stream.Stream;
import akka.NotUsed;
import jdocs.AbstractJavaTest;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import static org.junit.Assert.*;
import akka.actor.ActorSystem;
import akka.japi.Pair;
import akka.japi.tuple.Tuple3;
import akka.stream.*;
import akka.stream.javadsl.*;
import akka.stream.testkit.TestPublisher;
import akka.stream.testkit.TestSubscriber;
import akka.stream.testkit.javadsl.TestSink;
import akka.stream.testkit.javadsl.TestSource;
import akka.testkit.TestLatch;
import scala.concurrent.Await;
import scala.concurrent.duration.Duration;
import scala.util.Random;
public class RateTransformationDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RateTransformationDocTest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
final Random r = new Random();
@Test
public void conflateShouldSummarize() throws Exception {
//#conflate-summarize
final Flow<Double, Tuple3<Double, Double, Integer>, NotUsed> statsFlow =
Flow.of(Double.class)
.conflateWithSeed(elem -> Collections.singletonList(elem), (acc, elem) -> {
return Stream
.concat(acc.stream(), Collections.singletonList(elem).stream())
.collect(Collectors.toList());
})
.map(s -> {
final Double mean = s.stream().mapToDouble(d -> d).sum() / s.size();
final DoubleStream se = s.stream().mapToDouble(x -> Math.pow(x - mean, 2));
final Double stdDev = Math.sqrt(se.sum() / s.size());
return new Tuple3<>(stdDev, mean, s.size());
});
//#conflate-summarize
final CompletionStage<List<Tuple3<Double, Double, Integer>>> fut = Source.repeat(0).map(i -> r.nextGaussian())
.via(statsFlow)
.grouped(10)
.runWith(Sink.head(), mat);
fut.toCompletableFuture().get(1, TimeUnit.SECONDS);
}
@Test
public void conflateShouldSample() throws Exception {
//#conflate-sample
final Double p = 0.01;
final Flow<Double, Double, NotUsed> sampleFlow = Flow.of(Double.class)
.conflateWithSeed(elem -> Collections.singletonList(elem), (acc, elem) -> {
if (r.nextDouble() < p) {
return Stream
.concat(acc.stream(), Collections.singletonList(elem).stream())
.collect(Collectors.toList());
}
return acc;
})
.mapConcat(d -> d);
//#conflate-sample
final CompletionStage<Double> fut = Source.from(new ArrayList<Double>(Collections.nCopies(1000, 1.0)))
.via(sampleFlow)
.runWith(Sink.fold(0.0, (agg, next) -> agg + next), mat);
final Double count = fut.toCompletableFuture().get(1, TimeUnit.SECONDS);
}
@Test
public void expandShouldRepeatLast() throws Exception {
//#expand-last
final Flow<Double, Double, NotUsed> lastFlow = Flow.of(Double.class)
.expand(in -> Stream.iterate(in, i -> i).iterator());
//#expand-last
final Pair<TestPublisher.Probe<Double>, CompletionStage<List<Double>>> probeFut = TestSource.<Double> probe(system)
.via(lastFlow)
.grouped(10)
.toMat(Sink.head(), Keep.both())
.run(mat);
final TestPublisher.Probe<Double> probe = probeFut.first();
final CompletionStage<List<Double>> fut = probeFut.second();
probe.sendNext(1.0);
final List<Double> expanded = fut.toCompletableFuture().get(1, TimeUnit.SECONDS);
assertEquals(expanded.size(), 10);
assertEquals(expanded.stream().mapToDouble(d -> d).sum(), 10, 0.1);
}
@Test
public void expandShouldTrackDrift() throws Exception {
@SuppressWarnings("unused")
//#expand-drift
final Flow<Double, Pair<Double, Integer>, NotUsed> driftFlow = Flow.of(Double.class)
.expand(d -> Stream.iterate(0, i -> i + 1).map(i -> new Pair<>(d, i)).iterator());
//#expand-drift
final TestLatch latch = new TestLatch(2, system);
final Flow<Double, Pair<Double, Integer>, NotUsed> realDriftFlow = Flow.of(Double.class)
.expand(d -> { latch.countDown(); return Stream.iterate(0, i -> i + 1).map(i -> new Pair<>(d, i)).iterator(); });
final Pair<TestPublisher.Probe<Double>, TestSubscriber.Probe<Pair<Double, Integer>>> pubSub = TestSource.<Double> probe(system)
.via(realDriftFlow)
.toMat(TestSink.<Pair<Double, Integer>> probe(system), Keep.both())
.run(mat);
final TestPublisher.Probe<Double> pub = pubSub.first();
final TestSubscriber.Probe<Pair<Double, Integer>> sub = pubSub.second();
sub.request(1);
pub.sendNext(1.0);
sub.expectNext(new Pair<>(1.0, 0));
sub.requestNext(new Pair<>(1.0, 1));
sub.requestNext(new Pair<>(1.0, 2));
pub.sendNext(2.0);
Await.ready(latch, Duration.create(1, TimeUnit.SECONDS));
sub.requestNext(new Pair<>(2.0, 0));
}
}

View file

@ -0,0 +1,269 @@
/*
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream;
import akka.NotUsed;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.japi.function.Creator;
import akka.stream.*;
import akka.stream.javadsl.*;
import akka.testkit.TestProbe;
import jdocs.AbstractJavaTest;
import jdocs.stream.TwitterStreamQuickstartDocTest.Model.Author;
import jdocs.stream.TwitterStreamQuickstartDocTest.Model.Tweet;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
//#imports
import org.reactivestreams.Publisher;
import org.reactivestreams.Subscriber;
import org.reactivestreams.Processor;
//#imports
import org.reactivestreams.Subscription;
import java.lang.Exception;
import static jdocs.stream.ReactiveStreamsDocTest.Fixture.Data.authors;
import static jdocs.stream.TwitterStreamQuickstartDocTest.Model.AKKA;
public class ReactiveStreamsDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer mat;
static TestProbe storageProbe;
static TestProbe alertProbe;
@BeforeClass
public static void setup() {
system = ActorSystem.create("ReactiveStreamsDocTest");
mat = ActorMaterializer.create(system);
storageProbe = new TestProbe(system);
alertProbe = new TestProbe(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
storageProbe = null;
alertProbe = null;
}
static class Fixture {
// below class additionally helps with aligning code includes nicely
static class Data {
static //#authors
final Flow<Tweet, Author, NotUsed> authors = Flow.of(Tweet.class)
.filter(t -> t.hashtags().contains(AKKA))
.map(t -> t.author);
//#authors
}
static interface RS {
//#tweets-publisher
Publisher<Tweet> tweets();
//#tweets-publisher
//#author-storage-subscriber
Subscriber<Author> storage();
//#author-storage-subscriber
//#author-alert-subscriber
Subscriber<Author> alert();
//#author-alert-subscriber
}
}
final Fixture.RS rs = new Fixture.RS() {
@Override
public Publisher<Tweet> tweets() {
return TwitterStreamQuickstartDocTest.Model.tweets.runWith(Sink.asPublisher(AsPublisher.WITHOUT_FANOUT), mat);
}
/**
* This is a minimal version of SubscriberProbe,
* which lives in akka-stream-testkit (test scope) and for
* now wanted to avoid setting up (test -> compile) dependency for maven).
*
* TODO: Once SubscriberProbe is easily used here replace this MPS with it.
*/
class MinimalProbeSubscriber<T> implements Subscriber<T> {
private final ActorRef ref;
public MinimalProbeSubscriber(ActorRef ref) {
this.ref = ref;
}
@Override
public void onSubscribe(Subscription s) {
s.request(Long.MAX_VALUE);
}
@Override
public void onNext(T t) {
ref.tell(t, ActorRef.noSender());
}
@Override
public void onError(Throwable t) {
ref.tell(t, ActorRef.noSender());
}
@Override
public void onComplete() {
ref.tell("complete", ActorRef.noSender());
}
}
@Override
public Subscriber<Author> storage() {
return new MinimalProbeSubscriber<>(storageProbe.ref());
}
@Override
public Subscriber<Author> alert() {
return new MinimalProbeSubscriber<>(alertProbe.ref());
}
};
@Test
public void reactiveStreamsPublisherViaFlowToSubscriber() throws Exception {
new TestKit(system) {
final TestProbe probe = new TestProbe(system);
{
//#connect-all
Source.fromPublisher(rs.tweets())
.via(authors)
.to(Sink.fromSubscriber(rs.storage()));
//#connect-all
}
};
}
@Test
public void flowAsPublisherAndSubscriber() throws Exception {
new TestKit(system) {
final TestProbe probe = new TestProbe(system);
{
//#flow-publisher-subscriber
final Processor<Tweet, Author> processor =
authors.toProcessor().run(mat);
rs.tweets().subscribe(processor);
processor.subscribe(rs.storage());
//#flow-publisher-subscriber
assertStorageResult();
}
};
}
@Test
public void sourceAsPublisher() throws Exception {
new TestKit(system) {
final TestProbe probe = new TestProbe(system);
{
//#source-publisher
final Publisher<Author> authorPublisher =
Source.fromPublisher(rs.tweets())
.via(authors)
.runWith(Sink.asPublisher(AsPublisher.WITHOUT_FANOUT), mat);
authorPublisher.subscribe(rs.storage());
//#source-publisher
assertStorageResult();
}
};
}
@Test
public void sourceAsFanoutPublisher() throws Exception {
new TestKit(system) {
final TestProbe probe = new TestProbe(system);
{
//#source-fanoutPublisher
final Publisher<Author> authorPublisher =
Source.fromPublisher(rs.tweets())
.via(authors)
.runWith(Sink.asPublisher(AsPublisher.WITH_FANOUT), mat);
authorPublisher.subscribe(rs.storage());
authorPublisher.subscribe(rs.alert());
//#source-fanoutPublisher
assertStorageResult();
}
};
}
@Test
public void sinkAsSubscriber() throws Exception {
new TestKit(system) {
final TestProbe probe = new TestProbe(system);
{
//#sink-subscriber
final Subscriber<Author> storage = rs.storage();
final Subscriber<Tweet> tweetSubscriber =
authors
.to(Sink.fromSubscriber(storage))
.runWith(Source.asSubscriber(), mat);
rs.tweets().subscribe(tweetSubscriber);
//#sink-subscriber
assertStorageResult();
}
};
}
@Test
public void useProcessor() throws Exception {
new TestKit(system) {
{
//#use-processor
// An example Processor factory
final Creator<Processor<Integer, Integer>> factory =
new Creator<Processor<Integer, Integer>>() {
public Processor<Integer, Integer> create() {
return Flow.of(Integer.class).toProcessor().run(mat);
}
};
final Flow<Integer, Integer, NotUsed> flow = Flow.fromProcessor(factory);
//#use-processor
}
};
}
void assertStorageResult() {
storageProbe.expectMsg(new Author("rolandkuhn"));
storageProbe.expectMsg(new Author("patriknw"));
storageProbe.expectMsg(new Author("bantonsson"));
storageProbe.expectMsg(new Author("drewhk"));
storageProbe.expectMsg(new Author("ktosopl"));
storageProbe.expectMsg(new Author("mmartynas"));
storageProbe.expectMsg(new Author("akkateam"));
storageProbe.expectMsg("complete");
}
}

View file

@ -0,0 +1,64 @@
package jdocs.stream;
import akka.actor.ActorRef;
import java.util.function.Predicate;
/**
* Acts as if `System.out.println()` yet swallows all messages. Useful for putting printlines in examples yet without poluting the build with them.
*/
public class SilenceSystemOut {
private SilenceSystemOut() {
}
public static System get() {
return new System(new System.Println() {
@Override
public void println(String s) {
// ignore
}
});
}
public static System get(ActorRef probe) {
return new System(new System.Println() {
@Override
public void println(String s) {
probe.tell(s, ActorRef.noSender());
}
});
}
public static System get(Predicate<String> filter, ActorRef probe) {
return new System(new System.Println() {
@Override
public void println(String s) {
if (filter.test(s))
probe.tell(s, ActorRef.noSender());
}
});
}
public static class System {
public final Println out;
public System(Println out) {
this.out = out;
}
public static abstract class Println {
public abstract void println(String s);
public void println(Object s) {
println(s.toString());
}
public void printf(String format, Object... args) {
println(String.format(format, args));
}
}
}
}

View file

@ -0,0 +1,131 @@
/**
* Copyright (C) 2014-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream;
import java.util.Arrays;
import java.util.concurrent.TimeUnit;
import akka.NotUsed;
import jdocs.AbstractJavaTest;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.duration.FiniteDuration;
import akka.actor.ActorSystem;
import akka.actor.Cancellable;
import akka.stream.*;
import akka.stream.javadsl.*;
public class StreamBuffersRateDocTest extends AbstractJavaTest {
static class Job {}
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("StreamBuffersDocTest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
final SilenceSystemOut.System System = SilenceSystemOut.get();
@Test
public void demonstratePipelining() {
//#pipelining
Source.from(Arrays.asList(1, 2, 3))
.map(i -> {System.out.println("A: " + i); return i;}).async()
.map(i -> {System.out.println("B: " + i); return i;}).async()
.map(i -> {System.out.println("C: " + i); return i;}).async()
.runWith(Sink.ignore(), mat);
//#pipelining
}
@Test
@SuppressWarnings("unused")
public void demonstrateBufferSizes() {
//#materializer-buffer
final Materializer materializer = ActorMaterializer.create(
ActorMaterializerSettings.create(system)
.withInputBuffer(64, 64), system);
//#materializer-buffer
//#section-buffer
final Flow<Integer, Integer, NotUsed> flow1 =
Flow.of(Integer.class)
.map(elem -> elem * 2).async()
.addAttributes(Attributes.inputBuffer(1, 1)); // the buffer size of this map is 1
final Flow<Integer, Integer, NotUsed> flow2 =
flow1.via(
Flow.of(Integer.class)
.map(elem -> elem / 2)).async(); // the buffer size of this map is the default
//#section-buffer
}
@Test
public void demonstrateBufferAbstractionLeak() {
//#buffering-abstraction-leak
final FiniteDuration oneSecond =
FiniteDuration.create(1, TimeUnit.SECONDS);
final Source<String, Cancellable> msgSource =
Source.tick(oneSecond, oneSecond, "message!");
final Source<String, Cancellable> tickSource =
Source.tick(oneSecond.mul(3), oneSecond.mul(3), "tick");
final Flow<String, Integer, NotUsed> conflate =
Flow.of(String.class).conflateWithSeed(
first -> 1, (count, elem) -> count + 1);
RunnableGraph.fromGraph(GraphDSL.create(b -> {
// this is the asynchronous stage in this graph
final FanInShape2<String, Integer, Integer> zipper =
b.add(ZipWith.create((String tick, Integer count) -> count).async());
b.from(b.add(msgSource)).via(b.add(conflate)).toInlet(zipper.in1());
b.from(b.add(tickSource)).toInlet(zipper.in0());
b.from(zipper.out()).to(b.add(Sink.foreach(elem -> System.out.println(elem))));
return ClosedShape.getInstance();
})).run(mat);
//#buffering-abstraction-leak
}
@Test
public void demonstrateExplicitBuffers() {
final Source<Job, NotUsed> inboundJobsConnector = Source.empty();
//#explicit-buffers-backpressure
// Getting a stream of jobs from an imaginary external system as a Source
final Source<Job, NotUsed> jobs = inboundJobsConnector;
jobs.buffer(1000, OverflowStrategy.backpressure());
//#explicit-buffers-backpressure
//#explicit-buffers-droptail
jobs.buffer(1000, OverflowStrategy.dropTail());
//#explicit-buffers-droptail
//#explicit-buffers-dropnew
jobs.buffer(1000, OverflowStrategy.dropNew());
//#explicit-buffers-dropnew
//#explicit-buffers-drophead
jobs.buffer(1000, OverflowStrategy.dropHead());
//#explicit-buffers-drophead
//#explicit-buffers-dropbuffer
jobs.buffer(1000, OverflowStrategy.dropBuffer());
//#explicit-buffers-dropbuffer
//#explicit-buffers-fail
jobs.buffer(1000, OverflowStrategy.fail());
//#explicit-buffers-fail
}
}

View file

@ -0,0 +1,178 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream;
import akka.Done;
import akka.NotUsed;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.japi.Pair;
import akka.stream.*;
import akka.stream.javadsl.*;
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.Arrays;
import java.util.Iterator;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.assertEquals;
public class StreamPartialGraphDSLDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("StreamPartialGraphDSLDocTest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
@Test
public void demonstrateBuildWithOpenPorts() throws Exception {
//#simple-partial-graph-dsl
final Graph<FanInShape2<Integer, Integer, Integer>, NotUsed> zip =
ZipWith.create((Integer left, Integer right) -> Math.max(left, right));
final Graph<UniformFanInShape<Integer, Integer>, NotUsed> pickMaxOfThree =
GraphDSL.create(builder -> {
final FanInShape2<Integer, Integer, Integer> zip1 = builder.add(zip);
final FanInShape2<Integer, Integer, Integer> zip2 = builder.add(zip);
builder.from(zip1.out()).toInlet(zip2.in0());
// return the shape, which has three inputs and one output
return new UniformFanInShape<Integer, Integer>(zip2.out(),
new Inlet[] {zip1.in0(), zip1.in1(), zip2.in1()});
});
final Sink<Integer, CompletionStage<Integer>> resultSink = Sink.<Integer>head();
final RunnableGraph<CompletionStage<Integer>> g =
RunnableGraph.<CompletionStage<Integer>>fromGraph(
GraphDSL.create(resultSink, (builder, sink) -> {
// import the partial graph explicitly
final UniformFanInShape<Integer, Integer> pm = builder.add(pickMaxOfThree);
builder.from(builder.add(Source.single(1))).toInlet(pm.in(0));
builder.from(builder.add(Source.single(2))).toInlet(pm.in(1));
builder.from(builder.add(Source.single(3))).toInlet(pm.in(2));
builder.from(pm.out()).to(sink);
return ClosedShape.getInstance();
}));
final CompletionStage<Integer> max = g.run(mat);
//#simple-partial-graph-dsl
assertEquals(Integer.valueOf(3), max.toCompletableFuture().get(3, TimeUnit.SECONDS));
}
//#source-from-partial-graph-dsl
// first create an indefinite source of integer numbers
class Ints implements Iterator<Integer> {
private int next = 0;
@Override
public boolean hasNext() {
return true;
}
@Override
public Integer next() {
return next++;
}
}
//#source-from-partial-graph-dsl
@Test
public void demonstrateBuildSourceFromPartialGraphDSLCreate() throws Exception {
//#source-from-partial-graph-dsl
final Source<Integer, NotUsed> ints = Source.fromIterator(() -> new Ints());
final Source<Pair<Integer, Integer>, NotUsed> pairs = Source.fromGraph(
GraphDSL.create(
builder -> {
final FanInShape2<Integer, Integer, Pair<Integer, Integer>> zip =
builder.add(Zip.create());
builder.from(builder.add(ints.filter(i -> i % 2 == 0))).toInlet(zip.in0());
builder.from(builder.add(ints.filter(i -> i % 2 == 1))).toInlet(zip.in1());
return SourceShape.of(zip.out());
}));
final CompletionStage<Pair<Integer, Integer>> firstPair =
pairs.runWith(Sink.<Pair<Integer, Integer>>head(), mat);
//#source-from-partial-graph-dsl
assertEquals(new Pair<>(0, 1), firstPair.toCompletableFuture().get(3, TimeUnit.SECONDS));
}
@Test
public void demonstrateBuildFlowFromPartialGraphDSLCreate() throws Exception {
//#flow-from-partial-graph-dsl
final Flow<Integer, Pair<Integer, String>, NotUsed> pairs = Flow.fromGraph(GraphDSL.create(
b -> {
final UniformFanOutShape<Integer, Integer> bcast = b.add(Broadcast.create(2));
final FanInShape2<Integer, String, Pair<Integer, String>> zip =
b.add(Zip.create());
b.from(bcast).toInlet(zip.in0());
b.from(bcast).via(b.add(Flow.of(Integer.class).map(i -> i.toString()))).toInlet(zip.in1());
return FlowShape.of(bcast.in(), zip.out());
}));
//#flow-from-partial-graph-dsl
final CompletionStage<Pair<Integer, String>> matSink =
//#flow-from-partial-graph-dsl
Source.single(1).via(pairs).runWith(Sink.<Pair<Integer, String>>head(), mat);
//#flow-from-partial-graph-dsl
assertEquals(new Pair<>(1, "1"), matSink.toCompletableFuture().get(3, TimeUnit.SECONDS));
}
@Test
public void demonstrateBuildSourceWithCombine() throws Exception {
//#source-combine
Source<Integer, NotUsed> source1 = Source.single(1);
Source<Integer, NotUsed> source2 = Source.single(2);
final Source<Integer, NotUsed> sources = Source.combine(source1, source2, new ArrayList<>(),
i -> Merge.<Integer>create(i));
//#source-combine
final CompletionStage<Integer> result=
//#source-combine
sources.runWith(Sink.<Integer, Integer>fold(0, (a,b) -> a + b), mat);
//#source-combine
assertEquals(Integer.valueOf(3), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
}
@Test
public void demonstrateBuildSinkWithCombine() throws Exception {
final TestKit probe = new TestKit(system);
ActorRef actorRef = probe.getRef();
//#sink-combine
Sink<Integer, NotUsed> sendRemotely = Sink.actorRef(actorRef, "Done");
Sink<Integer, CompletionStage<Done>> localProcessing = Sink.<Integer>foreach(a -> { /*do something useful*/ } );
Sink<Integer, NotUsed> sinks = Sink.combine(sendRemotely,localProcessing, new ArrayList<>(), a -> Broadcast.create(a));
Source.<Integer>from(Arrays.asList(new Integer[]{0, 1, 2})).runWith(sinks, mat);
//#sink-combine
probe.expectMsgEquals(0);
probe.expectMsgEquals(1);
probe.expectMsgEquals(2);
}
}

View file

@ -0,0 +1,234 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import akka.NotUsed;
import jdocs.AbstractJavaTest;
import akka.testkit.javadsl.TestKit;
import org.junit.*;
import static org.junit.Assert.assertEquals;
import akka.actor.*;
import akka.testkit.*;
import akka.japi.Pair;
import akka.stream.*;
import akka.stream.javadsl.*;
import akka.stream.testkit.*;
import akka.stream.testkit.javadsl.*;
import akka.testkit.TestProbe;
import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
public class StreamTestKitDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("StreamTestKitDocTest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
@Test
public void strictCollection() throws Exception {
//#strict-collection
final Sink<Integer, CompletionStage<Integer>> sinkUnderTest = Flow.of(Integer.class)
.map(i -> i * 2)
.toMat(Sink.fold(0, (agg, next) -> agg + next), Keep.right());
final CompletionStage<Integer> future = Source.from(Arrays.asList(1, 2, 3, 4))
.runWith(sinkUnderTest, mat);
final Integer result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
assert(result == 20);
//#strict-collection
}
@Test
public void groupedPartOfInfiniteStream() throws Exception {
//#grouped-infinite
final Source<Integer, NotUsed> sourceUnderTest = Source.repeat(1)
.map(i -> i * 2);
final CompletionStage<List<Integer>> future = sourceUnderTest
.take(10)
.runWith(Sink.seq(), mat);
final List<Integer> result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
assertEquals(result, Collections.nCopies(10, 2));
//#grouped-infinite
}
@Test
public void foldedStream() throws Exception {
//#folded-stream
final Flow<Integer, Integer, NotUsed> flowUnderTest = Flow.of(Integer.class)
.takeWhile(i -> i < 5);
final CompletionStage<Integer> future = Source.from(Arrays.asList(1, 2, 3, 4, 5, 6))
.via(flowUnderTest).runWith(Sink.fold(0, (agg, next) -> agg + next), mat);
final Integer result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
assert(result == 10);
//#folded-stream
}
@Test
public void pipeToTestProbe() throws Exception {
//#pipeto-testprobe
final Source<List<Integer>, NotUsed> sourceUnderTest = Source
.from(Arrays.asList(1, 2, 3, 4))
.grouped(2);
final TestProbe probe = new TestProbe(system);
final CompletionStage<List<List<Integer>>> future = sourceUnderTest
.grouped(2)
.runWith(Sink.head(), mat);
akka.pattern.PatternsCS.pipe(future, system.dispatcher()).to(probe.ref());
probe.expectMsg(Duration.create(3, TimeUnit.SECONDS),
Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4))
);
//#pipeto-testprobe
}
public enum Tick { TOCK, COMPLETED };
@Test
public void sinkActorRef() throws Exception {
//#sink-actorref
final Source<Tick, Cancellable> sourceUnderTest = Source.tick(
FiniteDuration.create(0, TimeUnit.MILLISECONDS),
FiniteDuration.create(200, TimeUnit.MILLISECONDS),
Tick.TOCK);
final TestProbe probe = new TestProbe(system);
final Cancellable cancellable = sourceUnderTest
.to(Sink.actorRef(probe.ref(), Tick.COMPLETED)).run(mat);
probe.expectMsg(Duration.create(3, TimeUnit.SECONDS), Tick.TOCK);
probe.expectNoMsg(Duration.create(100, TimeUnit.MILLISECONDS));
probe.expectMsg(Duration.create(3, TimeUnit.SECONDS), Tick.TOCK);
cancellable.cancel();
probe.expectMsg(Duration.create(3, TimeUnit.SECONDS), Tick.COMPLETED);
//#sink-actorref
}
@Test
public void sourceActorRef() throws Exception {
//#source-actorref
final Sink<Integer, CompletionStage<String>> sinkUnderTest = Flow.of(Integer.class)
.map(i -> i.toString())
.toMat(Sink.fold("", (agg, next) -> agg + next), Keep.right());
final Pair<ActorRef, CompletionStage<String>> refAndCompletionStage =
Source.<Integer>actorRef(8, OverflowStrategy.fail())
.toMat(sinkUnderTest, Keep.both())
.run(mat);
final ActorRef ref = refAndCompletionStage.first();
final CompletionStage<String> future = refAndCompletionStage.second();
ref.tell(1, ActorRef.noSender());
ref.tell(2, ActorRef.noSender());
ref.tell(3, ActorRef.noSender());
ref.tell(new akka.actor.Status.Success("done"), ActorRef.noSender());
final String result = future.toCompletableFuture().get(1, TimeUnit.SECONDS);
assertEquals(result, "123");
//#source-actorref
}
@Test
public void testSinkProbe() {
//#test-sink-probe
final Source<Integer, NotUsed> sourceUnderTest = Source.from(Arrays.asList(1, 2, 3, 4))
.filter(elem -> elem % 2 == 0)
.map(elem -> elem * 2);
sourceUnderTest
.runWith(TestSink.probe(system), mat)
.request(2)
.expectNext(4, 8)
.expectComplete();
//#test-sink-probe
}
@Test
public void testSourceProbe() {
//#test-source-probe
final Sink<Integer, NotUsed> sinkUnderTest = Sink.cancelled();
TestSource.<Integer>probe(system)
.toMat(sinkUnderTest, Keep.left())
.run(mat)
.expectCancellation();
//#test-source-probe
}
@Test
public void injectingFailure() throws Exception {
//#injecting-failure
final Sink<Integer, CompletionStage<Integer>> sinkUnderTest = Sink.head();
final Pair<TestPublisher.Probe<Integer>, CompletionStage<Integer>> probeAndCompletionStage =
TestSource.<Integer>probe(system)
.toMat(sinkUnderTest, Keep.both())
.run(mat);
final TestPublisher.Probe<Integer> probe = probeAndCompletionStage.first();
final CompletionStage<Integer> future = probeAndCompletionStage.second();
probe.sendError(new Exception("boom"));
try {
future.toCompletableFuture().get(3, TimeUnit.SECONDS);
assert false;
} catch (ExecutionException ee) {
final Throwable exception = ee.getCause();
assertEquals(exception.getMessage(), "boom");
}
//#injecting-failure
}
@Test
public void testSourceAndTestSink() throws Exception {
//#test-source-and-sink
final Flow<Integer, Integer, NotUsed> flowUnderTest = Flow.of(Integer.class)
.mapAsyncUnordered(2, sleep -> akka.pattern.PatternsCS.after(
Duration.create(10, TimeUnit.MILLISECONDS),
system.scheduler(),
system.dispatcher(),
CompletableFuture.completedFuture(sleep)
));
final Pair<TestPublisher.Probe<Integer>, TestSubscriber.Probe<Integer>> pubAndSub =
TestSource.<Integer>probe(system)
.via(flowUnderTest)
.toMat(TestSink.<Integer>probe(system), Keep.both())
.run(mat);
final TestPublisher.Probe<Integer> pub = pubAndSub.first();
final TestSubscriber.Probe<Integer> sub = pubAndSub.second();
sub.request(3);
pub.sendNext(3);
pub.sendNext(2);
pub.sendNext(1);
sub.expectNextUnordered(1, 2, 3);
pub.sendError(new Exception("Power surge in the linear subroutine C-47!"));
final Throwable ex = sub.expectError();
assert(ex.getMessage().contains("C-47"));
//#test-source-and-sink
}
}

View file

@ -0,0 +1,362 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream;
import akka.Done;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.japi.JavaPartialFunction;
//#imports
import akka.stream.*;
import akka.stream.javadsl.*;
//#imports
import jdocs.AbstractJavaTest;
import jdocs.stream.TwitterStreamQuickstartDocTest.Model.Author;
import jdocs.stream.TwitterStreamQuickstartDocTest.Model.Hashtag;
import jdocs.stream.TwitterStreamQuickstartDocTest.Model.Tweet;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Set;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import static jdocs.stream.TwitterStreamQuickstartDocTest.Model.AKKA;
import static jdocs.stream.TwitterStreamQuickstartDocTest.Model.tweets;
@SuppressWarnings("unused")
public class TwitterStreamQuickstartDocTest extends AbstractJavaTest {
private static final long serialVersionUID = 1L;
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("TwitterStreamQuickstartDocTest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
static abstract class Model {
//#model
public static class Author {
public final String handle;
public Author(String handle) {
this.handle = handle;
}
// ...
//#model
@Override
public String toString() {
return "Author(" + handle + ")";
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Author author = (Author) o;
if (handle != null ? !handle.equals(author.handle) : author.handle != null) {
return false;
}
return true;
}
@Override
public int hashCode() {
return handle != null ? handle.hashCode() : 0;
}
//#model
}
//#model
//#model
public static class Hashtag {
public final String name;
public Hashtag(String name) {
this.name = name;
}
// ...
//#model
@Override
public int hashCode() {
return name.hashCode();
}
@Override
public boolean equals(Object obj) {
if (this == obj)
return true;
if (obj == null)
return false;
if (getClass() != obj.getClass())
return false;
Hashtag other = (Hashtag) obj;
return name.equals(other.name);
}
@Override
public String toString() {
return "Hashtag(" + name + ")";
}
//#model
}
//#model
//#model
public static class Tweet {
public final Author author;
public final long timestamp;
public final String body;
public Tweet(Author author, long timestamp, String body) {
this.author = author;
this.timestamp = timestamp;
this.body = body;
}
public Set<Hashtag> hashtags() {
return Arrays.asList(body.split(" ")).stream()
.filter(a -> a.startsWith("#"))
.map(a -> new Hashtag(a))
.collect(Collectors.toSet());
}
// ...
//#model
@Override
public String toString() {
return "Tweet(" + author + "," + timestamp + "," + body + ")";
}
//#model
}
//#model
//#model
public static final Hashtag AKKA = new Hashtag("#akka");
//#model
public static final Source<Tweet, NotUsed> tweets = Source.from(
Arrays.asList(new Tweet[] {
new Tweet(new Author("rolandkuhn"), System.currentTimeMillis(), "#akka rocks!"),
new Tweet(new Author("patriknw"), System.currentTimeMillis(), "#akka !"),
new Tweet(new Author("bantonsson"), System.currentTimeMillis(), "#akka !"),
new Tweet(new Author("drewhk"), System.currentTimeMillis(), "#akka !"),
new Tweet(new Author("ktosopl"), System.currentTimeMillis(), "#akka on the rocks!"),
new Tweet(new Author("mmartynas"), System.currentTimeMillis(), "wow #akka !"),
new Tweet(new Author("akkateam"), System.currentTimeMillis(), "#akka rocks!"),
new Tweet(new Author("bananaman"), System.currentTimeMillis(), "#bananas rock!"),
new Tweet(new Author("appleman"), System.currentTimeMillis(), "#apples rock!"),
new Tweet(new Author("drama"), System.currentTimeMillis(), "we compared #apples to #oranges!")
}));
}
static abstract class Example0 {
//#tweet-source
Source<Tweet, NotUsed> tweets;
//#tweet-source
}
static abstract class Example1 {
//#first-sample
//#materializer-setup
final ActorSystem system = ActorSystem.create("reactive-tweets");
final Materializer mat = ActorMaterializer.create(system);
//#first-sample
//#materializer-setup
}
static class Example2 {
public void run(final Materializer mat) throws TimeoutException, InterruptedException, ExecutionException {
//#backpressure-by-readline
final CompletionStage<Done> completion =
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))
.map(i -> { System.out.println("map => " + i); return i; })
.runForeach(i -> System.console().readLine("Element = %s continue reading? [press enter]\n", i), mat);
completion.toCompletableFuture().get(1, TimeUnit.SECONDS);
//#backpressure-by-readline
}
}
@Test
public void demonstrateFilterAndMap() {
final SilenceSystemOut.System System = SilenceSystemOut.get();
//#first-sample
//#authors-filter-map
final Source<Author, NotUsed> authors =
tweets
.filter(t -> t.hashtags().contains(AKKA))
.map(t -> t.author);
//#first-sample
//#authors-filter-map
new Object() {
//#authors-collect
JavaPartialFunction<Tweet, Author> collectFunction =
new JavaPartialFunction<Tweet, Author>() {
public Author apply(Tweet t, boolean isCheck) {
if (t.hashtags().contains(AKKA)) {
if (isCheck) return null; // to spare the expensive or side-effecting code
return t.author;
} else {
throw noMatch();
}
}
};
final Source<Author, NotUsed> authors =
tweets.collect(collectFunction);
//#authors-collect
};
//#first-sample
//#authors-foreachsink-println
authors.runWith(Sink.foreach(a -> System.out.println(a)), mat);
//#first-sample
//#authors-foreachsink-println
//#authors-foreach-println
authors.runForeach(a -> System.out.println(a), mat);
//#authors-foreach-println
}
@Test
public void demonstrateMapConcat() {
//#hashtags-mapConcat
final Source<Hashtag, NotUsed> hashtags =
tweets.mapConcat(t -> new ArrayList<Hashtag>(t.hashtags()));
//#hashtags-mapConcat
}
static abstract class HiddenDefinitions {
//#graph-dsl-broadcast
Sink<Author, NotUsed> writeAuthors;
Sink<Hashtag, NotUsed> writeHashtags;
//#graph-dsl-broadcast
}
@Test
public void demonstrateBroadcast() {
final Sink<Author, CompletionStage<Done>> writeAuthors = Sink.ignore();
final Sink<Hashtag, CompletionStage<Done>> writeHashtags = Sink.ignore();
//#graph-dsl-broadcast
RunnableGraph.fromGraph(GraphDSL.create(b -> {
final UniformFanOutShape<Tweet, Tweet> bcast = b.add(Broadcast.create(2));
final FlowShape<Tweet, Author> toAuthor =
b.add(Flow.of(Tweet.class).map(t -> t.author));
final FlowShape<Tweet, Hashtag> toTags =
b.add(Flow.of(Tweet.class).mapConcat(t -> new ArrayList<Hashtag>(t.hashtags())));
final SinkShape<Author> authors = b.add(writeAuthors);
final SinkShape<Hashtag> hashtags = b.add(writeHashtags);
b.from(b.add(tweets)).viaFanOut(bcast).via(toAuthor).to(authors);
b.from(bcast).via(toTags).to(hashtags);
return ClosedShape.getInstance();
})).run(mat);
//#graph-dsl-broadcast
}
long slowComputation(Tweet t) {
try {
// act as if performing some heavy computation
Thread.sleep(500);
} catch (InterruptedException e) {}
return 42;
}
@Test
public void demonstrateSlowProcessing() {
//#tweets-slow-consumption-dropHead
tweets
.buffer(10, OverflowStrategy.dropHead())
.map(t -> slowComputation(t))
.runWith(Sink.ignore(), mat);
//#tweets-slow-consumption-dropHead
}
@Test
public void demonstrateCountOnFiniteStream() {
//#tweets-fold-count
final Sink<Integer, CompletionStage<Integer>> sumSink =
Sink.<Integer, Integer>fold(0, (acc, elem) -> acc + elem);
final RunnableGraph<CompletionStage<Integer>> counter =
tweets.map(t -> 1).toMat(sumSink, Keep.right());
final CompletionStage<Integer> sum = counter.run(mat);
sum.thenAcceptAsync(c -> System.out.println("Total tweets processed: " + c),
system.dispatcher());
//#tweets-fold-count
new Object() {
//#tweets-fold-count-oneline
final CompletionStage<Integer> sum = tweets.map(t -> 1).runWith(sumSink, mat);
//#tweets-fold-count-oneline
};
}
@Test
public void demonstrateMaterializeMultipleTimes() {
final Source<Tweet, NotUsed> tweetsInMinuteFromNow = tweets; // not really in second, just acting as if
//#tweets-runnable-flow-materialized-twice
final Sink<Integer, CompletionStage<Integer>> sumSink =
Sink.<Integer, Integer>fold(0, (acc, elem) -> acc + elem);
final RunnableGraph<CompletionStage<Integer>> counterRunnableGraph =
tweetsInMinuteFromNow
.filter(t -> t.hashtags().contains(AKKA))
.map(t -> 1)
.toMat(sumSink, Keep.right());
// materialize the stream once in the morning
final CompletionStage<Integer> morningTweetsCount = counterRunnableGraph.run(mat);
// and once in the evening, reusing the blueprint
final CompletionStage<Integer> eveningTweetsCount = counterRunnableGraph.run(mat);
//#tweets-runnable-flow-materialized-twice
}
}

View file

@ -0,0 +1,90 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream.io;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.io.IOException;
import java.util.concurrent.CompletionStage;
import akka.Done;
import akka.actor.ActorSystem;
import akka.stream.ActorAttributes;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.FileIO;
import jdocs.AbstractJavaTest;
import jdocs.stream.SilenceSystemOut;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import akka.stream.*;
import akka.util.ByteString;
public class StreamFileDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("StreamFileDocTest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
final SilenceSystemOut.System System = SilenceSystemOut.get();
{
//#file-source
final Path file = Paths.get("example.csv");
//#file-source
}
@Test
public void demonstrateMaterializingBytesWritten() throws IOException {
final Path file = Files.createTempFile(getClass().getName(), ".tmp");
try {
//#file-source
Sink<ByteString, CompletionStage<Done>> printlnSink =
Sink.<ByteString> foreach(chunk -> System.out.println(chunk.utf8String()));
CompletionStage<IOResult> ioResult =
FileIO.fromPath(file)
.to(printlnSink)
.run(mat);
//#file-source
} finally {
Files.delete(file);
}
}
@Test
public void demonstrateSettingDispatchersInCode() throws IOException {
final Path file = Files.createTempFile(getClass().getName(), ".tmp");
try {
Sink<ByteString, CompletionStage<IOResult>> fileSink =
//#custom-dispatcher-code
FileIO.toPath(file)
.withAttributes(ActorAttributes.dispatcher("custom-blocking-io-dispatcher"));
//#custom-dispatcher-code
} finally {
Files.delete(file);
}
}
}

View file

@ -0,0 +1,168 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package jdocs.stream.io;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.ConcurrentLinkedQueue;
import akka.NotUsed;
import akka.stream.javadsl.Framing;
import jdocs.AbstractJavaTest;
import jdocs.stream.SilenceSystemOut;
import akka.testkit.javadsl.TestKit;
import java.net.InetSocketAddress;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import akka.actor.ActorSystem;
import akka.stream.*;
import akka.stream.javadsl.*;
import akka.stream.javadsl.Tcp.*;
import akka.stream.stage.*;
import akka.testkit.SocketUtil;
import akka.testkit.TestProbe;
import akka.util.ByteString;
public class StreamTcpDocTest extends AbstractJavaTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("StreamTcpDocTest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
final SilenceSystemOut.System System = SilenceSystemOut.get();
private final ConcurrentLinkedQueue<String> input = new ConcurrentLinkedQueue<String>();
{
input.add("Hello world");
input.add("What a lovely day");
}
private String readLine(String prompt) {
String s = input.poll();
return (s == null ? "q": s);
}
@Test
public void demonstrateSimpleServerConnection() {
{
//#echo-server-simple-bind
// IncomingConnection and ServerBinding imported from Tcp
final Source<IncomingConnection, CompletionStage<ServerBinding>> connections =
Tcp.get(system).bind("127.0.0.1", 8889);
//#echo-server-simple-bind
}
{
final InetSocketAddress localhost = SocketUtil.temporaryServerAddress("127.0.0.1", false);
final Source<IncomingConnection, CompletionStage<ServerBinding>> connections =
Tcp.get(system).bind(localhost.getHostString(), localhost.getPort());
//#echo-server-simple-handle
connections.runForeach(connection -> {
System.out.println("New connection from: " + connection.remoteAddress());
final Flow<ByteString, ByteString, NotUsed> echo = Flow.of(ByteString.class)
.via(Framing.delimiter(ByteString.fromString("\n"), 256, FramingTruncation.DISALLOW))
.map(ByteString::utf8String)
.map(s -> s + "!!!\n")
.map(ByteString::fromString);
connection.handleWith(echo, mat);
}, mat);
//#echo-server-simple-handle
}
}
@Test
public void actuallyWorkingClientServerApp() {
final InetSocketAddress localhost = SocketUtil.temporaryServerAddress("127.0.0.1", false);
final TestProbe serverProbe = new TestProbe(system);
final Source<IncomingConnection,CompletionStage<ServerBinding>> connections =
Tcp.get(system).bind(localhost.getHostString(), localhost.getPort());
//#welcome-banner-chat-server
connections.runForeach(connection -> {
// server logic, parses incoming commands
final Flow<String, String, NotUsed> commandParser =
Flow.<String>create()
.takeWhile(elem -> !elem.equals("BYE"))
.map(elem -> elem + "!");
final String welcomeMsg = "Welcome to: " + connection.localAddress() +
" you are: " + connection.remoteAddress() + "!";
final Source<String, NotUsed> welcome = Source.single(welcomeMsg);
final Flow<ByteString, ByteString, NotUsed> serverLogic =
Flow.of(ByteString.class)
.via(Framing.delimiter(ByteString.fromString("\n"), 256, FramingTruncation.DISALLOW))
.map(ByteString::utf8String)
//#welcome-banner-chat-server
.map(command -> {
serverProbe.ref().tell(command, null);
return command;
})
//#welcome-banner-chat-server
.via(commandParser)
.merge(welcome)
.map(s -> s + "\n")
.map(ByteString::fromString);
connection.handleWith(serverLogic, mat);
}, mat);
//#welcome-banner-chat-server
{
//#repl-client
final Flow<ByteString, ByteString, CompletionStage<OutgoingConnection>> connection =
Tcp.get(system).outgoingConnection("127.0.0.1", 8889);
//#repl-client
}
{
final Flow<ByteString, ByteString, CompletionStage<OutgoingConnection>> connection =
Tcp.get(system).outgoingConnection(localhost.getHostString(), localhost.getPort());
//#repl-client
final Flow<String, ByteString, NotUsed> replParser =
Flow.<String>create()
.takeWhile(elem -> !elem.equals("q"))
.concat(Source.single("BYE")) // will run after the original flow completes
.map(elem -> ByteString.fromString(elem + "\n"));
final Flow<ByteString, ByteString, NotUsed> repl = Flow.of(ByteString.class)
.via(Framing.delimiter(ByteString.fromString("\n"), 256, FramingTruncation.DISALLOW))
.map(ByteString::utf8String)
.map(text -> {System.out.println("Server: " + text); return "next";})
.map(elem -> readLine("> "))
.via(replParser);
connection.join(repl).run(mat);
//#repl-client
}
serverProbe.expectMsg("Hello world");
serverProbe.expectMsg("What a lovely day");
serverProbe.expectMsg("BYE");
}
}

View file

@ -0,0 +1,271 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.*;
import akka.stream.javadsl.Flow;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.stream.stage.*;
import akka.testkit.javadsl.TestKit;
import akka.util.ByteString;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.Tuple2;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
public class RecipeByteStrings extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeByteStrings");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
final Source<ByteString, NotUsed> rawBytes = Source.from(Arrays.asList(
ByteString.fromArray(new byte[] { 1, 2 }),
ByteString.fromArray(new byte[] { 3 }),
ByteString.fromArray(new byte[] { 4, 5, 6 }),
ByteString.fromArray(new byte[] { 7, 8, 9 })));
@Test
public void chunker() throws Exception {
new TestKit(system) {
final int CHUNK_LIMIT = 2;
//#bytestring-chunker
class Chunker extends GraphStage<FlowShape<ByteString, ByteString>> {
private final int chunkSize;
public Inlet<ByteString> in = Inlet.<ByteString>create("Chunker.in");
public Outlet<ByteString> out = Outlet.<ByteString>create("Chunker.out");
private FlowShape<ByteString, ByteString> shape = FlowShape.of(in, out);
public Chunker(int chunkSize) {
this.chunkSize = chunkSize;
}
@Override
public FlowShape<ByteString, ByteString> shape() {
return shape;
}
@Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new GraphStageLogic(shape) {
private ByteString buffer = ByteString.empty();
{
setHandler(out, new AbstractOutHandler(){
@Override
public void onPull() throws Exception {
if (isClosed(in)) emitChunk();
else pull(in);
}
});
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() throws Exception {
ByteString elem = grab(in);
buffer = buffer.concat(elem);
emitChunk();
}
@Override
public void onUpstreamFinish() throws Exception {
if (buffer.isEmpty()) completeStage();
else {
// There are elements left in buffer, so
// we keep accepting downstream pulls and push from buffer until emptied.
//
// It might be though, that the upstream finished while it was pulled, in which
// case we will not get an onPull from the downstream, because we already had one.
// In that case we need to emit from the buffer.
if (isAvailable(out)) emitChunk();
}
}
});
}
private void emitChunk() {
if (buffer.isEmpty()) {
if (isClosed(in)) completeStage();
else pull(in);
} else {
Tuple2<ByteString, ByteString> split = buffer.splitAt(chunkSize);
ByteString chunk = split._1();
buffer = split._2();
push(out, chunk);
}
}
};
}
}
//#bytestring-chunker
{
//#bytestring-chunker2
Source<ByteString, NotUsed> chunksStream =
rawBytes.via(new Chunker(CHUNK_LIMIT));
//#bytestring-chunker2
CompletionStage<List<ByteString>> chunksFuture = chunksStream.limit(10).runWith(Sink.seq(), mat);
List<ByteString> chunks = chunksFuture.toCompletableFuture().get(3, TimeUnit.SECONDS);
for (ByteString chunk : chunks) {
assertTrue(chunk.size() <= 2);
}
ByteString sum = ByteString.empty();
for (ByteString chunk : chunks) {
sum = sum.concat(chunk);
}
assertEquals(sum, ByteString.fromArray(new byte[] { 1, 2, 3, 4, 5, 6, 7, 8, 9 }));
}
};
}
@Test
public void limiterShouldWork() throws Exception {
new TestKit(system) {
final int SIZE_LIMIT = 9;
//#bytes-limiter
class ByteLimiter extends GraphStage<FlowShape<ByteString, ByteString>> {
final long maximumBytes;
public Inlet<ByteString> in = Inlet.<ByteString>create("ByteLimiter.in");
public Outlet<ByteString> out = Outlet.<ByteString>create("ByteLimiter.out");
private FlowShape<ByteString, ByteString> shape = FlowShape.of(in, out);
public ByteLimiter(long maximumBytes) {
this.maximumBytes = maximumBytes;
}
@Override
public FlowShape<ByteString, ByteString> shape() {
return shape;
}
@Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new GraphStageLogic(shape) {
private int count = 0;
{
setHandler(out, new AbstractOutHandler() {
@Override
public void onPull() throws Exception {
pull(in);
}
});
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() throws Exception {
ByteString chunk = grab(in);
count += chunk.size();
if (count > maximumBytes) {
failStage(new IllegalStateException("Too much bytes"));
} else {
push(out, chunk);
}
}
});
}
};
}
}
//#bytes-limiter
{
//#bytes-limiter2
Flow<ByteString, ByteString, NotUsed> limiter =
Flow.of(ByteString.class).via(new ByteLimiter(SIZE_LIMIT));
//#bytes-limiter2
final Source<ByteString, NotUsed> bytes1 = Source.from(Arrays.asList(
ByteString.fromArray(new byte[] { 1, 2 }),
ByteString.fromArray(new byte[] { 3 }),
ByteString.fromArray(new byte[] { 4, 5, 6 }),
ByteString.fromArray(new byte[] { 7, 8, 9 })));
final Source<ByteString, NotUsed> bytes2 = Source.from(Arrays.asList(
ByteString.fromArray(new byte[] { 1, 2 }),
ByteString.fromArray(new byte[] { 3 }),
ByteString.fromArray(new byte[] { 4, 5, 6 }),
ByteString.fromArray(new byte[] { 7, 8, 9, 10 })));
List<ByteString> got = bytes1.via(limiter).limit(10).runWith(Sink.seq(), mat).toCompletableFuture().get(3, TimeUnit.SECONDS);
ByteString acc = ByteString.empty();
for (ByteString b : got) {
acc = acc.concat(b);
}
assertEquals(acc, ByteString.fromArray(new byte[] { 1, 2, 3, 4, 5, 6, 7, 8, 9 }));
boolean thrown = false;
try {
bytes2.via(limiter).limit(10).runWith(Sink.seq(), mat).toCompletableFuture().get(3, TimeUnit.SECONDS);
} catch (ExecutionException ex) {
assertEquals(ex.getCause().getClass(), IllegalStateException.class);
thrown = true;
}
assertTrue("Expected IllegalStateException to be thrown", thrown);
}
};
}
@Test
public void compacting() throws Exception {
new TestKit(system) {
{
final Source<ByteString, NotUsed> rawBytes = Source.from(Arrays.asList(
ByteString.fromArray(new byte[] { 1, 2 }),
ByteString.fromArray(new byte[] { 3 }),
ByteString.fromArray(new byte[] { 4, 5, 6 }),
ByteString.fromArray(new byte[] { 7, 8, 9 })));
//#compacting-bytestrings
Source<ByteString, NotUsed> compacted = rawBytes.map(ByteString::compact);
//#compacting-bytestrings
List<ByteString> got = compacted.limit(10).runWith(Sink.seq(), mat).toCompletableFuture().get(3, TimeUnit.SECONDS);
for (ByteString byteString : got) {
assertTrue(byteString.isCompact());
}
}
};
}
}

View file

@ -0,0 +1,67 @@
/**
* Copyright (C) 2016-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
import akka.stream.javadsl.Compression;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.testkit.javadsl.TestKit;
import akka.util.ByteString;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.concurrent.TimeUnit;
import java.util.zip.GZIPOutputStream;
public class RecipeDecompress extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeDecompress");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
private ByteString gzip(final String s) throws IOException {
final ByteArrayOutputStream buf = new ByteArrayOutputStream();
final GZIPOutputStream out = new GZIPOutputStream(buf);
try {
out.write(s.getBytes(StandardCharsets.UTF_8));
} finally {
out.close();
}
return ByteString.fromArray(buf.toByteArray());
}
@Test
public void parseLines() throws Exception {
final Source<ByteString, NotUsed> compressed = Source.single(gzip("Hello World"));
//#decompress-gzip
final Source<String, NotUsed> uncompressed = compressed
.via(Compression.gunzip(100))
.map(b -> b.utf8String());
//#decompress-gzip
uncompressed.runWith(Sink.head(), mat).toCompletableFuture().get(1, TimeUnit.SECONDS);
}
}

View file

@ -0,0 +1,129 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.*;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.stream.stage.*;
import akka.testkit.javadsl.TestKit;
import akka.util.ByteString;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
import java.util.Arrays;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.assertEquals;
public class RecipeDigest extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeDigest");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
//#calculating-digest
class DigestCalculator extends GraphStage<FlowShape<ByteString, ByteString>> {
private final String algorithm;
public Inlet<ByteString> in = Inlet.<ByteString>create("DigestCalculator.in");
public Outlet<ByteString> out = Outlet.<ByteString>create("DigestCalculator.out");
private FlowShape<ByteString, ByteString> shape = FlowShape.of(in, out);
public DigestCalculator(String algorithm) {
this.algorithm = algorithm;
}
@Override
public FlowShape<ByteString, ByteString> shape() {
return shape;
}
@Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new GraphStageLogic(shape) {
final MessageDigest digest;
{
try {
digest = MessageDigest.getInstance(algorithm);
} catch(NoSuchAlgorithmException ex) {
throw new RuntimeException(ex);
}
setHandler(out, new AbstractOutHandler() {
@Override
public void onPull() throws Exception {
pull(in);
}
});
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() throws Exception {
ByteString chunk = grab(in);
digest.update(chunk.toArray());
pull(in);
}
@Override
public void onUpstreamFinish() throws Exception {
// If the stream is finished, we need to emit the digest
// before completing
emit(out, ByteString.fromArray(digest.digest()));
completeStage();
}
});
}
};
}
}
//#calculating-digest
@Test
public void work() throws Exception {
new TestKit(system) {
{
Source<ByteString, NotUsed> data = Source.from(Arrays.asList(
ByteString.fromString("abcdbcdecdef"),
ByteString.fromString("defgefghfghighijhijkijkljklmklmnlmnomnopnopq")));
//#calculating-digest2
final Source<ByteString, NotUsed> digest = data
.via(new DigestCalculator("SHA-256"));
//#calculating-digest2
ByteString got = digest.runWith(Sink.head(), mat).toCompletableFuture().get(3, TimeUnit.SECONDS);
assertEquals(ByteString.fromInts(
0x24, 0x8d, 0x6a, 0x61,
0xd2, 0x06, 0x38, 0xb8,
0xe5, 0xc0, 0x26, 0x93,
0x0c, 0x3e, 0x60, 0x39,
0xa3, 0x3c, 0xe4, 0x59,
0x64, 0xff, 0x21, 0x67,
0xf6, 0xec, 0xed, 0xd4,
0x19, 0xdb, 0x06, 0xc1), got);
}
};
}
}

View file

@ -0,0 +1,77 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.Done;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.*;
import akka.stream.javadsl.*;
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;
import java.util.concurrent.CompletionStage;
public class RecipeDroppyBroadcast extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeDroppyBroadcast");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
@Test
public void work() throws Exception {
new TestKit(system) {
//#droppy-bcast
// Makes a sink drop elements if too slow
public <T> Sink<T, CompletionStage<Done>> droppySink(Sink<T, CompletionStage<Done>> sink, int size) {
return Flow.<T> create()
.buffer(size, OverflowStrategy.dropHead())
.toMat(sink, Keep.right());
}
//#droppy-bcast
{
final List<Integer> nums = new ArrayList<>();
for (int i = 0; i < 100; i++) {
nums.add(i + 1);
}
final Sink<Integer, CompletionStage<Done>> mySink1 = Sink.ignore();
final Sink<Integer, CompletionStage<Done>> mySink2 = Sink.ignore();
final Sink<Integer, CompletionStage<Done>> mySink3 = Sink.ignore();
final Source<Integer, NotUsed> myData = Source.from(nums);
//#droppy-bcast2
RunnableGraph.fromGraph(GraphDSL.create(builder -> {
final int outputCount = 3;
final UniformFanOutShape<Integer, Integer> bcast =
builder.add(Broadcast.create(outputCount));
builder.from(builder.add(myData)).toFanOut(bcast);
builder.from(bcast).to(builder.add(droppySink(mySink1, 10)));
builder.from(bcast).to(builder.add(droppySink(mySink2, 10)));
builder.from(bcast).to(builder.add(droppySink(mySink3, 10)));
return ClosedShape.getInstance();
}));
//#droppy-bcast2
}
};
}
}

View file

@ -0,0 +1,60 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.assertEquals;
public class RecipeFlattenList extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeFlattenList");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
@Test
public void workWithMapConcat() throws Exception {
new TestKit(system) {
{
Source<List<Message>, NotUsed> someDataSource = Source
.from(Arrays.asList(Arrays.asList(new Message("1")), Arrays.asList(new Message("2"), new Message("3"))));
//#flattening-lists
Source<List<Message>, NotUsed> myData = someDataSource;
Source<Message, NotUsed> flattened = myData.mapConcat(i -> i);
//#flattening-lists
List<Message> got = flattened.limit(10).runWith(Sink.seq(), mat).toCompletableFuture().get(1, TimeUnit.SECONDS);
assertEquals(got.get(0), new Message("1"));
assertEquals(got.get(1), new Message("2"));
assertEquals(got.get(2), new Message("3"));
}
};
}
}

View file

@ -0,0 +1,235 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.*;
import akka.pattern.PatternsCS;
import akka.stream.*;
import akka.stream.javadsl.*;
import akka.stream.testkit.TestSubscriber;
import akka.stream.testkit.javadsl.TestSink;
import akka.testkit.javadsl.TestKit;
import akka.util.Timeout;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import java.util.*;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
import static junit.framework.TestCase.assertTrue;
public class RecipeGlobalRateLimit extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeGlobalRateLimit");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
static
//#global-limiter-actor
public class Limiter extends AbstractActor {
public static class WantToPass {}
public static final WantToPass WANT_TO_PASS = new WantToPass();
public static class MayPass {}
public static final MayPass MAY_PASS = new MayPass();
public static class ReplenishTokens {}
public static final ReplenishTokens REPLENISH_TOKENS = new ReplenishTokens();
private final int maxAvailableTokens;
private final FiniteDuration tokenRefreshPeriod;
private final int tokenRefreshAmount;
private final List<ActorRef> waitQueue = new ArrayList<>();
private final Cancellable replenishTimer;
private int permitTokens;
public static Props props(int maxAvailableTokens, FiniteDuration tokenRefreshPeriod,
int tokenRefreshAmount) {
return Props.create(Limiter.class, maxAvailableTokens, tokenRefreshPeriod,
tokenRefreshAmount);
}
private Limiter(int maxAvailableTokens, FiniteDuration tokenRefreshPeriod,
int tokenRefreshAmount) {
this.maxAvailableTokens = maxAvailableTokens;
this.tokenRefreshPeriod = tokenRefreshPeriod;
this.tokenRefreshAmount = tokenRefreshAmount;
this.permitTokens = maxAvailableTokens;
this.replenishTimer = system.scheduler().schedule(
this.tokenRefreshPeriod,
this.tokenRefreshPeriod,
getSelf(),
REPLENISH_TOKENS,
getContext().getSystem().dispatcher(),
getSelf());
}
@Override
public Receive createReceive() {
return open();
}
private Receive open() {
return receiveBuilder()
.match(ReplenishTokens.class, rt -> {
permitTokens = Math.min(permitTokens + tokenRefreshAmount, maxAvailableTokens);
})
.match(WantToPass.class, wtp -> {
permitTokens -= 1;
getSender().tell(MAY_PASS, getSelf());
if (permitTokens == 0) {
getContext().become(closed());
}
})
.build();
}
private Receive closed() {
return receiveBuilder()
.match(ReplenishTokens.class, rt -> {
permitTokens = Math.min(permitTokens + tokenRefreshAmount, maxAvailableTokens);
releaseWaiting();
})
.match(WantToPass.class, wtp -> {
waitQueue.add(getSender());
})
.build();
}
private void releaseWaiting() {
final List<ActorRef> toBeReleased = new ArrayList<>(permitTokens);
for (Iterator<ActorRef> it = waitQueue.iterator(); permitTokens > 0 && it.hasNext();) {
toBeReleased.add(it.next());
it.remove();
permitTokens --;
}
toBeReleased.stream().forEach(ref -> ref.tell(MAY_PASS, getSelf()));
if (permitTokens > 0) {
getContext().become(open());
}
}
@Override
public void postStop() {
replenishTimer.cancel();
waitQueue.stream().forEach(ref -> {
ref.tell(new Status.Failure(new IllegalStateException("limiter stopped")), getSelf());
});
}
}
//#global-limiter-actor
@Test
public void work() throws Exception {
new TestKit(system) {
//#global-limiter-flow
public <T> Flow<T, T, NotUsed> limitGlobal(ActorRef limiter, FiniteDuration maxAllowedWait) {
final int parallelism = 4;
final Flow<T, T, NotUsed> f = Flow.create();
return f.mapAsync(parallelism, element -> {
final Timeout triggerTimeout = new Timeout(maxAllowedWait);
final CompletionStage<Object> limiterTriggerFuture =
PatternsCS.ask(limiter, Limiter.WANT_TO_PASS, triggerTimeout);
return limiterTriggerFuture.thenApplyAsync(response -> element, system.dispatcher());
});
}
//#global-limiter-flow
{
// Use a large period and emulate the timer by hand instead
ActorRef limiter = system.actorOf(Limiter.props(2, new FiniteDuration(100, TimeUnit.DAYS), 1), "limiter");
final Iterator<String> e1 = new Iterator<String>() {
@Override
public boolean hasNext() {
return true;
}
@Override
public String next() {
return "E1";
}
};
final Iterator<String> e2 = new Iterator<String>() {
@Override
public boolean hasNext() {
return true;
}
@Override
public String next() {
return "E2";
}
};
final FiniteDuration twoSeconds = (FiniteDuration) dilated(Duration.create(2, TimeUnit.SECONDS));
final Sink<String, TestSubscriber.Probe<String>> sink = TestSink.probe(system);
final TestSubscriber.Probe<String> probe =
RunnableGraph.<TestSubscriber.Probe<String>>fromGraph(
GraphDSL.create(sink, (builder, s) -> {
final int inputPorts = 2;
final UniformFanInShape<String, String> merge = builder.add(Merge.create(inputPorts));
final SourceShape<String> source1 =
builder.add(Source.<String>fromIterator(() -> e1).via(limitGlobal(limiter, twoSeconds)));
final SourceShape<String> source2 =
builder.add(Source.<String>fromIterator(() -> e2).via(limitGlobal(limiter, twoSeconds)));
builder.from(source1).toFanIn(merge);
builder.from(source2).toFanIn(merge);
builder.from(merge).to(s);
return ClosedShape.getInstance();
})
).run(mat);
probe.expectSubscription().request(1000);
FiniteDuration fiveHundredMillis = FiniteDuration.create(500, TimeUnit.MILLISECONDS);
assertTrue(probe.expectNext().startsWith("E"));
assertTrue(probe.expectNext().startsWith("E"));
probe.expectNoMsg(fiveHundredMillis);
limiter.tell(Limiter.REPLENISH_TOKENS, getTestActor());
assertTrue(probe.expectNext().startsWith("E"));
probe.expectNoMsg(fiveHundredMillis);
final Set<String> resultSet = new HashSet<>();
for (int i = 0; i < 100; i++) {
limiter.tell(Limiter.REPLENISH_TOKENS, getTestActor());
resultSet.add(probe.expectNext());
}
assertTrue(resultSet.contains("E1"));
assertTrue(resultSet.contains("E2"));
probe.expectError();
}
};
}
}

View file

@ -0,0 +1,199 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.actor.ActorSystem;
import akka.japi.Pair;
import akka.stream.*;
import akka.stream.javadsl.Keep;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.stream.stage.*;
import akka.stream.testkit.TestPublisher;
import akka.stream.testkit.TestSubscriber;
import akka.stream.testkit.javadsl.TestSink;
import akka.stream.testkit.javadsl.TestSource;
import akka.testkit.javadsl.TestKit;
import akka.util.ByteString;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.duration.FiniteDuration;
import java.util.concurrent.TimeUnit;
public class RecipeHold extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeHold");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
//#hold-version-1
class HoldWithInitial<T> extends GraphStage<FlowShape<T, T>> {
public Inlet<T> in = Inlet.<T>create("HoldWithInitial.in");
public Outlet<T> out = Outlet.<T>create("HoldWithInitial.out");
private FlowShape<T, T> shape = FlowShape.of(in, out);
private final T initial;
public HoldWithInitial(T initial) {
this.initial = initial;
}
@Override
public FlowShape<T, T> shape() {
return shape;
}
@Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new GraphStageLogic(shape) {
private T currentValue = initial;
{
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() throws Exception {
currentValue = grab(in);
pull(in);
}
});
setHandler(out, new AbstractOutHandler() {
@Override
public void onPull() throws Exception {
push(out, currentValue);
}
});
}
@Override
public void preStart() {
pull(in);
}
};
}
}
//#hold-version-1
//#hold-version-2
class HoldWithWait<T> extends GraphStage<FlowShape<T, T>> {
public Inlet<T> in = Inlet.<T>create("HoldWithInitial.in");
public Outlet<T> out = Outlet.<T>create("HoldWithInitial.out");
private FlowShape<T, T> shape = FlowShape.of(in, out);
@Override
public FlowShape<T, T> shape() {
return shape;
}
@Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new GraphStageLogic(shape) {
private T currentValue = null;
private boolean waitingFirstValue = true;
{
setHandler(in, new AbstractInHandler() {
@Override
public void onPush() throws Exception {
currentValue = grab(in);
if (waitingFirstValue) {
waitingFirstValue = false;
if (isAvailable(out)) push(out, currentValue);
}
pull(in);
}
});
setHandler(out, new AbstractOutHandler() {
@Override
public void onPull() throws Exception {
if (!waitingFirstValue) push(out, currentValue);
}
});
}
@Override
public void preStart() {
pull(in);
}
};
}
}
//#hold-version-2
@Test
public void workForVersion1() throws Exception {
new TestKit(system) {
{
final Source<Integer, TestPublisher.Probe<Integer>> source = TestSource.probe(system);
final Sink<Integer, TestSubscriber.Probe<Integer>> sink = TestSink.probe(system);
Pair<TestPublisher.Probe<Integer>, TestSubscriber.Probe<Integer>> pubSub =
source.via(new HoldWithInitial<>(0)).toMat(sink, Keep.both()).run(mat);
TestPublisher.Probe<Integer> pub = pubSub.first();
TestSubscriber.Probe<Integer> sub = pubSub.second();
sub.requestNext(0);
sub.requestNext(0);
pub.sendNext(1);
pub.sendNext(2);
sub.request(2);
sub.expectNext(2, 2);
pub.sendComplete();
sub.request(1);
sub.expectComplete();
}
};
}
@Test
public void workForVersion2() throws Exception {
new TestKit(system) {
{
final Source<Integer, TestPublisher.Probe<Integer>> source = TestSource.probe(system);
final Sink<Integer, TestSubscriber.Probe<Integer>> sink = TestSink.probe(system);
Pair<TestPublisher.Probe<Integer>, TestSubscriber.Probe<Integer>> pubSub =
source.via(new HoldWithWait<>()).toMat(sink, Keep.both()).run(mat);
TestPublisher.Probe<Integer> pub = pubSub.first();
TestSubscriber.Probe<Integer> sub = pubSub.second();
FiniteDuration timeout = FiniteDuration.create(200, TimeUnit.MILLISECONDS);
sub.request(1);
sub.expectNoMsg(timeout);
pub.sendNext(1);
sub.expectNext(1);
pub.sendNext(2);
pub.sendNext(3);
sub.request(2);
sub.expectNext(3, 3);
pub.sendComplete();
sub.request(1);
sub.expectComplete();
}
};
}
}

View file

@ -0,0 +1,59 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
import akka.stream.javadsl.Flow;
import akka.testkit.javadsl.TestKit;
import akka.util.ByteString;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.util.concurrent.TimeUnit;
public class RecipeKeepAlive extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeKeepAlive");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
class Tick {}
public final Tick TICK = new Tick();
@Test
public void workForVersion1() throws Exception {
new TestKit(system) {
{
final ByteString keepAliveMessage = ByteString.fromArray(new byte[]{11});
//@formatter:off
//#inject-keepalive
Flow<ByteString, ByteString, NotUsed> keepAliveInject =
Flow.of(ByteString.class).keepAlive(
scala.concurrent.duration.Duration.create(1, TimeUnit.SECONDS),
() -> keepAliveMessage);
//#inject-keepalive
//@formatter:on
// Enough to compile, tested elsewhere as a built-in stage
}
};
}
}

View file

@ -0,0 +1,97 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.event.Logging;
import akka.event.LoggingAdapter;
import akka.stream.ActorMaterializer;
import akka.stream.Attributes;
import akka.stream.Materializer;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.testkit.DebugFilter;
import akka.testkit.javadsl.TestKit;
import com.typesafe.config.ConfigFactory;
import jdocs.stream.SilenceSystemOut;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.runtime.AbstractFunction0;
import java.util.Arrays;
public class RecipeLoggingElements extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeLoggingElements", ConfigFactory.parseString("akka.loglevel=DEBUG\nakka.loggers = [akka.testkit.TestEventListener]"));
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
@Test
public void workWithPrintln() throws Exception {
new TestKit(system) {
final SilenceSystemOut.System System = SilenceSystemOut.get(getTestActor());
{
final Source<String, NotUsed> mySource = Source.from(Arrays.asList("1", "2", "3"));
//#println-debug
mySource.map(elem -> {
System.out.println(elem);
return elem;
});
//#println-debug
}
};
}
@Test
public void workWithLog() throws Exception {
new TestKit(system) {
private <T> T analyse(T i) {
return i;
}
{
final Source<String, NotUsed> mySource = Source.from(Arrays.asList("1", "2", "3"));
final int onElement = Logging.WarningLevel();
final int onFinish = Logging.ErrorLevel();
final int onFailure = Logging.ErrorLevel();
//#log-custom
// customise log levels
mySource.log("before-map")
.withAttributes(Attributes.createLogLevels(onElement, onFinish, onFailure))
.map(i -> analyse(i));
// or provide custom logging adapter
final LoggingAdapter adapter = Logging.getLogger(system, "customLogger");
mySource.log("custom", adapter);
//#log-custom
new DebugFilter("customLogger", "[custom] Element: ", false, false, 3).intercept(new AbstractFunction0<Object> () {
public Void apply() {
mySource.log("custom", adapter).runWith(Sink.ignore(), mat);
return null;
}
}, system);
}
};
}
}

View file

@ -0,0 +1,154 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.actor.ActorSystem;
import akka.japi.Pair;
import akka.stream.*;
import akka.stream.javadsl.*;
import akka.stream.testkit.TestPublisher;
import akka.stream.testkit.TestSubscriber;
import akka.stream.testkit.javadsl.TestSink;
import akka.stream.testkit.javadsl.TestSource;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.duration.FiniteDuration;
import java.util.Arrays;
import java.util.concurrent.TimeUnit;
public class RecipeManualTrigger extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeManualTrigger");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
class Trigger {
}
public final Trigger TRIGGER = new Trigger();
@Test
public void zipped() throws Exception {
new TestKit(system) {
{
final Source<Trigger, TestPublisher.Probe<Trigger>> triggerSource = TestSource.probe(system);
final Sink<Message, TestSubscriber.Probe<Message>> messageSink = TestSink.probe(system);
//#manually-triggered-stream
final RunnableGraph<Pair<TestPublisher.Probe<Trigger>, TestSubscriber.Probe<Message>>> g =
RunnableGraph.<Pair<TestPublisher.Probe<Trigger>, TestSubscriber.Probe<Message>>>fromGraph(
GraphDSL.create(
triggerSource,
messageSink,
(p, s) -> new Pair<>(p, s),
(builder, source, sink) -> {
SourceShape<Message> elements =
builder.add(Source.from(Arrays.asList("1", "2", "3", "4")).map(t -> new Message(t)));
FlowShape<Pair<Message, Trigger>, Message> takeMessage =
builder.add(Flow.<Pair<Message, Trigger>>create().map(p -> p.first()));
final FanInShape2<Message, Trigger, Pair<Message, Trigger>> zip =
builder.add(Zip.create());
builder.from(elements).toInlet(zip.in0());
builder.from(source).toInlet(zip.in1());
builder.from(zip.out()).via(takeMessage).to(sink);
return ClosedShape.getInstance();
}
)
);
//#manually-triggered-stream
Pair<TestPublisher.Probe<Trigger>, TestSubscriber.Probe<Message>> pubSub = g.run(mat);
TestPublisher.Probe<Trigger> pub = pubSub.first();
TestSubscriber.Probe<Message> sub = pubSub.second();
FiniteDuration timeout = FiniteDuration.create(100, TimeUnit.MILLISECONDS);
sub.expectSubscription().request(1000);
sub.expectNoMsg(timeout);
pub.sendNext(TRIGGER);
sub.expectNext(new Message("1"));
sub.expectNoMsg(timeout);
pub.sendNext(TRIGGER);
pub.sendNext(TRIGGER);
sub.expectNext(new Message("2"));
sub.expectNext(new Message("3"));
sub.expectNoMsg(timeout);
pub.sendNext(TRIGGER);
sub.expectNext(new Message("4"));
sub.expectComplete();
}
};
}
@Test
public void zipWith() throws Exception {
new TestKit(system) {
{
final Source<Trigger, TestPublisher.Probe<Trigger>> triggerSource = TestSource.probe(system);
final Sink<Message, TestSubscriber.Probe<Message>> messageSink = TestSink.probe(system);
//#manually-triggered-stream-zipwith
final RunnableGraph<Pair<TestPublisher.Probe<Trigger>, TestSubscriber.Probe<Message>>> g =
RunnableGraph.<Pair<TestPublisher.Probe<Trigger>, TestSubscriber.Probe<Message>>>fromGraph(
GraphDSL.create(
triggerSource,
messageSink,
(p, s) -> new Pair<>(p, s),
(builder, source, sink) -> {
final SourceShape<Message> elements =
builder.add(Source.from(Arrays.asList("1", "2", "3", "4")).map(t -> new Message(t)));
final FanInShape2<Message, Trigger, Message> zipWith =
builder.add(ZipWith.create((msg, trigger) -> msg));
builder.from(elements).toInlet(zipWith.in0());
builder.from(source).toInlet(zipWith.in1());
builder.from(zipWith.out()).to(sink);
return ClosedShape.getInstance();
}
)
);
//#manually-triggered-stream-zipwith
Pair<TestPublisher.Probe<Trigger>, TestSubscriber.Probe<Message>> pubSub = g.run(mat);
TestPublisher.Probe<Trigger> pub = pubSub.first();
TestSubscriber.Probe<Message> sub = pubSub.second();
FiniteDuration timeout = FiniteDuration.create(100, TimeUnit.MILLISECONDS);
sub.expectSubscription().request(1000);
sub.expectNoMsg(timeout);
pub.sendNext(TRIGGER);
sub.expectNext(new Message("1"));
sub.expectNoMsg(timeout);
pub.sendNext(TRIGGER);
pub.sendNext(TRIGGER);
sub.expectNext(new Message("2"));
sub.expectNext(new Message("3"));
sub.expectNoMsg(timeout);
pub.sendNext(TRIGGER);
sub.expectNext(new Message("4"));
sub.expectComplete();
}
};
}
}

View file

@ -0,0 +1,98 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.japi.Pair;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
import akka.stream.javadsl.Flow;
import akka.stream.javadsl.Keep;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.stream.testkit.TestPublisher;
import akka.stream.testkit.TestSubscriber;
import akka.stream.testkit.javadsl.TestSink;
import akka.stream.testkit.javadsl.TestSource;
import akka.testkit.TestLatch;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import java.util.concurrent.TimeUnit;
public class RecipeMissedTicks extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeMissedTicks");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
@Test
public void work() throws Exception {
new TestKit(system) {
class Tick {
}
final Tick Tick = new Tick();
{
final Source<Tick, TestPublisher.Probe<Tick>> tickStream = TestSource.probe(system);
final Sink<Integer, TestSubscriber.Probe<Integer>> sink = TestSink.probe(system);
@SuppressWarnings("unused")
//#missed-ticks
final Flow<Tick, Integer, NotUsed> missedTicks =
Flow.of(Tick.class).conflateWithSeed(tick -> 0, (missed, tick) -> missed + 1);
//#missed-ticks
final TestLatch latch = new TestLatch(3, system);
final Flow<Tick, Integer, NotUsed> realMissedTicks =
Flow.of(Tick.class).conflateWithSeed(tick -> 0, (missed, tick) -> { latch.countDown(); return missed + 1; });
Pair<TestPublisher.Probe<Tick>, TestSubscriber.Probe<Integer>> pubSub =
tickStream.via(realMissedTicks).toMat(sink, Keep.both()).run(mat);
TestPublisher.Probe<Tick> pub = pubSub.first();
TestSubscriber.Probe<Integer> sub = pubSub.second();
pub.sendNext(Tick);
pub.sendNext(Tick);
pub.sendNext(Tick);
pub.sendNext(Tick);
FiniteDuration timeout = FiniteDuration.create(200, TimeUnit.MILLISECONDS);
Await.ready(latch, Duration.create(1, TimeUnit.SECONDS));
sub.request(1);
sub.expectNext(3);
sub.request(1);
sub.expectNoMsg(timeout);
pub.sendNext(Tick);
sub.expectNext(0);
pub.sendComplete();
sub.request(1);
sub.expectComplete();
}
};
}
}

View file

@ -0,0 +1,153 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.japi.Function;
import akka.japi.Pair;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.stream.javadsl.SubSource;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
import static java.util.stream.Collectors.toList;
import static junit.framework.TestCase.assertTrue;
public class RecipeMultiGroupByTest extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeMultiGroupBy");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
static class Topic {
private final String name;
public Topic(String name) {
this.name = name;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Topic topic = (Topic) o;
if (name != null ? !name.equals(topic.name) : topic.name != null) {
return false;
}
return true;
}
@Override
public int hashCode() {
return name != null ? name.hashCode() : 0;
}
}
@Test
public void work() throws Exception {
new TestKit(system) {
final List<Topic> extractTopics(Message m) {
final List<Topic> topics = new ArrayList<>(2);
if (m.msg.startsWith("1")) {
topics.add(new Topic("1"));
} else {
topics.add(new Topic("1"));
topics.add(new Topic("2"));
}
return topics;
}
{
final Source<Message, NotUsed> elems = Source
.from(Arrays.asList("1: a", "1: b", "all: c", "all: d", "1: e"))
.map(s -> new Message(s));
//#multi-groupby
final Function<Message, List<Topic>> topicMapper = m -> extractTopics(m);
final Source<Pair<Message, Topic>, NotUsed> messageAndTopic = elems
.mapConcat((Message msg) -> {
List<Topic> topicsForMessage = topicMapper.apply(msg);
// Create a (Msg, Topic) pair for each of the topics
// the message belongs to
return topicsForMessage
.stream()
.map(topic -> new Pair<Message, Topic>(msg, topic))
.collect(toList());
});
SubSource<Pair<Message, Topic>, NotUsed> multiGroups = messageAndTopic
.groupBy(2, pair -> pair.second())
.map(pair -> {
Message message = pair.first();
Topic topic = pair.second();
// do what needs to be done
//#multi-groupby
return pair;
//#multi-groupby
});
//#multi-groupby
CompletionStage<List<String>> result = multiGroups
.grouped(10)
.mergeSubstreams()
.map(pair -> {
Topic topic = pair.get(0).second();
return topic.name + mkString(pair.stream().map(p -> p.first().msg).collect(toList()), "[", ", ", "]");
})
.grouped(10)
.runWith(Sink.head(), mat);
List<String> got = result.toCompletableFuture().get(3, TimeUnit.SECONDS);
assertTrue(got.contains("1[1: a, 1: b, all: c, all: d, 1: e]"));
assertTrue(got.contains("2[all: c, all: d]"));
}
};
}
public static final String mkString(List<String> l, String start, String separate, String end) {
StringBuilder sb = new StringBuilder(start);
for (String s : l) {
sb.append(s).append(separate);
}
return sb
.delete(sb.length() - separate.length(), sb.length())
.append(end).toString();
}
}

View file

@ -0,0 +1,58 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
import akka.stream.javadsl.Framing;
import akka.stream.javadsl.FramingTruncation;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.testkit.javadsl.TestKit;
import akka.util.ByteString;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.util.Arrays;
import java.util.concurrent.TimeUnit;
public class RecipeParseLines extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeParseLines");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
@Test
public void parseLines() throws Exception {
final Source<ByteString, NotUsed> rawData = Source.from(Arrays.asList(
ByteString.fromString("Hello World"),
ByteString.fromString("\r"),
ByteString.fromString("!\r"),
ByteString.fromString("\nHello Akka!\r\nHello Streams!"),
ByteString.fromString("\r\n\r\n")));
//#parse-lines
final Source<String, NotUsed> lines = rawData
.via(Framing.delimiter(ByteString.fromString("\r\n"), 100, FramingTruncation.ALLOW))
.map(b -> b.utf8String());
//#parse-lines
lines.limit(10).runWith(Sink.seq(), mat).toCompletableFuture().get(1, TimeUnit.SECONDS);
}
}

View file

@ -0,0 +1,122 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.japi.Pair;
import akka.japi.function.Function;
import akka.japi.function.Function2;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
import akka.stream.javadsl.Flow;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
public class RecipeReduceByKeyTest extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeReduceByKey");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
@Test
public void work() throws Exception {
new TestKit(system) {
{
final Source<String, NotUsed> words = Source.from(Arrays.asList("hello", "world", "and", "hello", "akka"));
//#word-count
final int MAXIMUM_DISTINCT_WORDS = 1000;
final Source<Pair<String, Integer>, NotUsed> counts = words
// split the words into separate streams first
.groupBy(MAXIMUM_DISTINCT_WORDS, i -> i)
//transform each element to pair with number of words in it
.map(i -> new Pair<>(i, 1))
// add counting logic to the streams
.reduce((left, right) -> new Pair<>(left.first(), left.second() + right.second()))
// get a stream of word counts
.mergeSubstreams();
//#word-count
final CompletionStage<List<Pair<String, Integer>>> f = counts.grouped(10).runWith(Sink.head(), mat);
final Set<Pair<String, Integer>> result = f.toCompletableFuture().get(3, TimeUnit.SECONDS).stream().collect(Collectors.toSet());
final Set<Pair<String, Integer>> expected = new HashSet<>();
expected.add(new Pair<>("hello", 2));
expected.add(new Pair<>("world", 1));
expected.add(new Pair<>("and", 1));
expected.add(new Pair<>("akka", 1));
Assert.assertEquals(expected, result);
}
};
}
//#reduce-by-key-general
static public <In, K, Out> Flow<In, Pair<K, Out>, NotUsed> reduceByKey(
int maximumGroupSize,
Function<In, K> groupKey,
Function<In, Out> map,
Function2<Out, Out, Out> reduce) {
return Flow.<In> create()
.groupBy(maximumGroupSize, groupKey)
.map(i -> new Pair<>(groupKey.apply(i), map.apply(i)))
.reduce((left, right) -> new Pair<>(left.first(), reduce.apply(left.second(), right.second())))
.mergeSubstreams();
}
//#reduce-by-key-general
@Test
public void workGeneralised() throws Exception {
new TestKit(system) {
{
final Source<String, NotUsed> words = Source.from(Arrays.asList("hello", "world", "and", "hello", "akka"));
//#reduce-by-key-general2
final int MAXIMUM_DISTINCT_WORDS = 1000;
Source<Pair<String, Integer>, NotUsed> counts = words.via(reduceByKey(
MAXIMUM_DISTINCT_WORDS,
word -> word,
word -> 1,
(left, right) -> left + right));
//#reduce-by-key-general2
final CompletionStage<List<Pair<String, Integer>>> f = counts.grouped(10).runWith(Sink.head(), mat);
final Set<Pair<String, Integer>> result = f.toCompletableFuture().get(3, TimeUnit.SECONDS).stream().collect(Collectors.toSet());
final Set<Pair<String, Integer>> expected = new HashSet<>();
expected.add(new Pair<>("hello", 2));
expected.add(new Pair<>("world", 1));
expected.add(new Pair<>("and", 1));
expected.add(new Pair<>("akka", 1));
Assert.assertEquals(expected, result);
}
};
}
}

View file

@ -0,0 +1,89 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
public class RecipeSeq extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeLoggingElements");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
@Test
public void drainSourceToList() throws Exception {
new TestKit(system) {
{
final Source<String, NotUsed> mySource = Source.from(Arrays.asList("1", "2", "3"));
//#draining-to-list-unsafe
// Dangerous: might produce a collection with 2 billion elements!
final CompletionStage<List<String>> strings = mySource.runWith(Sink.seq(), mat);
//#draining-to-list-unsafe
strings.toCompletableFuture().get(3, TimeUnit.SECONDS);
}
};
}
@Test
public void drainSourceToListWithLimit() throws Exception {
new TestKit(system) {
{
final Source<String, NotUsed> mySource = Source.from(Arrays.asList("1", "2", "3"));
//#draining-to-list-safe
final int MAX_ALLOWED_SIZE = 100;
// OK. Future will fail with a `StreamLimitReachedException`
// if the number of incoming elements is larger than max
final CompletionStage<List<String>> strings =
mySource.limit(MAX_ALLOWED_SIZE).runWith(Sink.seq(), mat);
//#draining-to-list-safe
strings.toCompletableFuture().get(1, TimeUnit.SECONDS);
}
};
}
public void drainSourceToListWithTake() throws Exception {
new TestKit(system) {
{
final Source<String, NotUsed> mySource = Source.from(Arrays.asList("1", "2", "3"));
final int MAX_ALLOWED_SIZE = 100;
//#draining-to-list-safe
// OK. Collect up until max-th elements only, then cancel upstream
final CompletionStage<List<String>> strings =
mySource.take(MAX_ALLOWED_SIZE).runWith(Sink.seq(), mat);
//#draining-to-list-safe
strings.toCompletableFuture().get(1, TimeUnit.SECONDS);
}
};
}
}

View file

@ -0,0 +1,79 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.japi.Pair;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
import akka.stream.javadsl.Flow;
import akka.stream.testkit.TestPublisher;
import akka.stream.testkit.TestSubscriber;
import akka.stream.testkit.javadsl.TestSink;
import akka.stream.testkit.javadsl.TestSource;
import akka.testkit.TestLatch;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.duration.Duration;
import java.util.concurrent.TimeUnit;
public class RecipeSimpleDrop extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeSimpleDrop");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
@Test
public void work() throws Exception {
new TestKit(system) {
{
@SuppressWarnings("unused")
//#simple-drop
final Flow<Message, Message, NotUsed> droppyStream =
Flow.of(Message.class).conflate((lastMessage, newMessage) -> newMessage);
//#simple-drop
final TestLatch latch = new TestLatch(2, system);
final Flow<Message, Message, NotUsed> realDroppyStream =
Flow.of(Message.class).conflate((lastMessage, newMessage) -> { latch.countDown(); return newMessage; });
final Pair<TestPublisher.Probe<Message>, TestSubscriber.Probe<Message>> pubSub = TestSource
.<Message> probe(system)
.via(realDroppyStream)
.toMat(TestSink.probe(system),
(pub, sub) -> new Pair<>(pub, sub))
.run(mat);
final TestPublisher.Probe<Message> pub = pubSub.first();
final TestSubscriber.Probe<Message> sub = pubSub.second();
pub.sendNext(new Message("1"));
pub.sendNext(new Message("2"));
pub.sendNext(new Message("3"));
Await.ready(latch, Duration.create(1, TimeUnit.SECONDS));
sub.requestNext(new Message("3"));
pub.sendComplete();
sub.request(1);
sub.expectComplete();
}
};
}
}

View file

@ -0,0 +1,42 @@
package jdocs.stream.javadsl.cookbook;
import jdocs.AbstractJavaTest;
public abstract class RecipeTest extends AbstractJavaTest {
final class Message {
public final String msg;
public Message(String msg) {
this.msg = msg;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Message message = (Message) o;
if (msg != null ? !msg.equals(message.msg) : message.msg != null) {
return false;
}
return true;
}
@Override
public int hashCode() {
return msg != null ? msg.hashCode() : 0;
}
}
final class Trigger {
}
final class Job {
}
}

View file

@ -0,0 +1,87 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package jdocs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.*;
import akka.stream.javadsl.*;
import akka.testkit.javadsl.TestKit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.duration.FiniteDuration;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.assertTrue;
public class RecipeWorkerPool extends RecipeTest {
static ActorSystem system;
static Materializer mat;
@BeforeClass
public static void setup() {
system = ActorSystem.create("RecipeWorkerPool");
mat = ActorMaterializer.create(system);
}
@AfterClass
public static void tearDown() {
TestKit.shutdownActorSystem(system);
system = null;
mat = null;
}
//#worker-pool
public static <In, Out> Flow<In, Out, NotUsed> balancer(
Flow<In, Out, NotUsed> worker, int workerCount) {
return Flow.fromGraph(GraphDSL.create(b -> {
boolean waitForAllDownstreams = true;
final UniformFanOutShape<In, In> balance =
b.add(Balance.<In>create(workerCount, waitForAllDownstreams));
final UniformFanInShape<Out, Out> merge =
b.add(Merge.<Out>create(workerCount));
for (int i = 0; i < workerCount; i++) {
b.from(balance.out(i)).via(b.add(worker.async())).toInlet(merge.in(i));
}
return FlowShape.of(balance.in(), merge.out());
}));
}
//#worker-pool
@Test
public void workForVersion1() throws Exception {
new TestKit(system) {
{
Source<Message, NotUsed> data =
Source
.from(Arrays.asList("1", "2", "3", "4", "5"))
.map(t -> new Message(t));
Flow<Message, Message, NotUsed> worker = Flow.of(Message.class).map(m -> new Message(m.msg + " done"));
//#worker-pool2
Flow<Message, Message, NotUsed> balancer = balancer(worker, 3);
Source<Message, NotUsed> processedJobs = data.via(balancer);
//#worker-pool2
FiniteDuration timeout = FiniteDuration.create(200, TimeUnit.MILLISECONDS);
CompletionStage<List<String>> future = processedJobs.map(m -> m.msg).limit(10).runWith(Sink.seq(), mat);
List<String> got = future.toCompletableFuture().get(1, TimeUnit.SECONDS);
assertTrue(got.contains("1 done"));
assertTrue(got.contains("2 done"));
assertTrue(got.contains("3 done"));
assertTrue(got.contains("4 done"));
assertTrue(got.contains("5 done"));
}
};
}
}