rename akka-docs dir to docs (#62)
This commit is contained in:
parent
13dce0ec69
commit
708da8caec
1029 changed files with 2033 additions and 2039 deletions
267
docs/src/test/java/jdocs/stream/BidiFlowDocTest.java
Normal file
267
docs/src/test/java/jdocs/stream/BidiFlowDocTest.java
Normal file
|
|
@ -0,0 +1,267 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://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 org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.stream.javadsl.GraphDSL;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.pf.PFBuilder;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.stream.stage.*;
|
||||
import org.apache.pekko.util.ByteIterator;
|
||||
import org.apache.pekko.util.ByteString;
|
||||
import static org.apache.pekko.util.ByteString.emptyByteString;
|
||||
import org.apache.pekko.util.ByteStringBuilder;
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
|
||||
public class BidiFlowDocTest extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("FlowDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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 = emptyByteString();
|
||||
// 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(), system);
|
||||
assertArrayEquals(
|
||||
new Message[] {new Pong(0), new Pong(1), new Pong(2)},
|
||||
result.toCompletableFuture().get(1, TimeUnit.SECONDS).toArray(new Message[0]));
|
||||
// #compose
|
||||
}
|
||||
}
|
||||
314
docs/src/test/java/jdocs/stream/CompositionDocTest.java
Normal file
314
docs/src/test/java/jdocs/stream/CompositionDocTest.java
Normal file
|
|
@ -0,0 +1,314 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://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 org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.stream.ClosedShape;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.stream.javadsl.Tcp.OutgoingConnection;
|
||||
import org.apache.pekko.util.ByteString;
|
||||
|
||||
public class CompositionDocTest extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("CompositionDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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
|
||||
}
|
||||
}
|
||||
362
docs/src/test/java/jdocs/stream/FlowDocTest.java
Normal file
362
docs/src/test/java/jdocs/stream/FlowDocTest.java
Normal file
|
|
@ -0,0 +1,362 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.Done;
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.AbstractActor;
|
||||
import org.apache.pekko.actor.ActorRef;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.actor.Cancellable;
|
||||
import org.apache.pekko.dispatch.Futures;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.Arrays;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
public class FlowDocTest extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("FlowDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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, Integer::sum), system); // 55
|
||||
|
||||
// returns new Source<Integer>, with `map()` appended
|
||||
final Source<Integer, NotUsed> zeroes = source.map(x -> 0);
|
||||
final Sink<Integer, CompletionStage<Integer>> fold = Sink.fold(0, Integer::sum);
|
||||
zeroes.runWith(fold, system); // 0
|
||||
// #source-immutable
|
||||
|
||||
int result = zeroes.runWith(fold, system).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.fold(0, Integer::sum);
|
||||
|
||||
// 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(system);
|
||||
// #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.fold(0, Integer::sum);
|
||||
|
||||
// materialize the flow, getting the Sink's materialized value
|
||||
final CompletionStage<Integer> sum = source.runWith(sink, system);
|
||||
// #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.fold(0, Integer::sum);
|
||||
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(system);
|
||||
final CompletionStage<Integer> sum2 = runnable.run(system);
|
||||
|
||||
// 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 {
|
||||
final Object tick = new Object();
|
||||
|
||||
final Duration oneSecond = Duration.ofSeconds(1);
|
||||
// akka.actor.Cancellable
|
||||
final Source<Object, Cancellable> timer = Source.tick(oneSecond, oneSecond, tick);
|
||||
|
||||
Sink.ignore().runWith(timer, system);
|
||||
|
||||
final Source<String, Cancellable> timerMap = timer.map(t -> "tick");
|
||||
// WRONG: returned type is not the timers Cancellable!
|
||||
// Cancellable timerCancellable = Sink.ignore().runWith(timerMap, mat);
|
||||
|
||||
// retain the materialized map, in order to retrieve the timer's Cancellable
|
||||
final Cancellable timerCancellable = timer.to(Sink.ignore()).run(system);
|
||||
timerCancellable.cancel();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void creatingSourcesSinks() throws Exception {
|
||||
// #source-sink
|
||||
// Create a source from an Iterable
|
||||
List<Integer> list = new LinkedList<>();
|
||||
list.add(1);
|
||||
list.add(2);
|
||||
list.add(3);
|
||||
Source.from(list);
|
||||
|
||||
// Create a source form a Future
|
||||
Source.future(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::sum);
|
||||
|
||||
// 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 {
|
||||
|
||||
Duration oneSecond = Duration.ofSeconds(1);
|
||||
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, system);
|
||||
CompletableFuture<Optional<Integer>> r5 = flow.to(sink).runWith(source, system);
|
||||
Pair<CompletableFuture<Optional<Integer>>, CompletionStage<Integer>> r6 =
|
||||
flow.runWith(source, sink, system);
|
||||
|
||||
// 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
|
||||
}
|
||||
|
||||
@Test
|
||||
public void sourcePreMaterialization() {
|
||||
// #source-prematerialization
|
||||
Source<String, ActorRef> matValuePoweredSource =
|
||||
Source.actorRef(
|
||||
elem -> {
|
||||
// complete stream immediately if we send it Done
|
||||
if (elem == Done.done()) return Optional.of(CompletionStrategy.immediately());
|
||||
else return Optional.empty();
|
||||
},
|
||||
// never fail the stream because of a message
|
||||
elem -> Optional.empty(),
|
||||
100,
|
||||
OverflowStrategy.fail());
|
||||
|
||||
Pair<ActorRef, Source<String, NotUsed>> actorRefSourcePair =
|
||||
matValuePoweredSource.preMaterialize(system);
|
||||
|
||||
actorRefSourcePair.first().tell("Hello!", ActorRef.noSender());
|
||||
|
||||
// pass source around for materialization
|
||||
actorRefSourcePair.second().runWith(Sink.foreach(System.out::println), system);
|
||||
// #source-prematerialization
|
||||
}
|
||||
|
||||
public void fusingAndAsync() {
|
||||
// #flow-async
|
||||
Source.range(1, 3).map(x -> x + 1).async().map(x -> x * 2).to(Sink.ignore());
|
||||
// #flow-async
|
||||
}
|
||||
|
||||
// #materializer-from-actor-context
|
||||
final class RunWithMyself extends AbstractActor {
|
||||
|
||||
Materializer mat = Materializer.createMaterializer(context());
|
||||
|
||||
@Override
|
||||
public void preStart() throws Exception {
|
||||
Source.repeat("hello")
|
||||
.runWith(
|
||||
Sink.onComplete(
|
||||
tryDone -> {
|
||||
System.out.println("Terminated stream: " + tryDone);
|
||||
}),
|
||||
mat);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Receive createReceive() {
|
||||
return receiveBuilder()
|
||||
.match(
|
||||
String.class,
|
||||
p -> {
|
||||
// this WILL terminate the above stream as well
|
||||
context().stop(self());
|
||||
})
|
||||
.build();
|
||||
}
|
||||
}
|
||||
// #materializer-from-actor-context
|
||||
|
||||
// #materializer-from-system-in-actor
|
||||
final class RunForever extends AbstractActor {
|
||||
|
||||
private final Materializer materializer;
|
||||
|
||||
public RunForever(Materializer materializer) {
|
||||
this.materializer = materializer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preStart() throws Exception {
|
||||
Source.repeat("hello")
|
||||
.runWith(
|
||||
Sink.onComplete(
|
||||
tryDone -> {
|
||||
System.out.println("Terminated stream: " + tryDone);
|
||||
}),
|
||||
materializer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Receive createReceive() {
|
||||
return receiveBuilder()
|
||||
.match(
|
||||
String.class,
|
||||
p -> {
|
||||
// will NOT terminate the stream (it's bound to the system!)
|
||||
context().stop(self());
|
||||
})
|
||||
.build();
|
||||
}
|
||||
// #materializer-from-system-in-actor
|
||||
|
||||
}
|
||||
}
|
||||
202
docs/src/test/java/jdocs/stream/FlowErrorDocTest.java
Normal file
202
docs/src/test/java/jdocs/stream/FlowErrorDocTest.java
Normal file
|
|
@ -0,0 +1,202 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://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 org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.japi.pf.PFBuilder;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.Supervision;
|
||||
import org.apache.pekko.stream.ActorAttributes;
|
||||
import org.apache.pekko.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 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, system);
|
||||
// division by zero will fail the stream and the
|
||||
// result here will be a CompletionStage failed with 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 Source<Integer, NotUsed> source =
|
||||
Source.from(Arrays.asList(0, 1, 2, 3, 4, 5))
|
||||
.map(elem -> 100 / elem)
|
||||
.withAttributes(ActorAttributes.withSupervisionStrategy(decider));
|
||||
final Sink<Integer, CompletionStage<Integer>> fold = Sink.fold(0, (acc, elem) -> acc + elem);
|
||||
|
||||
final RunnableGraph<CompletionStage<Integer>> runnableGraph = source.toMat(fold, Keep.right());
|
||||
|
||||
final RunnableGraph<CompletionStage<Integer>> withCustomSupervision =
|
||||
runnableGraph.withAttributes(ActorAttributes.withSupervisionStrategy(decider));
|
||||
|
||||
final CompletionStage<Integer> result = withCustomSupervision.run(system);
|
||||
// the element causing division by zero will be dropped
|
||||
// result here will be a CompletionStage completed with 228
|
||||
// #resume
|
||||
|
||||
assertEquals(Integer.valueOf(228), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateResumeSectionStream() throws Exception {
|
||||
// #resume-section
|
||||
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, system);
|
||||
// the elements causing division by zero will be dropped
|
||||
// result here will be a Future completed with 150
|
||||
// #resume-section
|
||||
|
||||
assertEquals(Integer.valueOf(150), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateRestartSectionStream() throws Exception {
|
||||
// #restart-section
|
||||
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(), system);
|
||||
// the negative element cause the scan stage to be restarted,
|
||||
// i.e. start from 0 again
|
||||
// result here will be a Future completed with List(0, 1, 4, 0, 5, 12)
|
||||
// #restart-section
|
||||
|
||||
assertEquals(
|
||||
Arrays.asList(0, 1, 4, 0, 5, 12), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateRecover() {
|
||||
// #recover
|
||||
Source.from(Arrays.asList(0, 1, 2, 3, 4, 5, 6))
|
||||
.map(
|
||||
n -> {
|
||||
// assuming `4` and `5` are unexpected values that could throw exception
|
||||
if (Arrays.asList(4, 5).contains(n))
|
||||
throw new RuntimeException(String.format("Boom! Bad value found: %s", n));
|
||||
else return n.toString();
|
||||
})
|
||||
.recover(
|
||||
new PFBuilder<Throwable, String>()
|
||||
.match(RuntimeException.class, Throwable::getMessage)
|
||||
.build())
|
||||
.runForeach(System.out::println, system);
|
||||
// #recover
|
||||
|
||||
/*
|
||||
Output:
|
||||
//#recover-output
|
||||
0
|
||||
1
|
||||
2
|
||||
3 // last element before failure
|
||||
Boom! Bad value found: 4 // first element on failure
|
||||
//#recover-output
|
||||
*/
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateRecoverWithRetries() {
|
||||
// #recoverWithRetries
|
||||
Source<String, NotUsed> planB = Source.from(Arrays.asList("five", "six", "seven", "eight"));
|
||||
|
||||
Source.from(Arrays.asList(0, 1, 2, 3, 4, 5, 6))
|
||||
.map(
|
||||
n -> {
|
||||
if (n < 5) return n.toString();
|
||||
else throw new RuntimeException("Boom!");
|
||||
})
|
||||
.recoverWithRetries(
|
||||
1, // max attempts
|
||||
new PFBuilder<Throwable, Source<String, NotUsed>>()
|
||||
.match(RuntimeException.class, ex -> planB)
|
||||
.build())
|
||||
.runForeach(System.out::println, system);
|
||||
// #recoverWithRetries
|
||||
|
||||
/*
|
||||
Output:
|
||||
//#recoverWithRetries-output
|
||||
0
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
five
|
||||
six
|
||||
seven
|
||||
eight
|
||||
//#recoverWithRetries-output
|
||||
*/
|
||||
}
|
||||
}
|
||||
166
docs/src/test/java/jdocs/stream/FlowParallelismDocTest.java
Normal file
166
docs/src/test/java/jdocs/stream/FlowParallelismDocTest.java
Normal file
|
|
@ -0,0 +1,166 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
|
||||
public class FlowParallelismDocTest extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("FlowParallellismDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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
|
||||
}
|
||||
}
|
||||
161
docs/src/test/java/jdocs/stream/FlowStreamRefsDocTest.java
Normal file
161
docs/src/test/java/jdocs/stream/FlowStreamRefsDocTest.java
Normal file
|
|
@ -0,0 +1,161 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.AbstractActor;
|
||||
import org.apache.pekko.actor.ActorRef;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.actor.Props;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.junit.Test;
|
||||
import scala.concurrent.duration.FiniteDuration;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class FlowStreamRefsDocTest extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system = null;
|
||||
static Materializer mat = null;
|
||||
|
||||
@Test
|
||||
public void compileOnlySpec() {
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// #offer-source
|
||||
static class RequestLogs {
|
||||
public final long streamId;
|
||||
|
||||
public RequestLogs(long streamId) {
|
||||
this.streamId = streamId;
|
||||
}
|
||||
}
|
||||
|
||||
static class LogsOffer {
|
||||
final SourceRef<String> sourceRef;
|
||||
|
||||
public LogsOffer(SourceRef<String> sourceRef) {
|
||||
this.sourceRef = sourceRef;
|
||||
}
|
||||
}
|
||||
|
||||
static class DataSource extends AbstractActor {
|
||||
@Override
|
||||
public Receive createReceive() {
|
||||
return receiveBuilder().match(RequestLogs.class, this::handleRequestLogs).build();
|
||||
}
|
||||
|
||||
private void handleRequestLogs(RequestLogs requestLogs) {
|
||||
Source<String, NotUsed> logs = streamLogs(requestLogs.streamId);
|
||||
SourceRef<String> logsRef = logs.runWith(StreamRefs.sourceRef(), mat);
|
||||
|
||||
getSender().tell(new LogsOffer(logsRef), getSelf());
|
||||
}
|
||||
|
||||
private Source<String, NotUsed> streamLogs(long streamId) {
|
||||
return Source.repeat("[INFO] some interesting logs here (for id: " + streamId + ")");
|
||||
}
|
||||
}
|
||||
// #offer-source
|
||||
|
||||
public void offerSource() {
|
||||
new TestKit(system) {
|
||||
{
|
||||
|
||||
// #offer-source-use
|
||||
ActorRef sourceActor = system.actorOf(Props.create(DataSource.class), "dataSource");
|
||||
|
||||
sourceActor.tell(new RequestLogs(1337), getTestActor());
|
||||
LogsOffer offer = expectMsgClass(LogsOffer.class);
|
||||
|
||||
offer.sourceRef.getSource().runWith(Sink.foreach(log -> System.out.println(log)), mat);
|
||||
|
||||
// #offer-source-use
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// #offer-sink
|
||||
static class PrepareUpload {
|
||||
final String id;
|
||||
|
||||
public PrepareUpload(String id) {
|
||||
this.id = id;
|
||||
}
|
||||
}
|
||||
|
||||
static class MeasurementsSinkReady {
|
||||
final String id;
|
||||
final SinkRef<String> sinkRef;
|
||||
|
||||
public MeasurementsSinkReady(String id, SinkRef<String> ref) {
|
||||
this.id = id;
|
||||
this.sinkRef = ref;
|
||||
}
|
||||
}
|
||||
|
||||
static class DataReceiver extends AbstractActor {
|
||||
@Override
|
||||
public Receive createReceive() {
|
||||
return receiveBuilder()
|
||||
.match(
|
||||
PrepareUpload.class,
|
||||
prepare -> {
|
||||
Sink<String, NotUsed> sink = logsSinkFor(prepare.id);
|
||||
SinkRef<String> sinkRef = StreamRefs.<String>sinkRef().to(sink).run(mat);
|
||||
|
||||
getSender().tell(new MeasurementsSinkReady(prepare.id, sinkRef), getSelf());
|
||||
})
|
||||
.build();
|
||||
}
|
||||
|
||||
private Sink<String, NotUsed> logsSinkFor(String id) {
|
||||
return Sink.<String>ignore().mapMaterializedValue(done -> NotUsed.getInstance());
|
||||
}
|
||||
}
|
||||
// #offer-sink
|
||||
|
||||
public void offerSink() {
|
||||
new TestKit(system) {
|
||||
{
|
||||
|
||||
// #offer-sink-use
|
||||
ActorRef receiver = system.actorOf(Props.create(DataReceiver.class), "dataReceiver");
|
||||
|
||||
receiver.tell(new PrepareUpload("system-42-tmp"), getTestActor());
|
||||
MeasurementsSinkReady ready = expectMsgClass(MeasurementsSinkReady.class);
|
||||
|
||||
Source.repeat("hello").runWith(ready.sinkRef.getSink(), mat);
|
||||
// #offer-sink-use
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public void configureTimeouts() {
|
||||
new TestKit(system) {
|
||||
{
|
||||
|
||||
// #attr-sub-timeout
|
||||
FiniteDuration timeout = FiniteDuration.create(5, TimeUnit.SECONDS);
|
||||
Attributes timeoutAttributes = StreamRefAttributes.subscriptionTimeout(timeout);
|
||||
|
||||
// configuring Sink.sourceRef (notice that we apply the attributes to the Sink!):
|
||||
Source.repeat("hello")
|
||||
.runWith(StreamRefs.<String>sourceRef().addAttributes(timeoutAttributes), mat);
|
||||
|
||||
// configuring SinkRef.source:
|
||||
StreamRefs.<String>sinkRef()
|
||||
.addAttributes(timeoutAttributes)
|
||||
.runWith(Sink.<String>ignore(), mat); // not very interesting sink, just an example
|
||||
|
||||
// #attr-sub-timeout
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
179
docs/src/test/java/jdocs/stream/GraphCyclesDocTest.java
Normal file
179
docs/src/test/java/jdocs/stream/GraphCyclesDocTest.java
Normal file
|
|
@ -0,0 +1,179 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.stream.scaladsl.MergePreferred.MergePreferredShape;
|
||||
|
||||
public class GraphCyclesDocTest extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("GraphCyclesDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
}
|
||||
|
||||
static final 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
|
||||
}
|
||||
}
|
||||
770
docs/src/test/java/jdocs/stream/GraphStageDocTest.java
Normal file
770
docs/src/test/java/jdocs/stream/GraphStageDocTest.java
Normal file
|
|
@ -0,0 +1,770 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
// #imports
|
||||
import org.apache.pekko.Done;
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Option;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.japi.Predicate;
|
||||
import org.apache.pekko.japi.Function;
|
||||
import org.apache.pekko.japi.function.Procedure;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.stream.stage.*;
|
||||
// #imports
|
||||
import org.apache.pekko.stream.testkit.TestPublisher;
|
||||
import org.apache.pekko.stream.testkit.TestSubscriber;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.reactivestreams.Subscription;
|
||||
import scala.concurrent.ExecutionContext;
|
||||
|
||||
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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("GraphStageDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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, system);
|
||||
|
||||
// The source is reusable. This returns 5050
|
||||
CompletionStage<Integer> result2 =
|
||||
mySource.take(100).runFold(0, (sum, next) -> sum + next, system);
|
||||
// #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, system);
|
||||
}
|
||||
|
||||
// #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, system);
|
||||
|
||||
assertEquals(Integer.valueOf(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, system);
|
||||
|
||||
assertEquals(Integer.valueOf(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, system);
|
||||
|
||||
assertEquals(Integer.valueOf(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, system);
|
||||
|
||||
assertEquals(Integer.valueOf(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-operator-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, system);
|
||||
|
||||
// #graph-operator-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(system);
|
||||
|
||||
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 int silencePeriodInSeconds;
|
||||
|
||||
public TimedGate(int silencePeriodInSeconds) {
|
||||
this.silencePeriodInSeconds = silencePeriodInSeconds;
|
||||
}
|
||||
|
||||
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", java.time.Duration.ofSeconds(silencePeriodInSeconds));
|
||||
}
|
||||
}
|
||||
});
|
||||
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<>(2))
|
||||
.takeWithin(java.time.Duration.ofMillis(250))
|
||||
.runFold(0, (n, sum) -> n + sum, system);
|
||||
|
||||
assertEquals(Integer.valueOf(1), result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
// #materialized
|
||||
public class FirstValue<A>
|
||||
extends AbstractGraphStageWithMaterializedValue<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 Pair<GraphStageLogic, CompletionStage<A>> createLogicAndMaterializedValuePair(
|
||||
Attributes inheritedAttributes) {
|
||||
CompletableFuture<A> promise = new CompletableFuture<>();
|
||||
|
||||
GraphStageLogic logic =
|
||||
new GraphStageLogic(shape) {
|
||||
{
|
||||
setHandler(
|
||||
in,
|
||||
new AbstractInHandler() {
|
||||
@Override
|
||||
public void onPush() {
|
||||
A elem = grab(in);
|
||||
promise.complete(elem);
|
||||
push(out, elem);
|
||||
|
||||
// replace handler with one that only forwards elements
|
||||
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 Pair<>(logic, promise);
|
||||
}
|
||||
}
|
||||
// #materialized
|
||||
|
||||
public void demonstrateACustomMaterializedValue() throws Exception {
|
||||
// tests:
|
||||
RunnableGraph<CompletionStage<Integer>> flow =
|
||||
Source.from(Arrays.asList(1, 2, 3))
|
||||
.viaMat(new FirstValue<Integer>(), Keep.right())
|
||||
.to(Sink.ignore());
|
||||
|
||||
CompletionStage<Integer> result = flow.run(system);
|
||||
|
||||
assertEquals(Integer.valueOf(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, system);
|
||||
|
||||
assertEquals(Integer.valueOf(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(system);
|
||||
|
||||
Subscription sub = subscriber.expectSubscription();
|
||||
// this happens even though the subscriber has not signalled any demand
|
||||
publisher.sendNext(1);
|
||||
publisher.sendNext(2);
|
||||
|
||||
sub.cancel();
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,71 @@
|
|||
/*
|
||||
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.Attributes;
|
||||
import org.apache.pekko.stream.Materializer;
|
||||
import org.apache.pekko.stream.Outlet;
|
||||
import org.apache.pekko.stream.SourceShape;
|
||||
// #stage-with-logging
|
||||
import org.apache.pekko.stream.stage.AbstractOutHandler;
|
||||
import org.apache.pekko.stream.stage.GraphStage;
|
||||
import org.apache.pekko.stream.stage.GraphStageLogic;
|
||||
import org.apache.pekko.stream.stage.GraphStageLogicWithLogging;
|
||||
|
||||
// #stage-with-logging
|
||||
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 {}
|
||||
|
||||
// #operator-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);
|
||||
}
|
||||
});
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
// #operator-with-logging
|
||||
|
||||
}
|
||||
258
docs/src/test/java/jdocs/stream/HubDocTest.java
Normal file
258
docs/src/test/java/jdocs/stream/HubDocTest.java
Normal file
|
|
@ -0,0 +1,258 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.Done;
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.actor.Cancellable;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.stream.javadsl.PartitionHub.ConsumerInfo;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.function.ToLongBiFunction;
|
||||
|
||||
public class HubDocTest extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("GraphDSLDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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(system);
|
||||
|
||||
Source.single("Hello!").runWith(toConsumer, system);
|
||||
Source.single("Hub!").runWith(toConsumer, system);
|
||||
// #merge-hub
|
||||
}
|
||||
|
||||
@Test
|
||||
public void dynamicBroadcast() {
|
||||
// Used to be able to clean up the running stream
|
||||
Materializer materializer = Materializer.createMaterializer(system);
|
||||
|
||||
// #broadcast-hub
|
||||
// A simple producer that publishes a new "message" every second
|
||||
Source<String, Cancellable> producer =
|
||||
Source.tick(Duration.ofSeconds(1), Duration.ofSeconds(1), "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(system);
|
||||
|
||||
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(), system);
|
||||
// #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(Duration.ofSeconds(1));
|
||||
// #pub-sub-3
|
||||
|
||||
// #pub-sub-4
|
||||
UniqueKillSwitch killSwitch =
|
||||
Source.repeat("Hello World!")
|
||||
.viaMat(busFlow, Keep.right())
|
||||
.to(Sink.foreach(System.out::println))
|
||||
.run(system);
|
||||
|
||||
// Shut down externally
|
||||
killSwitch.shutdown();
|
||||
// #pub-sub-4
|
||||
}
|
||||
|
||||
@Test
|
||||
public void dynamicPartition() {
|
||||
// Used to be able to clean up the running stream
|
||||
Materializer materializer = Materializer.createMaterializer(system);
|
||||
|
||||
// #partition-hub
|
||||
// A simple producer that publishes a new "message-n" every second
|
||||
Source<String, Cancellable> producer =
|
||||
Source.tick(Duration.ofSeconds(1), Duration.ofSeconds(1), "message")
|
||||
.zipWith(Source.range(0, 100), (a, b) -> a + "-" + b);
|
||||
|
||||
// Attach a PartitionHub 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(
|
||||
PartitionHub.of(String.class, (size, elem) -> Math.abs(elem.hashCode() % size), 2, 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);
|
||||
// #partition-hub
|
||||
|
||||
// Cleanup
|
||||
materializer.shutdown();
|
||||
}
|
||||
|
||||
// #partition-hub-stateful-function
|
||||
// Using a class since variable must otherwise be final.
|
||||
// New instance is created for each materialization of the PartitionHub.
|
||||
static class RoundRobin<T> implements ToLongBiFunction<ConsumerInfo, T> {
|
||||
|
||||
private long i = -1;
|
||||
|
||||
@Override
|
||||
public long applyAsLong(ConsumerInfo info, T elem) {
|
||||
i++;
|
||||
return info.consumerIdByIdx((int) (i % info.size()));
|
||||
}
|
||||
}
|
||||
// #partition-hub-stateful-function
|
||||
|
||||
@Test
|
||||
public void dynamicStatefulPartition() {
|
||||
// Used to be able to clean up the running stream
|
||||
Materializer materializer = Materializer.createMaterializer(system);
|
||||
|
||||
// #partition-hub-stateful
|
||||
// A simple producer that publishes a new "message-n" every second
|
||||
Source<String, Cancellable> producer =
|
||||
Source.tick(Duration.ofSeconds(1), Duration.ofSeconds(1), "message")
|
||||
.zipWith(Source.range(0, 100), (a, b) -> a + "-" + b);
|
||||
|
||||
// Attach a PartitionHub 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(
|
||||
PartitionHub.ofStateful(String.class, () -> new RoundRobin<String>(), 2, 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);
|
||||
// #partition-hub-stateful
|
||||
|
||||
// Cleanup
|
||||
materializer.shutdown();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void dynamicFastestPartition() {
|
||||
// Used to be able to clean up the running stream
|
||||
Materializer materializer = Materializer.createMaterializer(system);
|
||||
|
||||
// #partition-hub-fastest
|
||||
Source<Integer, NotUsed> producer = Source.range(0, 100);
|
||||
|
||||
// ConsumerInfo.queueSize is the approximate number of buffered elements for a consumer.
|
||||
// Note that this is a moving target since the elements are consumed concurrently.
|
||||
RunnableGraph<Source<Integer, NotUsed>> runnableGraph =
|
||||
producer.toMat(
|
||||
PartitionHub.ofStateful(
|
||||
Integer.class,
|
||||
() ->
|
||||
(info, elem) -> {
|
||||
final List<Object> ids = info.getConsumerIds();
|
||||
int minValue = info.queueSize(0);
|
||||
long fastest = info.consumerIdByIdx(0);
|
||||
for (int i = 1; i < ids.size(); i++) {
|
||||
int value = info.queueSize(i);
|
||||
if (value < minValue) {
|
||||
minValue = value;
|
||||
fastest = info.consumerIdByIdx(i);
|
||||
}
|
||||
}
|
||||
return fastest;
|
||||
},
|
||||
2,
|
||||
8),
|
||||
Keep.right());
|
||||
|
||||
Source<Integer, NotUsed> fromProducer = runnableGraph.run(materializer);
|
||||
|
||||
fromProducer.runForeach(msg -> System.out.println("consumer1: " + msg), materializer);
|
||||
fromProducer
|
||||
.throttle(10, Duration.ofMillis(100))
|
||||
.runForeach(msg -> System.out.println("consumer2: " + msg), materializer);
|
||||
// #partition-hub-fastest
|
||||
|
||||
// Cleanup
|
||||
materializer.shutdown();
|
||||
}
|
||||
}
|
||||
840
docs/src/test/java/jdocs/stream/IntegrationDocTest.java
Normal file
840
docs/src/test/java/jdocs/stream/IntegrationDocTest.java
Normal file
|
|
@ -0,0 +1,840 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.Done;
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.*;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.testkit.TestProbe;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.apache.pekko.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.time.Duration;
|
||||
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 org.apache.pekko.pattern.Patterns.ask;
|
||||
import static jdocs.stream.TwitterStreamQuickstartDocTest.Model.AKKA;
|
||||
import static jdocs.stream.TwitterStreamQuickstartDocTest.Model.tweets;
|
||||
import static junit.framework.TestCase.assertTrue;
|
||||
|
||||
@SuppressWarnings("ALL")
|
||||
public class IntegrationDocTest extends AbstractJavaTest {
|
||||
|
||||
private static final SilenceSystemOut.System System = SilenceSystemOut.get();
|
||||
|
||||
static ActorSystem system;
|
||||
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("IntegrationDocTest", config);
|
||||
ref = system.actorOf(Props.create(Translator.class));
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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
|
||||
|
||||
// #actorRefWithBackpressure-actor
|
||||
enum Ack {
|
||||
INSTANCE;
|
||||
}
|
||||
|
||||
static class StreamInitialized {}
|
||||
|
||||
static class StreamCompleted {}
|
||||
|
||||
static class StreamFailure {
|
||||
private final Throwable cause;
|
||||
|
||||
public StreamFailure(Throwable cause) {
|
||||
this.cause = cause;
|
||||
}
|
||||
|
||||
public Throwable getCause() {
|
||||
return cause;
|
||||
}
|
||||
}
|
||||
|
||||
static class AckingReceiver extends AbstractLoggingActor {
|
||||
|
||||
private final ActorRef probe;
|
||||
|
||||
public AckingReceiver(ActorRef probe) {
|
||||
this.probe = probe;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Receive createReceive() {
|
||||
return receiveBuilder()
|
||||
.match(
|
||||
StreamInitialized.class,
|
||||
init -> {
|
||||
log().info("Stream initialized");
|
||||
probe.tell("Stream initialized", getSelf());
|
||||
sender().tell(Ack.INSTANCE, self());
|
||||
})
|
||||
.match(
|
||||
String.class,
|
||||
element -> {
|
||||
log().info("Received element: {}", element);
|
||||
probe.tell(element, getSelf());
|
||||
sender().tell(Ack.INSTANCE, self());
|
||||
})
|
||||
.match(
|
||||
StreamCompleted.class,
|
||||
completed -> {
|
||||
log().info("Stream completed");
|
||||
probe.tell("Stream completed", getSelf());
|
||||
})
|
||||
.match(
|
||||
StreamFailure.class,
|
||||
failed -> {
|
||||
log().error(failed.getCause(), "Stream failed!");
|
||||
probe.tell("Stream failed!", getSelf());
|
||||
})
|
||||
.build();
|
||||
}
|
||||
}
|
||||
// #actorRefWithBackpressure-actor
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
public void askStage() throws Exception {
|
||||
// #ask
|
||||
Source<String, NotUsed> words = Source.from(Arrays.asList("hello", "hi"));
|
||||
Timeout askTimeout = Timeout.apply(5, TimeUnit.SECONDS);
|
||||
|
||||
words
|
||||
.ask(5, ref, String.class, askTimeout)
|
||||
// continue processing of the replies from the actor
|
||||
.map(elem -> elem.toLowerCase())
|
||||
.runWith(Sink.ignore(), system);
|
||||
// #ask
|
||||
}
|
||||
|
||||
@Test
|
||||
public void actorRefWithBackpressure() throws Exception {
|
||||
// #actorRefWithBackpressure
|
||||
Source<String, NotUsed> words = Source.from(Arrays.asList("hello", "hi"));
|
||||
|
||||
final TestKit probe = new TestKit(system);
|
||||
|
||||
ActorRef receiver = system.actorOf(Props.create(AckingReceiver.class, probe.getRef()));
|
||||
|
||||
Sink<String, NotUsed> sink =
|
||||
Sink.<String>actorRefWithBackpressure(
|
||||
receiver,
|
||||
new StreamInitialized(),
|
||||
Ack.INSTANCE,
|
||||
new StreamCompleted(),
|
||||
ex -> new StreamFailure(ex));
|
||||
|
||||
words.map(el -> el.toLowerCase()).runWith(sink, system);
|
||||
|
||||
probe.expectMsg("Stream initialized");
|
||||
probe.expectMsg("hello");
|
||||
probe.expectMsg("hi");
|
||||
probe.expectMsg("Stream completed");
|
||||
// #actorRefWithBackpressure
|
||||
}
|
||||
|
||||
@Test
|
||||
public void callingExternalServiceWithMapAsync() throws Exception {
|
||||
new TestKit(system) {
|
||||
final TestKit probe = new TestKit(system);
|
||||
final AddressSystem addressSystem = new AddressSystem();
|
||||
final EmailServer emailServer = new EmailServer(probe.getRef());
|
||||
|
||||
{
|
||||
// #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(system);
|
||||
// #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(system);
|
||||
// #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(system);
|
||||
// #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(system);
|
||||
// #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), Duration.ofMillis(300L)))
|
||||
.to(Sink.ignore());
|
||||
// #save-tweets
|
||||
|
||||
saveTweets.run(system);
|
||||
|
||||
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);
|
||||
|
||||
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)
|
||||
.to(Sink.foreach(elem -> System.out.println("after: " + elem)))
|
||||
.withAttributes(Attributes.inputBuffer(4, 4))
|
||||
.run(system);
|
||||
// #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);
|
||||
|
||||
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)
|
||||
.to(Sink.foreach(elem -> System.out.println("after: " + elem)))
|
||||
.withAttributes(Attributes.inputBuffer(4, 4))
|
||||
.run(system);
|
||||
// #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"));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void illustrateSourceQueue() throws Exception {
|
||||
new TestKit(system) {
|
||||
{
|
||||
// #source-queue
|
||||
int bufferSize = 10;
|
||||
int elementsToProcess = 5;
|
||||
|
||||
BoundedSourceQueue<Integer> sourceQueue =
|
||||
Source.<Integer>queue(bufferSize)
|
||||
.throttle(elementsToProcess, Duration.ofSeconds(3))
|
||||
.map(x -> x * x)
|
||||
.to(Sink.foreach(x -> System.out.println("got: " + x)))
|
||||
.run(system);
|
||||
|
||||
Source<Integer, NotUsed> source = Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
|
||||
|
||||
source.map(x -> sourceQueue.offer(x)).runWith(Sink.ignore(), system);
|
||||
|
||||
// #source-queue
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void illustrateSynchronousSourceQueue() throws Exception {
|
||||
new TestKit(system) {
|
||||
{
|
||||
// #source-queue-synchronous
|
||||
int bufferSize = 10;
|
||||
int elementsToProcess = 5;
|
||||
|
||||
BoundedSourceQueue<Integer> sourceQueue =
|
||||
Source.<Integer>queue(bufferSize)
|
||||
.throttle(elementsToProcess, Duration.ofSeconds(3))
|
||||
.map(x -> x * x)
|
||||
.to(Sink.foreach(x -> System.out.println("got: " + x)))
|
||||
.run(system);
|
||||
|
||||
List<Integer> fastElements = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
|
||||
|
||||
fastElements.stream()
|
||||
.forEach(
|
||||
x -> {
|
||||
QueueOfferResult result = sourceQueue.offer(x);
|
||||
if (result == QueueOfferResult.enqueued()) {
|
||||
System.out.println("enqueued " + x);
|
||||
} else if (result == QueueOfferResult.dropped()) {
|
||||
System.out.println("dropped " + x);
|
||||
} else if (result instanceof QueueOfferResult.Failure) {
|
||||
QueueOfferResult.Failure failure = (QueueOfferResult.Failure) result;
|
||||
System.out.println("Offer failed " + failure.cause().getMessage());
|
||||
} else if (result instanceof QueueOfferResult.QueueClosed$) {
|
||||
System.out.println("Bounded Source Queue closed");
|
||||
}
|
||||
});
|
||||
|
||||
// #source-queue-synchronous
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void illustrateSourceActorRef() throws Exception {
|
||||
new TestKit(system) {
|
||||
{
|
||||
// #source-actorRef
|
||||
int bufferSize = 10;
|
||||
|
||||
Source<Integer, ActorRef> source =
|
||||
Source.actorRef(
|
||||
elem -> {
|
||||
// complete stream immediately if we send it Done
|
||||
if (elem == Done.done()) return Optional.of(CompletionStrategy.immediately());
|
||||
else return Optional.empty();
|
||||
},
|
||||
// never fail the stream because of a message
|
||||
elem -> Optional.empty(),
|
||||
bufferSize,
|
||||
OverflowStrategy.dropHead()); // note: backpressure is not supported
|
||||
ActorRef actorRef =
|
||||
source
|
||||
.map(x -> x * x)
|
||||
.to(Sink.foreach(x -> System.out.println("got: " + x)))
|
||||
.run(system);
|
||||
|
||||
actorRef.tell(1, ActorRef.noSender());
|
||||
actorRef.tell(2, ActorRef.noSender());
|
||||
actorRef.tell(3, ActorRef.noSender());
|
||||
actorRef.tell(
|
||||
new org.apache.pekko.actor.Status.Success(CompletionStrategy.draining()),
|
||||
ActorRef.noSender());
|
||||
// #source-actorRef
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
161
docs/src/test/java/jdocs/stream/KillSwitchDocTest.java
Normal file
161
docs/src/test/java/jdocs/stream/KillSwitchDocTest.java
Normal file
|
|
@ -0,0 +1,161 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.Keep;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.time.Duration;
|
||||
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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("GraphDSLDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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(Duration.ofSeconds(1), 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(system);
|
||||
|
||||
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(Duration.ofSeconds(1), 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(system);
|
||||
|
||||
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(Duration.ofSeconds(1), 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(system);
|
||||
final CompletionStage<Integer> completionStageDelayed =
|
||||
countingSrc
|
||||
.delay(Duration.ofSeconds(1), DelayOverflowStrategy.backpressure())
|
||||
.viaMat(killSwitch.flow(), Keep.right())
|
||||
.toMat(lastSnk, Keep.right())
|
||||
.run(system);
|
||||
|
||||
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(Duration.ofSeconds(1), 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(system);
|
||||
final CompletionStage<Integer> completionStage2 =
|
||||
countingSrc
|
||||
.viaMat(killSwitch.flow(), Keep.right())
|
||||
.toMat(lastSnk, Keep.right())
|
||||
.run(system);
|
||||
|
||||
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() {}
|
||||
}
|
||||
16
docs/src/test/java/jdocs/stream/Main.java
Normal file
16
docs/src/test/java/jdocs/stream/Main.java
Normal file
|
|
@ -0,0 +1,16 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
|
||||
// #main-app
|
||||
public class Main {
|
||||
public static void main(String[] argv) {
|
||||
final ActorSystem system = ActorSystem.create("QuickStart");
|
||||
// Code here
|
||||
}
|
||||
}
|
||||
// #main-app
|
||||
39
docs/src/test/java/jdocs/stream/MigrationsJava.java
Normal file
39
docs/src/test/java/jdocs/stream/MigrationsJava.java
Normal file
|
|
@ -0,0 +1,39 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
// #asPublisher-import
|
||||
import static org.apache.pekko.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
|
||||
}
|
||||
}
|
||||
88
docs/src/test/java/jdocs/stream/QuickStartDocTest.java
Normal file
88
docs/src/test/java/jdocs/stream/QuickStartDocTest.java
Normal file
|
|
@ -0,0 +1,88 @@
|
|||
/*
|
||||
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
// #stream-imports
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
// #stream-imports
|
||||
|
||||
// #other-imports
|
||||
import org.apache.pekko.Done;
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.util.ByteString;
|
||||
|
||||
import java.nio.file.Paths;
|
||||
import java.math.BigInteger;
|
||||
import java.time.Duration;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
// #other-imports
|
||||
|
||||
import jdocs.AbstractJavaTest;
|
||||
|
||||
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 {
|
||||
final ActorSystem system = ActorSystem.create("QuickStart");
|
||||
|
||||
// #create-source
|
||||
final Source<Integer, NotUsed> source = Source.range(1, 100);
|
||||
// #create-source
|
||||
|
||||
// #run-source
|
||||
source.runForeach(i -> System.out.println(i), system);
|
||||
// #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")), system);
|
||||
// #transform-source
|
||||
|
||||
// #use-transformed-sink
|
||||
factorials.map(BigInteger::toString).runWith(lineSink("factorial2.txt"), system);
|
||||
// #use-transformed-sink
|
||||
|
||||
// #add-streams
|
||||
factorials
|
||||
.zipWith(Source.range(0, 99), (num, idx) -> String.format("%d! = %s", idx, num))
|
||||
.throttle(1, Duration.ofSeconds(1))
|
||||
// #add-streams
|
||||
.take(2)
|
||||
// #add-streams
|
||||
.runForeach(s -> System.out.println(s), system);
|
||||
// #add-streams
|
||||
|
||||
// #run-source-and-terminate
|
||||
final CompletionStage<Done> done = source.runForeach(i -> System.out.println(i), system);
|
||||
|
||||
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
|
||||
|
||||
}
|
||||
239
docs/src/test/java/jdocs/stream/RateTransformationDocTest.java
Normal file
239
docs/src/test/java/jdocs/stream/RateTransformationDocTest.java
Normal file
|
|
@ -0,0 +1,239 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.japi.tuple.Tuple3;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.stream.javadsl.Keep;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.stream.testkit.TestPublisher;
|
||||
import org.apache.pekko.stream.testkit.TestSubscriber;
|
||||
import org.apache.pekko.stream.testkit.javadsl.TestSink;
|
||||
import org.apache.pekko.stream.testkit.javadsl.TestSource;
|
||||
import org.apache.pekko.testkit.TestLatch;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import scala.concurrent.Await;
|
||||
import scala.concurrent.duration.Duration;
|
||||
import scala.util.Random;
|
||||
|
||||
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 static org.junit.Assert.assertEquals;
|
||||
|
||||
public class RateTransformationDocTest extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RateTransformationDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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(), system);
|
||||
|
||||
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), system);
|
||||
|
||||
final Double count = fut.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void extrapolateShouldRepeatLast() throws Exception {
|
||||
// #extrapolate-last
|
||||
final Flow<Double, Double, NotUsed> lastFlow =
|
||||
Flow.of(Double.class).extrapolate(in -> Stream.iterate(in, i -> i).iterator());
|
||||
// #extrapolate-last
|
||||
|
||||
final Pair<TestPublisher.Probe<Double>, CompletionStage<List<Double>>> probeFut =
|
||||
TestSource.<Double>probe(system)
|
||||
.via(lastFlow)
|
||||
.grouped(10)
|
||||
.toMat(Sink.head(), Keep.both())
|
||||
.run(system);
|
||||
|
||||
final TestPublisher.Probe<Double> probe = probeFut.first();
|
||||
final CompletionStage<List<Double>> fut = probeFut.second();
|
||||
probe.sendNext(1.0);
|
||||
final List<Double> extrapolated = fut.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
assertEquals(extrapolated.size(), 10);
|
||||
assertEquals(extrapolated.stream().mapToDouble(d -> d).sum(), 10, 0.1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void extrapolateShouldSeedFirst() throws Exception {
|
||||
// #extrapolate-seed
|
||||
Double initial = 2.0;
|
||||
final Flow<Double, Double, NotUsed> lastFlow =
|
||||
Flow.of(Double.class).extrapolate(in -> Stream.iterate(in, i -> i).iterator(), initial);
|
||||
// #extrapolate-seed
|
||||
|
||||
final CompletionStage<List<Double>> fut =
|
||||
TestSource.<Double>probe(system)
|
||||
.via(lastFlow)
|
||||
.grouped(10)
|
||||
.toMat(Sink.head(), Keep.right())
|
||||
.run(system);
|
||||
|
||||
final List<Double> extrapolated = fut.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
assertEquals(extrapolated.size(), 10);
|
||||
assertEquals(extrapolated.stream().mapToDouble(d -> d).sum(), 10 * initial, 0.1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void extrapolateShouldTrackDrift() throws Exception {
|
||||
@SuppressWarnings("unused")
|
||||
// #extrapolate-drift
|
||||
final Flow<Double, Pair<Double, Integer>, NotUsed> driftFlow =
|
||||
Flow.of(Double.class)
|
||||
.map(d -> new Pair<>(d, 0))
|
||||
.extrapolate(
|
||||
d -> Stream.iterate(1, i -> i + 1).map(i -> new Pair<>(d.first(), i)).iterator());
|
||||
// #extrapolate-drift
|
||||
final TestLatch latch = new TestLatch(2, system);
|
||||
final Flow<Double, Pair<Double, Integer>, NotUsed> realDriftFlow =
|
||||
Flow.of(Double.class)
|
||||
.map(
|
||||
d -> {
|
||||
latch.countDown();
|
||||
return new Pair<>(d, 0);
|
||||
})
|
||||
.extrapolate(
|
||||
d -> {
|
||||
latch.countDown();
|
||||
return Stream.iterate(1, i -> i + 1)
|
||||
.map(i -> new Pair<>(d.first(), 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(system);
|
||||
|
||||
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));
|
||||
}
|
||||
|
||||
@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(system);
|
||||
|
||||
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));
|
||||
}
|
||||
}
|
||||
256
docs/src/test/java/jdocs/stream/ReactiveStreamsDocTest.java
Normal file
256
docs/src/test/java/jdocs/stream/ReactiveStreamsDocTest.java
Normal file
|
|
@ -0,0 +1,256 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorRef;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.function.Creator;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.testkit.TestProbe;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import jdocs.stream.TwitterStreamQuickstartDocTest.Model.Author;
|
||||
import jdocs.stream.TwitterStreamQuickstartDocTest.Model.Tweet;
|
||||
import org.apache.pekko.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 TestProbe storageProbe;
|
||||
static TestProbe alertProbe;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("ReactiveStreamsDocTest");
|
||||
storageProbe = new TestProbe(system);
|
||||
alertProbe = new TestProbe(system);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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), system);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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).
|
||||
*
|
||||
* <p>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(system);
|
||||
|
||||
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), system);
|
||||
|
||||
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), system);
|
||||
|
||||
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(), system);
|
||||
|
||||
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(system);
|
||||
}
|
||||
};
|
||||
|
||||
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");
|
||||
}
|
||||
}
|
||||
103
docs/src/test/java/jdocs/stream/RestartDocTest.java
Normal file
103
docs/src/test/java/jdocs/stream/RestartDocTest.java
Normal file
|
|
@ -0,0 +1,103 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.KillSwitch;
|
||||
import org.apache.pekko.stream.KillSwitches;
|
||||
import org.apache.pekko.stream.Materializer;
|
||||
import org.apache.pekko.stream.RestartSettings;
|
||||
import org.apache.pekko.stream.javadsl.Keep;
|
||||
import org.apache.pekko.stream.javadsl.RestartSource;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
public class RestartDocTest {
|
||||
|
||||
static ActorSystem system;
|
||||
static Materializer materializer;
|
||||
|
||||
// Mocking akka-http
|
||||
public static class Http {
|
||||
public static Http get(ActorSystem system) {
|
||||
return new Http();
|
||||
}
|
||||
|
||||
public CompletionStage<Http> singleRequest(String uri) {
|
||||
return new CompletableFuture<>();
|
||||
}
|
||||
|
||||
public NotUsed entity() {
|
||||
return NotUsed.getInstance();
|
||||
}
|
||||
}
|
||||
|
||||
public static class HttpRequest {
|
||||
public static String create(String uri) {
|
||||
return uri;
|
||||
}
|
||||
}
|
||||
|
||||
public static class ServerSentEvent {}
|
||||
|
||||
public static class EventStreamUnmarshalling {
|
||||
public static EventStreamUnmarshalling fromEventStream() {
|
||||
return new EventStreamUnmarshalling();
|
||||
}
|
||||
|
||||
public CompletionStage<Source<ServerSentEvent, NotUsed>> unmarshall(
|
||||
Http http, Materializer mat) {
|
||||
return new CompletableFuture<>();
|
||||
}
|
||||
}
|
||||
|
||||
public void doSomethingElse() {}
|
||||
|
||||
public void recoverWithBackoffSource() {
|
||||
// #restart-with-backoff-source
|
||||
RestartSettings settings =
|
||||
RestartSettings.create(
|
||||
Duration.ofSeconds(3), // min backoff
|
||||
Duration.ofSeconds(30), // max backoff
|
||||
0.2 // adds 20% "noise" to vary the intervals slightly
|
||||
)
|
||||
.withMaxRestarts(
|
||||
20, Duration.ofMinutes(5)); // limits the amount of restarts to 20 within 5 minutes
|
||||
|
||||
Source<ServerSentEvent, NotUsed> eventStream =
|
||||
RestartSource.withBackoff(
|
||||
settings,
|
||||
() ->
|
||||
// Create a source from a future of a source
|
||||
Source.completionStageSource(
|
||||
// Issue a GET request on the event stream
|
||||
Http.get(system)
|
||||
.singleRequest(HttpRequest.create("http://example.com/eventstream"))
|
||||
.thenCompose(
|
||||
response ->
|
||||
// Unmarshall it to a stream of ServerSentEvents
|
||||
EventStreamUnmarshalling.fromEventStream()
|
||||
.unmarshall(response, materializer))));
|
||||
// #restart-with-backoff-source
|
||||
|
||||
// #with-kill-switch
|
||||
KillSwitch killSwitch =
|
||||
eventStream
|
||||
.viaMat(KillSwitches.single(), Keep.right())
|
||||
.toMat(Sink.foreach(event -> System.out.println("Got event: " + event)), Keep.left())
|
||||
.run(materializer);
|
||||
|
||||
doSomethingElse();
|
||||
|
||||
killSwitch.shutdown();
|
||||
// #with-kill-switch
|
||||
|
||||
}
|
||||
}
|
||||
68
docs/src/test/java/jdocs/stream/SilenceSystemOut.java
Normal file
68
docs/src/test/java/jdocs/stream/SilenceSystemOut.java
Normal file
|
|
@ -0,0 +1,68 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.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 polluting 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 abstract static 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));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
40
docs/src/test/java/jdocs/stream/SinkRecipeDocTest.java
Normal file
40
docs/src/test/java/jdocs/stream/SinkRecipeDocTest.java
Normal file
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.function.Function;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
public class SinkRecipeDocTest extends AbstractJavaTest {
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("SinkRecipeDocTest");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void foreachAsync() {
|
||||
final Function<Integer, CompletionStage<Void>> asyncProcessing =
|
||||
param -> CompletableFuture.completedFuture(param).thenAccept(System.out::println);
|
||||
|
||||
// #forseachAsync-processing
|
||||
// final Function<Integer, CompletionStage<Void>> asyncProcessing = _
|
||||
|
||||
final Source<Integer, NotUsed> numberSource = Source.range(1, 100);
|
||||
|
||||
numberSource.runWith(Sink.foreachAsync(10, asyncProcessing), system);
|
||||
// #forseachAsync-processing
|
||||
}
|
||||
}
|
||||
142
docs/src/test/java/jdocs/stream/StreamBuffersRateDocTest.java
Normal file
142
docs/src/test/java/jdocs/stream/StreamBuffersRateDocTest.java
Normal file
|
|
@ -0,0 +1,142 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.actor.Cancellable;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.Arrays;
|
||||
|
||||
public class StreamBuffersRateDocTest extends AbstractJavaTest {
|
||||
|
||||
static class Job {}
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("StreamBuffersDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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(), system);
|
||||
// #pipelining
|
||||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("unused")
|
||||
public void demonstrateBufferSizes() {
|
||||
// #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 value from the surrounding graph it is
|
||||
// used in
|
||||
final RunnableGraph<NotUsed> runnableGraph =
|
||||
Source.range(1, 10).via(flow1).to(Sink.foreach(elem -> System.out.println(elem)));
|
||||
|
||||
final RunnableGraph<NotUsed> withOverridenDefaults =
|
||||
runnableGraph.withAttributes(Attributes.inputBuffer(64, 64));
|
||||
// #section-buffer
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateBufferAbstractionLeak() {
|
||||
// #buffering-abstraction-leak
|
||||
final Duration oneSecond = Duration.ofSeconds(1);
|
||||
final Source<String, Cancellable> msgSource = Source.tick(oneSecond, oneSecond, "message!");
|
||||
final Source<String, Cancellable> tickSource =
|
||||
Source.tick(oneSecond.multipliedBy(3), oneSecond.multipliedBy(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(system);
|
||||
// #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
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,190 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.Done;
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorRef;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.apache.pekko.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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("StreamPartialGraphDSLDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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 UniformFanInShape.<Integer, Integer>create(
|
||||
zip2.out(), Arrays.asList(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(system);
|
||||
// #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(), system);
|
||||
// #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(), system);
|
||||
// #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), system);
|
||||
// #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, system);
|
||||
// #sink-combine
|
||||
probe.expectMsgEquals(0);
|
||||
probe.expectMsgEquals(1);
|
||||
probe.expectMsgEquals(2);
|
||||
}
|
||||
}
|
||||
236
docs/src/test/java/jdocs/stream/StreamTestKitDocTest.java
Normal file
236
docs/src/test/java/jdocs/stream/StreamTestKitDocTest.java
Normal file
|
|
@ -0,0 +1,236 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.pekko.Done;
|
||||
import org.apache.pekko.NotUsed;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.junit.*;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import org.apache.pekko.actor.*;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.stream.testkit.*;
|
||||
import org.apache.pekko.stream.testkit.javadsl.*;
|
||||
|
||||
public class StreamTestKitDocTest extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("StreamTestKitDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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, system);
|
||||
final Integer result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(20, result.intValue());
|
||||
// #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(), system);
|
||||
final List<Integer> result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(Collections.nCopies(10, 2), result);
|
||||
// #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), system);
|
||||
final Integer result = future.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
assertEquals(10, result.intValue());
|
||||
// #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 TestKit probe = new TestKit(system);
|
||||
final CompletionStage<List<List<Integer>>> future =
|
||||
sourceUnderTest.grouped(2).runWith(Sink.head(), system);
|
||||
org.apache.pekko.pattern.Patterns.pipe(future, system.dispatcher()).to(probe.getRef());
|
||||
probe.expectMsg(Duration.ofSeconds(3), 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(Duration.ZERO, Duration.ofMillis(200), Tick.TOCK);
|
||||
|
||||
final TestKit probe = new TestKit(system);
|
||||
final Cancellable cancellable =
|
||||
sourceUnderTest.to(Sink.actorRef(probe.getRef(), Tick.COMPLETED)).run(system);
|
||||
probe.expectMsg(Duration.ofSeconds(3), Tick.TOCK);
|
||||
probe.expectNoMessage(Duration.ofMillis(100));
|
||||
probe.expectMsg(Duration.ofSeconds(3), Tick.TOCK);
|
||||
cancellable.cancel();
|
||||
probe.expectMsg(Duration.ofSeconds(3), 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(
|
||||
elem -> {
|
||||
// complete stream immediately if we send it Done
|
||||
if (elem == Done.done()) return Optional.of(CompletionStrategy.immediately());
|
||||
else return Optional.empty();
|
||||
},
|
||||
// never fail the stream because of a message
|
||||
elem -> Optional.empty(),
|
||||
8,
|
||||
OverflowStrategy.fail())
|
||||
.toMat(sinkUnderTest, Keep.both())
|
||||
.run(system);
|
||||
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(Done.getInstance(), ActorRef.noSender());
|
||||
|
||||
final String result = future.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
assertEquals("123", result);
|
||||
// #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), system)
|
||||
.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(system)
|
||||
.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(system);
|
||||
final TestPublisher.Probe<Integer> probe = probeAndCompletionStage.first();
|
||||
final CompletionStage<Integer> future = probeAndCompletionStage.second();
|
||||
probe.sendError(new Exception("boom"));
|
||||
|
||||
ExecutionException exception =
|
||||
Assert.assertThrows(
|
||||
ExecutionException.class, () -> future.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
assertEquals("boom", exception.getCause().getMessage());
|
||||
// #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 ->
|
||||
org.apache.pekko.pattern.Patterns.after(
|
||||
Duration.ofMillis(10),
|
||||
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(system);
|
||||
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();
|
||||
assertTrue(ex.getMessage().contains("C-47"));
|
||||
// #test-source-and-sink
|
||||
}
|
||||
}
|
||||
104
docs/src/test/java/jdocs/stream/SubstreamDocTest.java
Normal file
104
docs/src/test/java/jdocs/stream/SubstreamDocTest.java
Normal file
|
|
@ -0,0 +1,104 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class SubstreamDocTest extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("FlowDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateGroupBy() throws Exception {
|
||||
// #groupBy1
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)).groupBy(3, elem -> elem % 3);
|
||||
// #groupBy1
|
||||
|
||||
// #groupBy2
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))
|
||||
.groupBy(3, elem -> elem % 3)
|
||||
.to(Sink.ignore())
|
||||
.run(system);
|
||||
// #groupBy2
|
||||
|
||||
// #groupBy3
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))
|
||||
.groupBy(3, elem -> elem % 3)
|
||||
.mergeSubstreams()
|
||||
.runWith(Sink.ignore(), system);
|
||||
// #groupBy3
|
||||
|
||||
// #groupBy4
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))
|
||||
.groupBy(3, elem -> elem % 3)
|
||||
.mergeSubstreamsWithParallelism(2)
|
||||
.runWith(Sink.ignore(), system);
|
||||
// concatSubstreams is equivalent to mergeSubstreamsWithParallelism(1)
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))
|
||||
.groupBy(3, elem -> elem % 3)
|
||||
.concatSubstreams()
|
||||
.runWith(Sink.ignore(), system);
|
||||
// #groupBy4
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateSplitWhenAfter() throws Exception {
|
||||
// #splitWhenAfter
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)).splitWhen(elem -> elem == 3);
|
||||
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)).splitAfter(elem -> elem == 3);
|
||||
// #splitWhenAfter
|
||||
|
||||
// #wordCount
|
||||
String text =
|
||||
"This is the first line.\n" + "The second line.\n" + "There is also the 3rd line\n";
|
||||
|
||||
Source.from(Arrays.asList(text.split("")))
|
||||
.map(x -> x.charAt(0))
|
||||
.splitAfter(x -> x == '\n')
|
||||
.filter(x -> x != '\n')
|
||||
.map(x -> 1)
|
||||
.reduce((x, y) -> x + y)
|
||||
.to(Sink.foreach(x -> System.out.println(x)))
|
||||
.run(system);
|
||||
// #wordCount
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateflatMapConcatMerge() throws Exception {
|
||||
// #flatMapConcat
|
||||
Source.from(Arrays.asList(1, 2))
|
||||
.flatMapConcat(i -> Source.from(Arrays.asList(i, i, i)))
|
||||
.runWith(Sink.ignore(), system);
|
||||
// #flatMapConcat
|
||||
|
||||
// #flatMapMerge
|
||||
Source.from(Arrays.asList(1, 2))
|
||||
.flatMapMerge(2, i -> Source.from(Arrays.asList(i, i, i)))
|
||||
.runWith(Sink.ignore(), system);
|
||||
// #flatMapMerge
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,373 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream;
|
||||
|
||||
import org.apache.pekko.Done;
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.JavaPartialFunction;
|
||||
// #imports
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.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 org.apache.pekko.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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("TwitterStreamQuickstartDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
}
|
||||
|
||||
abstract static 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!")
|
||||
}));
|
||||
}
|
||||
|
||||
abstract static class Example0 {
|
||||
// #tweet-source
|
||||
Source<Tweet, NotUsed> tweets;
|
||||
// #tweet-source
|
||||
}
|
||||
|
||||
abstract static class Example1 {
|
||||
// #first-sample
|
||||
// #system-setup
|
||||
final ActorSystem system = ActorSystem.create("reactive-tweets");
|
||||
// #first-sample
|
||||
// #system-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)), system);
|
||||
// #first-sample
|
||||
// #authors-foreachsink-println
|
||||
|
||||
// #authors-foreach-println
|
||||
authors.runForeach(a -> System.out.println(a), system);
|
||||
// #authors-foreach-println
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateMapConcat() {
|
||||
// #hashtags-mapConcat
|
||||
final Source<Hashtag, NotUsed> hashtags =
|
||||
tweets.mapConcat(t -> new ArrayList<Hashtag>(t.hashtags()));
|
||||
// #hashtags-mapConcat
|
||||
}
|
||||
|
||||
abstract static 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(system);
|
||||
// #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(), system);
|
||||
// #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(system);
|
||||
|
||||
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, system);
|
||||
// #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(system);
|
||||
// and once in the evening, reusing the blueprint
|
||||
final CompletionStage<Integer> eveningTweetsCount = counterRunnableGraph.run(system);
|
||||
// #tweets-runnable-flow-materialized-twice
|
||||
|
||||
}
|
||||
}
|
||||
107
docs/src/test/java/jdocs/stream/io/StreamFileDocTest.java
Normal file
107
docs/src/test/java/jdocs/stream/io/StreamFileDocTest.java
Normal file
|
|
@ -0,0 +1,107 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://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 org.apache.pekko.Done;
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.ActorAttributes;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.FileIO;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import jdocs.stream.SilenceSystemOut;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.util.ByteString;
|
||||
|
||||
public class StreamFileDocTest extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("StreamFileDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
}
|
||||
|
||||
final SilenceSystemOut.System System = SilenceSystemOut.get();
|
||||
|
||||
{
|
||||
// Using 4 spaces here to align with code in try block below.
|
||||
// #file-source
|
||||
final Path file = Paths.get("example.csv");
|
||||
// #file-source
|
||||
}
|
||||
|
||||
{
|
||||
// #file-sink
|
||||
final Path file = Paths.get("greeting.txt");
|
||||
// #file-sink
|
||||
}
|
||||
|
||||
@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(system);
|
||||
// #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);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demontrateFileIOWriting() throws IOException {
|
||||
final Path file = Files.createTempFile(getClass().getName(), ".tmp");
|
||||
|
||||
try {
|
||||
// #file-sink
|
||||
Sink<ByteString, CompletionStage<IOResult>> fileSink = FileIO.toPath(file);
|
||||
Source<String, NotUsed> textSource = Source.single("Hello Akka Stream!");
|
||||
|
||||
CompletionStage<IOResult> ioResult =
|
||||
textSource.map(ByteString::fromString).runWith(fileSink, system);
|
||||
// #file-sink
|
||||
} finally {
|
||||
Files.delete(file);
|
||||
}
|
||||
}
|
||||
}
|
||||
188
docs/src/test/java/jdocs/stream/io/StreamTcpDocTest.java
Normal file
188
docs/src/test/java/jdocs/stream/io/StreamTcpDocTest.java
Normal file
|
|
@ -0,0 +1,188 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.io;
|
||||
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.stream.javadsl.Framing;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import jdocs.stream.SilenceSystemOut;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.stream.javadsl.Tcp.*;
|
||||
import org.apache.pekko.testkit.SocketUtil;
|
||||
import org.apache.pekko.testkit.TestProbe;
|
||||
import org.apache.pekko.util.ByteString;
|
||||
|
||||
public class StreamTcpDocTest extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("StreamTcpDocTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
}
|
||||
|
||||
final SilenceSystemOut.System System = SilenceSystemOut.get();
|
||||
|
||||
private final ConcurrentLinkedQueue<String> input = new ConcurrentLinkedQueue<>();
|
||||
|
||||
{
|
||||
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", 8888);
|
||||
// #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, system);
|
||||
},
|
||||
system);
|
||||
// #echo-server-simple-handle
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void actuallyWorkingClientServerApp() throws Exception {
|
||||
|
||||
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());
|
||||
final CompletionStage<ServerBinding> bindingCS =
|
||||
// #welcome-banner-chat-server
|
||||
connections
|
||||
.to(
|
||||
Sink.foreach(
|
||||
(IncomingConnection 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, system);
|
||||
}))
|
||||
.run(system);
|
||||
// #welcome-banner-chat-server
|
||||
|
||||
// make sure server is bound before we do anything else
|
||||
bindingCS.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
|
||||
{
|
||||
// just for docs, never actually used
|
||||
// #repl-client
|
||||
final Flow<ByteString, ByteString, CompletionStage<OutgoingConnection>> connection =
|
||||
Tcp.get(system).outgoingConnection("127.0.0.1", 8888);
|
||||
// #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);
|
||||
|
||||
CompletionStage<OutgoingConnection> connectionCS = connection.join(repl).run(system);
|
||||
// #repl-client
|
||||
|
||||
// make sure it got connected (or fails the test)
|
||||
connectionCS.toCompletableFuture().get(5L, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
serverProbe.expectMsg("Hello world");
|
||||
serverProbe.expectMsg("What a lovely day");
|
||||
serverProbe.expectMsg("BYE");
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,232 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.Done;
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.dispatch.Futures;
|
||||
import org.apache.pekko.japi.pf.PFBuilder;
|
||||
import org.apache.pekko.stream.javadsl.Keep;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.stream.testkit.TestSubscriber;
|
||||
import org.apache.pekko.stream.testkit.javadsl.TestSink;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import scala.concurrent.Await;
|
||||
import scala.concurrent.Promise;
|
||||
import scala.concurrent.duration.FiniteDuration;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
public class RecipeAdhocSourceTest extends RecipeTest {
|
||||
static ActorSystem system;
|
||||
Duration duration200mills = Duration.ofMillis(200);
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeAdhocSource");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
}
|
||||
|
||||
// #adhoc-source
|
||||
public <T> Source<T, ?> adhocSource(Source<T, ?> source, Duration timeout, int maxRetries) {
|
||||
return Source.lazySource(
|
||||
() ->
|
||||
source
|
||||
.backpressureTimeout(timeout)
|
||||
.recoverWithRetries(
|
||||
maxRetries,
|
||||
new PFBuilder<Throwable, Source<T, NotUsed>>()
|
||||
.match(
|
||||
TimeoutException.class,
|
||||
ex ->
|
||||
Source.lazySource(() -> source.backpressureTimeout(timeout))
|
||||
.mapMaterializedValue(v -> NotUsed.getInstance()))
|
||||
.build()));
|
||||
}
|
||||
// #adhoc-source
|
||||
|
||||
@Test
|
||||
@Ignore
|
||||
public void noStart() throws Exception {
|
||||
new TestKit(system) {
|
||||
{
|
||||
AtomicBoolean isStarted = new AtomicBoolean();
|
||||
adhocSource(
|
||||
Source.empty()
|
||||
.mapMaterializedValue(
|
||||
x -> {
|
||||
isStarted.set(true);
|
||||
return x;
|
||||
}),
|
||||
duration200mills,
|
||||
3);
|
||||
Thread.sleep(300);
|
||||
assertEquals(false, isStarted.get());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Test
|
||||
@Ignore
|
||||
public void startStream() throws Exception {
|
||||
new TestKit(system) {
|
||||
{
|
||||
TestSubscriber.Probe<String> probe =
|
||||
adhocSource(Source.repeat("a"), duration200mills, 3)
|
||||
.toMat(TestSink.probe(system), Keep.right())
|
||||
.run(system);
|
||||
probe.requestNext("a");
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Test
|
||||
@Ignore
|
||||
public void shutdownStream() throws Exception {
|
||||
new TestKit(system) {
|
||||
{
|
||||
Promise<Done> shutdown = Futures.promise();
|
||||
TestSubscriber.Probe<String> probe =
|
||||
adhocSource(
|
||||
Source.repeat("a")
|
||||
.watchTermination(
|
||||
(a, term) -> term.thenRun(() -> shutdown.success(Done.getInstance()))),
|
||||
duration200mills,
|
||||
3)
|
||||
.toMat(TestSink.probe(system), Keep.right())
|
||||
.run(system);
|
||||
|
||||
probe.requestNext("a");
|
||||
Thread.sleep(300);
|
||||
Await.result(shutdown.future(), duration("3 seconds"));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Test
|
||||
@Ignore
|
||||
public void notShutDownStream() throws Exception {
|
||||
new TestKit(system) {
|
||||
{
|
||||
Promise<Done> shutdown = Futures.promise();
|
||||
TestSubscriber.Probe<String> probe =
|
||||
adhocSource(
|
||||
Source.repeat("a")
|
||||
.watchTermination(
|
||||
(a, term) -> term.thenRun(() -> shutdown.success(Done.getInstance()))),
|
||||
duration200mills,
|
||||
3)
|
||||
.toMat(TestSink.probe(system), Keep.right())
|
||||
.run(system);
|
||||
|
||||
probe.requestNext("a");
|
||||
Thread.sleep(100);
|
||||
probe.requestNext("a");
|
||||
Thread.sleep(100);
|
||||
probe.requestNext("a");
|
||||
Thread.sleep(100);
|
||||
probe.requestNext("a");
|
||||
Thread.sleep(100);
|
||||
probe.requestNext("a");
|
||||
Thread.sleep(100);
|
||||
|
||||
assertEquals(false, shutdown.isCompleted());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Test
|
||||
@Ignore
|
||||
public void restartUponDemand() throws Exception {
|
||||
new TestKit(system) {
|
||||
{
|
||||
Promise<Done> shutdown = Futures.promise();
|
||||
AtomicInteger startedCount = new AtomicInteger(0);
|
||||
|
||||
Source<String, ?> source =
|
||||
Source.<String>empty()
|
||||
.mapMaterializedValue(x -> startedCount.incrementAndGet())
|
||||
.concat(Source.repeat("a"));
|
||||
|
||||
TestSubscriber.Probe<String> probe =
|
||||
adhocSource(
|
||||
source.watchTermination(
|
||||
(a, term) -> term.thenRun(() -> shutdown.success(Done.getInstance()))),
|
||||
duration200mills,
|
||||
3)
|
||||
.toMat(TestSink.probe(system), Keep.right())
|
||||
.run(system);
|
||||
|
||||
probe.requestNext("a");
|
||||
assertEquals(1, startedCount.get());
|
||||
Thread.sleep(200);
|
||||
Await.result(shutdown.future(), duration("3 seconds"));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Test
|
||||
@Ignore
|
||||
public void restartUptoMaxRetries() throws Exception {
|
||||
new TestKit(system) {
|
||||
{
|
||||
Promise<Done> shutdown = Futures.promise();
|
||||
AtomicInteger startedCount = new AtomicInteger(0);
|
||||
|
||||
Source<String, ?> source =
|
||||
Source.<String>empty()
|
||||
.mapMaterializedValue(x -> startedCount.incrementAndGet())
|
||||
.concat(Source.repeat("a"));
|
||||
|
||||
TestSubscriber.Probe<String> probe =
|
||||
adhocSource(
|
||||
source.watchTermination(
|
||||
(a, term) -> term.thenRun(() -> shutdown.success(Done.getInstance()))),
|
||||
duration200mills,
|
||||
3)
|
||||
.toMat(TestSink.probe(system), Keep.right())
|
||||
.run(system);
|
||||
|
||||
probe.requestNext("a");
|
||||
assertEquals(1, startedCount.get());
|
||||
|
||||
Thread.sleep(500);
|
||||
assertEquals(true, shutdown.isCompleted());
|
||||
|
||||
Thread.sleep(500);
|
||||
probe.requestNext("a");
|
||||
assertEquals(2, startedCount.get());
|
||||
|
||||
Thread.sleep(500);
|
||||
probe.requestNext("a");
|
||||
assertEquals(3, startedCount.get());
|
||||
|
||||
Thread.sleep(500);
|
||||
probe.requestNext("a");
|
||||
assertEquals(4, startedCount.get()); // startCount == 4, which means "re"-tried 3 times
|
||||
|
||||
Thread.sleep(500);
|
||||
assertEquals(TimeoutException.class, probe.expectError().getClass());
|
||||
probe.request(1); // send demand
|
||||
probe.expectNoMessage(FiniteDuration.create(200, "milliseconds")); // but no more restart
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,297 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.stream.stage.*;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.apache.pekko.util.ByteString;
|
||||
import static org.apache.pekko.util.ByteString.emptyByteString;
|
||||
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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeByteStrings");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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 = emptyByteString();
|
||||
|
||||
{
|
||||
setHandler(
|
||||
out,
|
||||
new AbstractOutHandler() {
|
||||
@Override
|
||||
public void onPull() throws Exception {
|
||||
emitChunk();
|
||||
}
|
||||
});
|
||||
|
||||
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(), system);
|
||||
|
||||
List<ByteString> chunks = chunksFuture.toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
|
||||
for (ByteString chunk : chunks) {
|
||||
assertTrue(chunk.size() <= 2);
|
||||
}
|
||||
|
||||
ByteString sum = emptyByteString();
|
||||
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(), system)
|
||||
.toCompletableFuture()
|
||||
.get(3, TimeUnit.SECONDS);
|
||||
ByteString acc = emptyByteString();
|
||||
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(), system)
|
||||
.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(), system)
|
||||
.toCompletableFuture()
|
||||
.get(3, TimeUnit.SECONDS);
|
||||
|
||||
for (ByteString byteString : got) {
|
||||
assertTrue(byteString.isCompact());
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,56 @@
|
|||
/*
|
||||
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Compression;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.apache.pekko.util.ByteString;
|
||||
import static org.apache.pekko.util.ByteString.emptyByteString;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class RecipeDecompress extends RecipeTest {
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeDecompress");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void parseLines() throws Exception {
|
||||
final Source<ByteString, NotUsed> dataStream =
|
||||
Source.single(ByteString.fromString("Hello World"));
|
||||
|
||||
final Source<ByteString, NotUsed> compressedStream = dataStream.via(Compression.gzip());
|
||||
|
||||
// #decompress-gzip
|
||||
final Source<ByteString, NotUsed> decompressedStream =
|
||||
compressedStream.via(Compression.gunzip(100));
|
||||
// #decompress-gzip
|
||||
|
||||
ByteString decompressedData =
|
||||
decompressedStream
|
||||
.runFold(emptyByteString(), ByteString::concat, system)
|
||||
.toCompletableFuture()
|
||||
.get(1, TimeUnit.SECONDS);
|
||||
String decompressedString = decompressedData.utf8String();
|
||||
Assert.assertEquals("Hello World", decompressedString);
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,122 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.stream.stage.*;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.apache.pekko.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.concurrent.TimeUnit;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
public class RecipeDigest extends RecipeTest {
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeDigest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
}
|
||||
|
||||
// #calculating-digest
|
||||
class DigestCalculator extends GraphStage<FlowShape<ByteString, ByteString>> {
|
||||
private final String algorithm;
|
||||
public Inlet<ByteString> in = Inlet.create("DigestCalculator.in");
|
||||
public Outlet<ByteString> out = Outlet.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() {
|
||||
pull(in);
|
||||
}
|
||||
});
|
||||
|
||||
setHandler(
|
||||
in,
|
||||
new AbstractInHandler() {
|
||||
@Override
|
||||
public void onPush() {
|
||||
ByteString chunk = grab(in);
|
||||
digest.update(chunk.toArray());
|
||||
pull(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onUpstreamFinish() {
|
||||
// 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.single(ByteString.fromString("abc"));
|
||||
|
||||
// #calculating-digest2
|
||||
final Source<ByteString, NotUsed> digest = data.via(new DigestCalculator("SHA-256"));
|
||||
// #calculating-digest2
|
||||
|
||||
ByteString got =
|
||||
digest.runWith(Sink.head(), system).toCompletableFuture().get(3, TimeUnit.SECONDS);
|
||||
|
||||
assertEquals(
|
||||
ByteString.fromInts(
|
||||
0xba, 0x78, 0x16, 0xbf, 0x8f, 0x01, 0xcf, 0xea, 0x41, 0x41, 0x40, 0xde, 0x5d, 0xae,
|
||||
0x22, 0x23, 0xb0, 0x03, 0x61, 0xa3, 0x96, 0x17, 0x7a, 0x9c, 0xb4, 0x10, 0xff, 0x61,
|
||||
0xf2, 0x00, 0x15, 0xad),
|
||||
got);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,75 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.Done;
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeDroppyBroadcast");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,63 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeFlattenList");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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(), system)
|
||||
.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"));
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,255 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.*;
|
||||
import org.apache.pekko.pattern.Patterns;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.stream.testkit.TestSubscriber;
|
||||
import org.apache.pekko.stream.testkit.javadsl.TestSink;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
import static junit.framework.TestCase.assertTrue;
|
||||
|
||||
public class RecipeGlobalRateLimit extends RecipeTest {
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeGlobalRateLimit");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
}
|
||||
|
||||
public
|
||||
// #global-limiter-actor
|
||||
static 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 Duration 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, Duration tokenRefreshPeriod, int tokenRefreshAmount) {
|
||||
return Props.create(
|
||||
Limiter.class, maxAvailableTokens, tokenRefreshPeriod, tokenRefreshAmount);
|
||||
}
|
||||
|
||||
private Limiter(int maxAvailableTokens, Duration tokenRefreshPeriod, int tokenRefreshAmount) {
|
||||
this.maxAvailableTokens = maxAvailableTokens;
|
||||
this.tokenRefreshPeriod = tokenRefreshPeriod;
|
||||
this.tokenRefreshAmount = tokenRefreshAmount;
|
||||
this.permitTokens = maxAvailableTokens;
|
||||
|
||||
this.replenishTimer =
|
||||
system
|
||||
.scheduler()
|
||||
.scheduleWithFixedDelay(
|
||||
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, Duration maxAllowedWait) {
|
||||
final int parallelism = 4;
|
||||
final Flow<T, T, NotUsed> f = Flow.create();
|
||||
|
||||
return f.mapAsync(
|
||||
parallelism,
|
||||
element -> {
|
||||
final CompletionStage<Object> limiterTriggerFuture =
|
||||
Patterns.ask(limiter, Limiter.WANT_TO_PASS, maxAllowedWait);
|
||||
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, Duration.ofDays(100), 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 Duration twoSeconds = dilated(Duration.ofSeconds(2));
|
||||
|
||||
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(system);
|
||||
|
||||
probe.expectSubscription().request(1000);
|
||||
|
||||
Duration fiveHundredMillis = Duration.ofMillis(500);
|
||||
|
||||
assertTrue(probe.expectNext().startsWith("E"));
|
||||
assertTrue(probe.expectNext().startsWith("E"));
|
||||
probe.expectNoMessage(fiveHundredMillis);
|
||||
|
||||
limiter.tell(Limiter.REPLENISH_TOKENS, getTestActor());
|
||||
assertTrue(probe.expectNext().startsWith("E"));
|
||||
probe.expectNoMessage(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();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
202
docs/src/test/java/jdocs/stream/javadsl/cookbook/RecipeHold.java
Normal file
202
docs/src/test/java/jdocs/stream/javadsl/cookbook/RecipeHold.java
Normal file
|
|
@ -0,0 +1,202 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.Keep;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.stream.stage.*;
|
||||
import org.apache.pekko.stream.testkit.TestPublisher;
|
||||
import org.apache.pekko.stream.testkit.TestSubscriber;
|
||||
import org.apache.pekko.stream.testkit.javadsl.TestSink;
|
||||
import org.apache.pekko.stream.testkit.javadsl.TestSource;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeHold");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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(system);
|
||||
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(system);
|
||||
TestPublisher.Probe<Integer> pub = pubSub.first();
|
||||
TestSubscriber.Probe<Integer> sub = pubSub.second();
|
||||
|
||||
FiniteDuration timeout = FiniteDuration.create(200, TimeUnit.MILLISECONDS);
|
||||
|
||||
sub.request(1);
|
||||
sub.expectNoMessage(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();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,53 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.apache.pekko.util.ByteString;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.time.Duration;
|
||||
|
||||
public class RecipeKeepAlive extends RecipeTest {
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeKeepAlive");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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(Duration.ofSeconds(1), () -> keepAliveMessage);
|
||||
// #inject-keepalive
|
||||
// @formatter:on
|
||||
|
||||
// Enough to compile, tested elsewhere as a built-in stage
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,115 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.event.Logging;
|
||||
import org.apache.pekko.event.LoggingAdapter;
|
||||
import org.apache.pekko.stream.Attributes;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.testkit.DebugFilter;
|
||||
import org.apache.pekko.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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system =
|
||||
ActorSystem.create(
|
||||
"RecipeLoggingElements",
|
||||
ConfigFactory.parseString(
|
||||
"akka.loglevel=DEBUG\nakka.loggers = [org.apache.pekko.testkit.TestEventListener]"));
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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"));
|
||||
|
||||
// #log-custom
|
||||
// customise log levels
|
||||
mySource
|
||||
.log("before-map")
|
||||
.withAttributes(
|
||||
Attributes.createLogLevels(
|
||||
Logging.WarningLevel(), // onElement
|
||||
Logging.InfoLevel(), // onFinish
|
||||
Logging.DebugLevel() // 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(), system);
|
||||
return null;
|
||||
}
|
||||
},
|
||||
system);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void errorLog() throws Exception {
|
||||
new TestKit(system) {
|
||||
{
|
||||
// #log-error
|
||||
Source.from(Arrays.asList(-1, 0, 1))
|
||||
.map(x -> 1 / x) // throwing ArithmeticException: / by zero
|
||||
.log("error logging")
|
||||
.runWith(Sink.ignore(), system);
|
||||
// #log-error
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,154 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.stream.testkit.TestPublisher;
|
||||
import org.apache.pekko.stream.testkit.TestSubscriber;
|
||||
import org.apache.pekko.stream.testkit.javadsl.TestSink;
|
||||
import org.apache.pekko.stream.testkit.javadsl.TestSource;
|
||||
import org.apache.pekko.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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeManualTrigger");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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(system);
|
||||
TestPublisher.Probe<Trigger> pub = pubSub.first();
|
||||
TestSubscriber.Probe<Message> sub = pubSub.second();
|
||||
|
||||
FiniteDuration timeout = FiniteDuration.create(100, TimeUnit.MILLISECONDS);
|
||||
sub.expectSubscription().request(1000);
|
||||
sub.expectNoMessage(timeout);
|
||||
|
||||
pub.sendNext(TRIGGER);
|
||||
sub.expectNext(new Message("1"));
|
||||
sub.expectNoMessage(timeout);
|
||||
|
||||
pub.sendNext(TRIGGER);
|
||||
pub.sendNext(TRIGGER);
|
||||
sub.expectNext(new Message("2"));
|
||||
sub.expectNext(new Message("3"));
|
||||
sub.expectNoMessage(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(system);
|
||||
TestPublisher.Probe<Trigger> pub = pubSub.first();
|
||||
TestSubscriber.Probe<Message> sub = pubSub.second();
|
||||
|
||||
FiniteDuration timeout = FiniteDuration.create(100, TimeUnit.MILLISECONDS);
|
||||
sub.expectSubscription().request(1000);
|
||||
sub.expectNoMessage(timeout);
|
||||
|
||||
pub.sendNext(TRIGGER);
|
||||
sub.expectNext(new Message("1"));
|
||||
sub.expectNoMessage(timeout);
|
||||
|
||||
pub.sendNext(TRIGGER);
|
||||
pub.sendNext(TRIGGER);
|
||||
sub.expectNext(new Message("2"));
|
||||
sub.expectNext(new Message("3"));
|
||||
sub.expectNoMessage(timeout);
|
||||
|
||||
pub.sendNext(TRIGGER);
|
||||
sub.expectNext(new Message("4"));
|
||||
sub.expectComplete();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.stream.javadsl.Keep;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.stream.testkit.TestPublisher;
|
||||
import org.apache.pekko.stream.testkit.TestSubscriber;
|
||||
import org.apache.pekko.stream.testkit.javadsl.TestSink;
|
||||
import org.apache.pekko.stream.testkit.javadsl.TestSource;
|
||||
import org.apache.pekko.testkit.TestLatch;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import scala.concurrent.Await;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class RecipeMissedTicks extends RecipeTest {
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeMissedTicks");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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(system);
|
||||
TestPublisher.Probe<Tick> pub = pubSub.first();
|
||||
TestSubscriber.Probe<Integer> sub = pubSub.second();
|
||||
|
||||
pub.sendNext(Tick);
|
||||
pub.sendNext(Tick);
|
||||
pub.sendNext(Tick);
|
||||
pub.sendNext(Tick);
|
||||
|
||||
scala.concurrent.duration.FiniteDuration timeout =
|
||||
scala.concurrent.duration.FiniteDuration.create(200, TimeUnit.MILLISECONDS);
|
||||
|
||||
Await.ready(latch, scala.concurrent.duration.Duration.create(1, TimeUnit.SECONDS));
|
||||
|
||||
sub.request(1);
|
||||
sub.expectNext(3);
|
||||
sub.request(1);
|
||||
sub.expectNoMessage(timeout);
|
||||
|
||||
pub.sendNext(Tick);
|
||||
sub.expectNext(0);
|
||||
|
||||
pub.sendComplete();
|
||||
sub.request(1);
|
||||
sub.expectComplete();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,155 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Function;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.stream.javadsl.SubSource;
|
||||
import org.apache.pekko.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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeMultiGroupBy");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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(), system);
|
||||
|
||||
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();
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,56 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Framing;
|
||||
import org.apache.pekko.stream.javadsl.FramingTruncation;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.apache.pekko.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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeParseLines");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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(), system).toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,127 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.japi.function.Function;
|
||||
import org.apache.pekko.japi.function.Function2;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeReduceByKey");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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(), system);
|
||||
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
|
||||
public static <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(), system);
|
||||
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);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,86 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeLoggingElements");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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(), system);
|
||||
// #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(), system);
|
||||
// #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(), system);
|
||||
// #draining-to-list-safe
|
||||
|
||||
strings.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,79 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.stream.testkit.TestPublisher;
|
||||
import org.apache.pekko.stream.testkit.TestSubscriber;
|
||||
import org.apache.pekko.stream.testkit.javadsl.TestSink;
|
||||
import org.apache.pekko.stream.testkit.javadsl.TestSource;
|
||||
import org.apache.pekko.testkit.TestLatch;
|
||||
import org.apache.pekko.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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeSimpleDrop");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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(system);
|
||||
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();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,64 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import com.typesafe.config.ConfigFactory;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class RecipeSourceFromFunction extends RecipeTest {
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system =
|
||||
ActorSystem.create(
|
||||
"RecipeSourceFromFunction",
|
||||
ConfigFactory.parseString(
|
||||
"akka.loglevel=DEBUG\nakka.loggers = [org.apache.pekko.testkit.TestEventListener]"));
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void beMappingOfRepeat() throws Exception {
|
||||
new TestKit(system) {
|
||||
final String builderFunction() {
|
||||
return UUID.randomUUID().toString();
|
||||
}
|
||||
|
||||
{
|
||||
// #source-from-function
|
||||
final Source<String, NotUsed> source =
|
||||
Source.repeat(NotUsed.getInstance()).map(elem -> builderFunction());
|
||||
// #source-from-function
|
||||
|
||||
final List<String> result =
|
||||
source
|
||||
.take(2)
|
||||
.runWith(Sink.seq(), system)
|
||||
.toCompletableFuture()
|
||||
.get(3, TimeUnit.SECONDS);
|
||||
|
||||
Assert.assertEquals(2, result.size());
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,80 @@
|
|||
/*
|
||||
* Copyright (C) since 2016 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.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.ExecutionException;
|
||||
|
||||
public class RecipeSplitter {
|
||||
private static ActorSystem system;
|
||||
|
||||
@Test
|
||||
public void simpleSplit() throws ExecutionException, InterruptedException {
|
||||
// #Simple-Split
|
||||
// Sample Source
|
||||
Source<String, NotUsed> source = Source.from(Arrays.asList("1-2-3", "2-3", "3-4"));
|
||||
|
||||
CompletionStage<List<Integer>> ret =
|
||||
source
|
||||
.map(s -> Arrays.asList(s.split("-")))
|
||||
.mapConcat(f -> f)
|
||||
// Sub-streams logic
|
||||
.map(s -> Integer.valueOf(s))
|
||||
.runWith(Sink.seq(), system);
|
||||
|
||||
// Verify results
|
||||
List<Integer> list = ret.toCompletableFuture().get();
|
||||
assert list.equals(Arrays.asList(1, 2, 3, 2, 3, 3, 4));
|
||||
// #Simple-Split
|
||||
}
|
||||
|
||||
@Test
|
||||
public void splitAggregate() throws ExecutionException, InterruptedException {
|
||||
// #Aggregate-Split
|
||||
// Sample Source
|
||||
Source<String, NotUsed> source = Source.from(Arrays.asList("1-2-3", "2-3", "3-4"));
|
||||
|
||||
CompletionStage<List<Integer>> ret =
|
||||
source
|
||||
.map(s -> Arrays.asList(s.split("-")))
|
||||
// split all messages into sub-streams
|
||||
.splitWhen(a -> true)
|
||||
// now split each collection
|
||||
.mapConcat(f -> f)
|
||||
// Sub-streams logic
|
||||
.map(s -> Integer.valueOf(s))
|
||||
// aggregate each sub-stream
|
||||
.reduce((a, b) -> a + b)
|
||||
// and merge back the result into the original stream
|
||||
.mergeSubstreams()
|
||||
.runWith(Sink.seq(), system);
|
||||
|
||||
// Verify results
|
||||
List<Integer> list = ret.toCompletableFuture().get();
|
||||
assert list.equals(Arrays.asList(6, 5, 7));
|
||||
// #Aggregate-Split
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() throws Exception {
|
||||
system = ActorSystem.create();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardown() throws Exception {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,85 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.javadsl.cookbook;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.*;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.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;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("RecipeWorkerPool");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = 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(), system);
|
||||
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"));
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,82 @@
|
|||
/*
|
||||
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
|
||||
// #import
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.japi.tuple.Tuple3;
|
||||
import org.apache.pekko.stream.ClosedShape;
|
||||
import org.apache.pekko.stream.UniformFanOutShape;
|
||||
import org.apache.pekko.stream.javadsl.Broadcast;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.stream.javadsl.GraphDSL;
|
||||
import org.apache.pekko.stream.javadsl.Keep;
|
||||
import org.apache.pekko.stream.javadsl.RunnableGraph;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
// #import
|
||||
|
||||
public class BroadcastDocExample {
|
||||
|
||||
private final ActorSystem system = ActorSystem.create("BroadcastDocExample");
|
||||
|
||||
void broadcastExample() {
|
||||
|
||||
// #broadcast
|
||||
|
||||
Source<Integer, NotUsed> source = Source.range(1, 10);
|
||||
|
||||
Sink<Integer, CompletionStage<Integer>> countSink =
|
||||
Flow.of(Integer.class).toMat(Sink.fold(0, (acc, elem) -> acc + 1), Keep.right());
|
||||
Sink<Integer, CompletionStage<Integer>> minSink =
|
||||
Flow.of(Integer.class).toMat(Sink.fold(0, Math::min), Keep.right());
|
||||
Sink<Integer, CompletionStage<Integer>> maxSink =
|
||||
Flow.of(Integer.class).toMat(Sink.fold(0, Math::max), Keep.right());
|
||||
|
||||
final Tuple3<CompletionStage<Integer>, CompletionStage<Integer>, CompletionStage<Integer>>
|
||||
result =
|
||||
RunnableGraph.fromGraph(
|
||||
GraphDSL.create3(
|
||||
countSink,
|
||||
minSink,
|
||||
maxSink,
|
||||
Tuple3::create,
|
||||
(builder, countS, minS, maxS) -> {
|
||||
final UniformFanOutShape<Integer, Integer> broadcast =
|
||||
builder.add(Broadcast.create(3));
|
||||
builder.from(builder.add(source)).viaFanOut(broadcast);
|
||||
builder.from(broadcast.out(0)).to(countS);
|
||||
builder.from(broadcast.out(1)).to(minS);
|
||||
builder.from(broadcast.out(2)).to(maxS);
|
||||
return ClosedShape.getInstance();
|
||||
}))
|
||||
.run(system);
|
||||
|
||||
// #broadcast
|
||||
|
||||
// #broadcast-async
|
||||
RunnableGraph.fromGraph(
|
||||
GraphDSL.create3(
|
||||
countSink.async(),
|
||||
minSink.async(),
|
||||
maxSink.async(),
|
||||
Tuple3::create,
|
||||
(builder, countS, minS, maxS) -> {
|
||||
final UniformFanOutShape<Integer, Integer> broadcast =
|
||||
builder.add(Broadcast.create(3));
|
||||
builder.from(builder.add(source)).viaFanOut(broadcast);
|
||||
|
||||
builder.from(broadcast.out(0)).to(countS);
|
||||
builder.from(broadcast.out(1)).to(minS);
|
||||
builder.from(broadcast.out(2)).to(maxS);
|
||||
return ClosedShape.getInstance();
|
||||
}));
|
||||
// #broadcast-async
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,87 @@
|
|||
/*
|
||||
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
|
||||
// #import
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.ClosedShape;
|
||||
import org.apache.pekko.stream.UniformFanInShape;
|
||||
import org.apache.pekko.stream.UniformFanOutShape;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.stream.javadsl.GraphDSL;
|
||||
import org.apache.pekko.stream.javadsl.MergeSequence;
|
||||
import org.apache.pekko.stream.javadsl.Partition;
|
||||
import org.apache.pekko.stream.javadsl.RunnableGraph;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
// #import
|
||||
|
||||
public class MergeSequenceDocExample {
|
||||
|
||||
private final ActorSystem system = ActorSystem.create("MergeSequenceDocExample");
|
||||
|
||||
interface Message {}
|
||||
|
||||
boolean shouldProcess(Message message) {
|
||||
return true;
|
||||
}
|
||||
|
||||
Source<Message, NotUsed> createSubscription() {
|
||||
return null;
|
||||
}
|
||||
|
||||
Flow<Pair<Message, Long>, Pair<Message, Long>, NotUsed> createMessageProcessor() {
|
||||
return null;
|
||||
}
|
||||
|
||||
Sink<Message, NotUsed> createMessageAcknowledger() {
|
||||
return null;
|
||||
}
|
||||
|
||||
void mergeSequenceExample() {
|
||||
|
||||
// #merge-sequence
|
||||
|
||||
Source<Message, NotUsed> subscription = createSubscription();
|
||||
Flow<Pair<Message, Long>, Pair<Message, Long>, NotUsed> messageProcessor =
|
||||
createMessageProcessor();
|
||||
Sink<Message, NotUsed> messageAcknowledger = createMessageAcknowledger();
|
||||
|
||||
RunnableGraph.fromGraph(
|
||||
GraphDSL.create(
|
||||
builder -> {
|
||||
// Partitions stream into messages that should or should not be processed
|
||||
UniformFanOutShape<Pair<Message, Long>, Pair<Message, Long>> partition =
|
||||
builder.add(
|
||||
Partition.create(2, element -> shouldProcess(element.first()) ? 0 : 1));
|
||||
// Merges stream by the index produced by zipWithIndex
|
||||
UniformFanInShape<Pair<Message, Long>, Pair<Message, Long>> merge =
|
||||
builder.add(MergeSequence.create(2, Pair::second));
|
||||
|
||||
builder.from(builder.add(subscription.zipWithIndex())).viaFanOut(partition);
|
||||
// First goes through message processor
|
||||
builder.from(partition.out(0)).via(builder.add(messageProcessor)).viaFanIn(merge);
|
||||
// Second partition bypasses message processor
|
||||
builder.from(partition.out(1)).viaFanIn(merge);
|
||||
|
||||
// Unwrap message index pairs and send to acknowledger
|
||||
builder
|
||||
.from(merge.out())
|
||||
.to(
|
||||
builder.add(
|
||||
Flow.<Pair<Message, Long>>create()
|
||||
.map(Pair::first)
|
||||
.to(messageAcknowledger)));
|
||||
|
||||
return ClosedShape.getInstance();
|
||||
}))
|
||||
.run(system);
|
||||
|
||||
// #merge-sequence
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,59 @@
|
|||
/*
|
||||
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
|
||||
// #import
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.stream.Attributes;
|
||||
import org.apache.pekko.stream.ClosedShape;
|
||||
import org.apache.pekko.stream.UniformFanOutShape;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.stream.javadsl.GraphDSL;
|
||||
import org.apache.pekko.stream.javadsl.Partition;
|
||||
import org.apache.pekko.stream.javadsl.RunnableGraph;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
// #import
|
||||
|
||||
public class PartitionDocExample {
|
||||
|
||||
private final ActorSystem system = ActorSystem.create("PartitionDocExample");
|
||||
|
||||
void partitionExample() {
|
||||
|
||||
// #partition
|
||||
|
||||
Source<Integer, NotUsed> source = Source.range(1, 10);
|
||||
|
||||
Sink<Integer, NotUsed> even =
|
||||
Flow.of(Integer.class)
|
||||
.log("even")
|
||||
.withAttributes(Attributes.createLogLevels(Attributes.logLevelInfo()))
|
||||
.to(Sink.ignore());
|
||||
Sink<Integer, NotUsed> odd =
|
||||
Flow.of(Integer.class)
|
||||
.log("odd")
|
||||
.withAttributes(Attributes.createLogLevels(Attributes.logLevelInfo()))
|
||||
.to(Sink.ignore());
|
||||
|
||||
RunnableGraph.fromGraph(
|
||||
GraphDSL.create(
|
||||
builder -> {
|
||||
UniformFanOutShape<Integer, Integer> partition =
|
||||
builder.add(
|
||||
Partition.create(
|
||||
Integer.class, 2, element -> (element % 2 == 0) ? 0 : 1));
|
||||
builder.from(builder.add(source)).viaFanOut(partition);
|
||||
builder.from(partition.out(0)).to(builder.add(even));
|
||||
builder.from(partition.out(1)).to(builder.add(odd));
|
||||
return ClosedShape.getInstance();
|
||||
}))
|
||||
.run(system);
|
||||
|
||||
// #partition
|
||||
}
|
||||
}
|
||||
172
docs/src/test/java/jdocs/stream/operators/SinkDocExamples.java
Normal file
172
docs/src/test/java/jdocs/stream/operators/SinkDocExamples.java
Normal file
|
|
@ -0,0 +1,172 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
|
||||
import org.apache.pekko.stream.javadsl.AsPublisher;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
// #takeLast-operator-example
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.reactivestreams.Publisher;
|
||||
// #takeLast-operator-example
|
||||
import java.util.*;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
public class SinkDocExamples {
|
||||
|
||||
private static final ActorSystem system = ActorSystem.create("SourceFromExample");
|
||||
|
||||
static void reduceExample() {
|
||||
|
||||
// #reduce-operator-example
|
||||
Source<Integer, NotUsed> ints = Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
|
||||
CompletionStage<Integer> sum = ints.runWith(Sink.reduce((a, b) -> a + b), system);
|
||||
sum.thenAccept(System.out::println);
|
||||
// 55
|
||||
// #reduce-operator-example
|
||||
}
|
||||
|
||||
static void seqExample() {
|
||||
// #seq-operator-example
|
||||
Source<Integer, NotUsed> ints = Source.from(Arrays.asList(1, 2, 3));
|
||||
CompletionStage<List<Integer>> result = ints.runWith(Sink.seq(), system);
|
||||
result.thenAccept(list -> list.forEach(System.out::println));
|
||||
// 1
|
||||
// 2
|
||||
// 3
|
||||
// #seq-operator-example
|
||||
}
|
||||
|
||||
static void takeLastExample() {
|
||||
// #takeLast-operator-example
|
||||
// pair of (Name, GPA)
|
||||
List<Pair<String, Double>> sortedStudents =
|
||||
Arrays.asList(
|
||||
new Pair<>("Benita", 2.1),
|
||||
new Pair<>("Adrian", 3.1),
|
||||
new Pair<>("Alexis", 4.0),
|
||||
new Pair<>("Kendra", 4.2),
|
||||
new Pair<>("Jerrie", 4.3),
|
||||
new Pair<>("Alison", 4.7));
|
||||
|
||||
Source<Pair<String, Double>, NotUsed> studentSource = Source.from(sortedStudents);
|
||||
|
||||
CompletionStage<List<Pair<String, Double>>> topThree =
|
||||
studentSource.runWith(Sink.takeLast(3), system);
|
||||
|
||||
topThree.thenAccept(
|
||||
result -> {
|
||||
System.out.println("#### Top students ####");
|
||||
for (int i = result.size() - 1; i >= 0; i--) {
|
||||
Pair<String, Double> s = result.get(i);
|
||||
System.out.println("Name: " + s.first() + ", " + "GPA: " + s.second());
|
||||
}
|
||||
});
|
||||
/*
|
||||
#### Top students ####
|
||||
Name: Alison, GPA: 4.7
|
||||
Name: Jerrie, GPA: 4.3
|
||||
Name: Kendra, GPA: 4.2
|
||||
*/
|
||||
// #takeLast-operator-example
|
||||
}
|
||||
|
||||
static void headExample() {
|
||||
// #head-operator-example
|
||||
Source<Integer, NotUsed> source = Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
|
||||
CompletionStage<Integer> result = source.runWith(Sink.head(), system);
|
||||
result.thenAccept(System.out::println);
|
||||
// 1
|
||||
// #head-operator-example
|
||||
}
|
||||
|
||||
static void lastExample() {
|
||||
// #last-operator-example
|
||||
Source<Integer, NotUsed> source = Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
|
||||
CompletionStage<Integer> result = source.runWith(Sink.last(), system);
|
||||
result.thenAccept(System.out::println);
|
||||
// 10
|
||||
// #last-operator-example
|
||||
}
|
||||
|
||||
static void lastOptionExample() {
|
||||
// #lastOption-operator-example
|
||||
Source<Integer, NotUsed> source = Source.empty();
|
||||
CompletionStage<Optional<Integer>> result = source.runWith(Sink.lastOption(), system);
|
||||
result.thenAccept(System.out::println);
|
||||
// Optional.empty
|
||||
// #lastOption-operator-example
|
||||
}
|
||||
|
||||
static void foldExample() {
|
||||
// #fold
|
||||
Source<Integer, NotUsed> source = Source.range(1, 100);
|
||||
CompletionStage<Integer> sum =
|
||||
source.runWith(Sink.fold(0, (res, element) -> res + element), system);
|
||||
sum.thenAccept(System.out::println);
|
||||
// #fold
|
||||
}
|
||||
|
||||
static NotUsed cancelledExample() {
|
||||
// #cancelled
|
||||
Source<Integer, NotUsed> source = Source.range(1, 5);
|
||||
NotUsed sum = source.runWith(Sink.cancelled(), system);
|
||||
return sum;
|
||||
// #cancelled
|
||||
}
|
||||
|
||||
static void headOptionExample() {
|
||||
// #headoption
|
||||
Source<Integer, NotUsed> source = Source.empty();
|
||||
CompletionStage<Optional<Integer>> result = source.runWith(Sink.headOption(), system);
|
||||
result.thenAccept(System.out::println);
|
||||
// Optional.empty
|
||||
// #headoption
|
||||
}
|
||||
|
||||
static void ignoreExample() {
|
||||
// #ignore
|
||||
Source<String, NotUsed> lines = readLinesFromFile();
|
||||
Source<UUID, NotUsed> databaseIds = lines.mapAsync(1, line -> saveLineToDatabase(line));
|
||||
databaseIds.mapAsync(1, uuid -> writeIdToFile(uuid)).runWith(Sink.ignore(), system);
|
||||
// #ignore
|
||||
}
|
||||
|
||||
static void asPublisherExample() {
|
||||
// #asPublisher
|
||||
Source<Integer, NotUsed> source = Source.range(1, 5);
|
||||
|
||||
Publisher<Integer> publisherFalse =
|
||||
source.runWith(Sink.asPublisher(AsPublisher.WITHOUT_FANOUT), system);
|
||||
CompletionStage<Integer> resultFromFirstSubscriberFalse =
|
||||
Source.fromPublisher(publisherFalse)
|
||||
.runWith(Sink.fold(0, (acc, element) -> acc + element), system);
|
||||
CompletionStage<Integer> resultFromSecondSubscriberFalse =
|
||||
Source.fromPublisher(publisherFalse)
|
||||
.runWith(Sink.fold(1, (acc, element) -> acc * element), system);
|
||||
|
||||
resultFromFirstSubscriberFalse.thenAccept(System.out::println); // 15
|
||||
resultFromSecondSubscriberFalse.thenAccept(
|
||||
System.out
|
||||
::println); // No output, because the source was not able to subscribe to the publisher.
|
||||
// #asPublisher
|
||||
}
|
||||
|
||||
private static Source<String, NotUsed> readLinesFromFile() {
|
||||
return Source.empty();
|
||||
}
|
||||
|
||||
private static CompletionStage<UUID> saveLineToDatabase(String line) {
|
||||
return CompletableFuture.completedFuture(UUID.randomUUID());
|
||||
}
|
||||
|
||||
private static CompletionStage<UUID> writeIdToFile(UUID uuid) {
|
||||
return CompletableFuture.completedFuture(uuid);
|
||||
}
|
||||
}
|
||||
155
docs/src/test/java/jdocs/stream/operators/SourceDocExamples.java
Normal file
155
docs/src/test/java/jdocs/stream/operators/SourceDocExamples.java
Normal file
|
|
@ -0,0 +1,155 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators;
|
||||
|
||||
// #imports
|
||||
// #range-imports
|
||||
import org.apache.pekko.Done;
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.actor.testkit.typed.javadsl.ManualTime;
|
||||
import org.apache.pekko.actor.testkit.typed.javadsl.TestKitJunitResource;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
// #range-imports
|
||||
|
||||
// #actor-ref-imports
|
||||
import org.apache.pekko.actor.ActorRef;
|
||||
import org.apache.pekko.stream.OverflowStrategy;
|
||||
import org.apache.pekko.stream.CompletionStrategy;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.testkit.TestProbe;
|
||||
// #actor-ref-imports
|
||||
|
||||
// #maybe
|
||||
import org.apache.pekko.stream.javadsl.RunnableGraph;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
// #maybe
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Optional;
|
||||
|
||||
// #imports
|
||||
|
||||
public class SourceDocExamples {
|
||||
|
||||
public static final TestKitJunitResource testKit = new TestKitJunitResource(ManualTime.config());
|
||||
|
||||
public static void fromExample() {
|
||||
final ActorSystem system = null;
|
||||
|
||||
// #source-from-example
|
||||
Source<Integer, NotUsed> ints = Source.from(Arrays.asList(0, 1, 2, 3, 4, 5));
|
||||
ints.runForeach(System.out::println, system);
|
||||
|
||||
String text =
|
||||
"Perfection is finally attained not when there is no longer more to add,"
|
||||
+ "but when there is no longer anything to take away.";
|
||||
Source<String, NotUsed> words = Source.from(Arrays.asList(text.split("\\s")));
|
||||
words.runForeach(System.out::println, system);
|
||||
// #source-from-example
|
||||
}
|
||||
|
||||
static void rangeExample() {
|
||||
|
||||
final ActorSystem system = ActorSystem.create("Source");
|
||||
|
||||
// #range
|
||||
|
||||
Source<Integer, NotUsed> source = Source.range(1, 100);
|
||||
|
||||
// #range
|
||||
|
||||
// #range
|
||||
Source<Integer, NotUsed> sourceStepFive = Source.range(1, 100, 5);
|
||||
|
||||
// #range
|
||||
|
||||
// #range
|
||||
Source<Integer, NotUsed> sourceStepNegative = Source.range(100, 1, -1);
|
||||
// #range
|
||||
|
||||
// #run-range
|
||||
source.runForeach(i -> System.out.println(i), system);
|
||||
// #run-range
|
||||
}
|
||||
|
||||
static void actorRef() {
|
||||
final ActorSystem system = null;
|
||||
|
||||
// #actor-ref
|
||||
|
||||
int bufferSize = 100;
|
||||
Source<Object, ActorRef> source =
|
||||
Source.actorRef(
|
||||
elem -> {
|
||||
// complete stream immediately if we send it Done
|
||||
if (elem == Done.done()) return Optional.of(CompletionStrategy.immediately());
|
||||
else return Optional.empty();
|
||||
},
|
||||
// never fail the stream because of a message
|
||||
elem -> Optional.empty(),
|
||||
bufferSize,
|
||||
OverflowStrategy.dropHead());
|
||||
|
||||
ActorRef actorRef = source.to(Sink.foreach(System.out::println)).run(system);
|
||||
actorRef.tell("hello", ActorRef.noSender());
|
||||
actorRef.tell("hello", ActorRef.noSender());
|
||||
|
||||
// The stream completes successfully with the following message
|
||||
actorRef.tell(Done.done(), ActorRef.noSender());
|
||||
// #actor-ref
|
||||
}
|
||||
|
||||
static void actorRefWithBackpressure() {
|
||||
final TestProbe probe = null;
|
||||
final ActorSystem system = null;
|
||||
|
||||
// #actorRefWithBackpressure
|
||||
Source<String, ActorRef> source =
|
||||
Source.<String>actorRefWithBackpressure(
|
||||
"ack",
|
||||
// complete when we send "complete"
|
||||
o -> {
|
||||
if (o == "complete") return Optional.of(CompletionStrategy.draining());
|
||||
else return Optional.empty();
|
||||
},
|
||||
// do not fail on any message
|
||||
o -> Optional.empty());
|
||||
|
||||
ActorRef actorRef = source.to(Sink.foreach(System.out::println)).run(system);
|
||||
probe.send(actorRef, "hello");
|
||||
probe.expectMsg("ack");
|
||||
probe.send(actorRef, "hello");
|
||||
probe.expectMsg("ack");
|
||||
|
||||
// The stream completes successfully with the following message
|
||||
actorRef.tell("complete", ActorRef.noSender());
|
||||
// #actorRefWithBackpressure
|
||||
}
|
||||
|
||||
static void maybeExample() {
|
||||
final ActorSystem system = null;
|
||||
|
||||
// #maybe
|
||||
Source<Integer, CompletableFuture<Optional<Integer>>> source = Source.<Integer>maybe();
|
||||
RunnableGraph<CompletableFuture<Optional<Integer>>> runnable =
|
||||
source.to(Sink.foreach(System.out::println));
|
||||
|
||||
CompletableFuture<Optional<Integer>> completable1 = runnable.run(system);
|
||||
completable1.complete(Optional.of(1)); // prints 1
|
||||
|
||||
CompletableFuture<Optional<Integer>> completable2 = runnable.run(system);
|
||||
completable2.complete(Optional.of(2)); // prints 2
|
||||
// #maybe
|
||||
}
|
||||
|
||||
static
|
||||
// #maybe-signature
|
||||
<Out> Source<Out, CompletableFuture<Optional<Out>>> maybe()
|
||||
// #maybe-signature
|
||||
{
|
||||
return Source.maybe();
|
||||
}
|
||||
}
|
||||
682
docs/src/test/java/jdocs/stream/operators/SourceOrFlow.java
Normal file
682
docs/src/test/java/jdocs/stream/operators/SourceOrFlow.java
Normal file
|
|
@ -0,0 +1,682 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators;
|
||||
|
||||
import org.apache.pekko.Done;
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorRef;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.event.LogMarker;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.japi.function.Function2;
|
||||
import org.apache.pekko.japi.pf.PFBuilder;
|
||||
import org.apache.pekko.stream.Attributes;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
|
||||
// #zip
|
||||
// #zip-with
|
||||
// #zip-with-index
|
||||
// #or-else
|
||||
// #prepend
|
||||
// #prependLazy
|
||||
// #concat
|
||||
// #concatLazy
|
||||
// #concatAllLazy
|
||||
// #interleave
|
||||
// #interleaveAll
|
||||
// #merge
|
||||
// #merge-sorted
|
||||
import org.apache.pekko.stream.javadsl.Keep;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
|
||||
import java.util.*;
|
||||
|
||||
// #merge-sorted
|
||||
// #merge
|
||||
// #interleave
|
||||
// #interleaveAll
|
||||
// #concat
|
||||
// #concatLazy
|
||||
// #concatAllLazy
|
||||
// #prepend
|
||||
// #prependLazy
|
||||
// #or-else
|
||||
// #zip-with-index
|
||||
// #zip-with
|
||||
// #zip
|
||||
|
||||
// #log
|
||||
import org.apache.pekko.event.LogMarker;
|
||||
import org.apache.pekko.stream.Attributes;
|
||||
|
||||
// #log
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
class SourceOrFlow {
|
||||
private static ActorSystem system = null;
|
||||
|
||||
void logExample() {
|
||||
Flow.of(String.class)
|
||||
// #log
|
||||
.log("myStream")
|
||||
.addAttributes(
|
||||
Attributes.createLogLevels(
|
||||
Attributes.logLevelOff(), // onElement
|
||||
Attributes.logLevelInfo(), // onFinish
|
||||
Attributes.logLevelError())) // onFailure
|
||||
// #log
|
||||
;
|
||||
}
|
||||
|
||||
void logWithMarkerExample() {
|
||||
Flow.of(String.class)
|
||||
// #logWithMarker
|
||||
.logWithMarker(
|
||||
"myStream", (e) -> LogMarker.create("myMarker", Collections.singletonMap("element", e)))
|
||||
.addAttributes(
|
||||
Attributes.createLogLevels(
|
||||
Attributes.logLevelOff(), // onElement
|
||||
Attributes.logLevelInfo(), // onFinish
|
||||
Attributes.logLevelError())) // onFailure
|
||||
// #logWithMarker
|
||||
;
|
||||
}
|
||||
|
||||
void zipWithIndexExample() {
|
||||
// #zip-with-index
|
||||
Source.from(Arrays.asList("apple", "orange", "banana"))
|
||||
.zipWithIndex()
|
||||
.runForeach(System.out::println, system);
|
||||
// this will print ('apple', 0), ('orange', 1), ('banana', 2)
|
||||
// #zip-with-index
|
||||
}
|
||||
|
||||
void zipExample() {
|
||||
// #zip
|
||||
Source<String, NotUsed> sourceFruits = Source.from(Arrays.asList("apple", "orange", "banana"));
|
||||
Source<String, NotUsed> sourceFirstLetters = Source.from(Arrays.asList("A", "O", "B"));
|
||||
sourceFruits.zip(sourceFirstLetters).runForeach(System.out::println, system);
|
||||
// this will print ('apple', 'A'), ('orange', 'O'), ('banana', 'B')
|
||||
|
||||
// #zip
|
||||
}
|
||||
|
||||
void zipWithExample() {
|
||||
// #zip-with
|
||||
Source<String, NotUsed> sourceCount = Source.from(Arrays.asList("one", "two", "three"));
|
||||
Source<String, NotUsed> sourceFruits = Source.from(Arrays.asList("apple", "orange", "banana"));
|
||||
sourceCount
|
||||
.zipWith(
|
||||
sourceFruits,
|
||||
(Function2<String, String, String>) (countStr, fruitName) -> countStr + " " + fruitName)
|
||||
.runForeach(System.out::println, system);
|
||||
// this will print 'one apple', 'two orange', 'three banana'
|
||||
|
||||
// #zip-with
|
||||
}
|
||||
|
||||
void prependExample() {
|
||||
// #prepend
|
||||
Source<String, NotUsed> ladies = Source.from(Arrays.asList("Emma", "Emily"));
|
||||
Source<String, NotUsed> gentlemen = Source.from(Arrays.asList("Liam", "William"));
|
||||
gentlemen.prepend(ladies).runForeach(System.out::println, system);
|
||||
// this will print "Emma", "Emily", "Liam", "William"
|
||||
|
||||
// #prepend
|
||||
}
|
||||
|
||||
void prependLazyExample() {
|
||||
// #prepend
|
||||
Source<String, NotUsed> ladies = Source.from(Arrays.asList("Emma", "Emily"));
|
||||
Source<String, NotUsed> gentlemen = Source.from(Arrays.asList("Liam", "William"));
|
||||
gentlemen.prependLazy(ladies).runForeach(System.out::println, system);
|
||||
// this will print "Emma", "Emily", "Liam", "William"
|
||||
|
||||
// #prepend
|
||||
}
|
||||
|
||||
void concatExample() {
|
||||
// #concat
|
||||
Source<Integer, NotUsed> sourceA = Source.from(Arrays.asList(1, 2, 3, 4));
|
||||
Source<Integer, NotUsed> sourceB = Source.from(Arrays.asList(10, 20, 30, 40));
|
||||
sourceA.concat(sourceB).runForeach(System.out::println, system);
|
||||
// prints 1, 2, 3, 4, 10, 20, 30, 40
|
||||
|
||||
// #concat
|
||||
}
|
||||
|
||||
void concatLazyExample() {
|
||||
// #concatLazy
|
||||
Source<Integer, NotUsed> sourceA = Source.from(Arrays.asList(1, 2, 3, 4));
|
||||
Source<Integer, NotUsed> sourceB = Source.from(Arrays.asList(10, 20, 30, 40));
|
||||
sourceA.concatLazy(sourceB).runForeach(System.out::println, system);
|
||||
// prints 1, 2, 3, 4, 10, 20, 30, 40
|
||||
|
||||
// #concatLazy
|
||||
}
|
||||
|
||||
void concatAllLazyExample() {
|
||||
// #concatAllLazy
|
||||
Source<Integer, NotUsed> sourceA = Source.from(Arrays.asList(1, 2, 3));
|
||||
Source<Integer, NotUsed> sourceB = Source.from(Arrays.asList(4, 5, 6));
|
||||
Source<Integer, NotUsed> sourceC = Source.from(Arrays.asList(7, 8, 9));
|
||||
sourceA
|
||||
.concatAllLazy(sourceB, sourceC)
|
||||
.fold(new StringJoiner(","), (joiner, input) -> joiner.add(String.valueOf(input)))
|
||||
.runForeach(System.out::println, system);
|
||||
// prints 1,2,3,4,5,6,7,8,9
|
||||
// #concatAllLazy
|
||||
}
|
||||
|
||||
void interleaveExample() {
|
||||
// #interleave
|
||||
Source<Integer, NotUsed> sourceA = Source.from(Arrays.asList(1, 2, 3, 4));
|
||||
Source<Integer, NotUsed> sourceB = Source.from(Arrays.asList(10, 20, 30, 40));
|
||||
sourceA.interleave(sourceB, 2).runForeach(System.out::println, system);
|
||||
// prints 1, 2, 10, 20, 3, 4, 30, 40
|
||||
|
||||
// #interleave
|
||||
}
|
||||
|
||||
void interleaveAllExample() {
|
||||
// #interleaveAll
|
||||
Source<Integer, NotUsed> sourceA = Source.from(Arrays.asList(1, 2, 7, 8));
|
||||
Source<Integer, NotUsed> sourceB = Source.from(Arrays.asList(3, 4, 9));
|
||||
Source<Integer, NotUsed> sourceC = Source.from(Arrays.asList(5, 6));
|
||||
sourceA
|
||||
.interleaveAll(Arrays.asList(sourceB, sourceC), 2, false)
|
||||
.fold(new StringJoiner(","), (joiner, input) -> joiner.add(String.valueOf(input)))
|
||||
.runForeach(System.out::println, system);
|
||||
// prints 1,2,3,4,5,6,7,8,9
|
||||
// #interleaveAll
|
||||
}
|
||||
|
||||
void mergeExample() {
|
||||
// #merge
|
||||
Source<Integer, NotUsed> sourceA = Source.from(Arrays.asList(1, 2, 3, 4));
|
||||
Source<Integer, NotUsed> sourceB = Source.from(Arrays.asList(10, 20, 30, 40));
|
||||
sourceA.merge(sourceB).runForeach(System.out::println, system);
|
||||
// merging is not deterministic, can for example print 1, 2, 3, 4, 10, 20, 30, 40
|
||||
|
||||
// #merge
|
||||
}
|
||||
|
||||
void mergeAllExample() {
|
||||
// #merge-all
|
||||
Source<Integer, NotUsed> sourceA = Source.from(Arrays.asList(1, 2, 3));
|
||||
Source<Integer, NotUsed> sourceB = Source.from(Arrays.asList(4, 5, 6));
|
||||
Source<Integer, NotUsed> sourceC = Source.from(Arrays.asList(7, 8, 9, 10));
|
||||
sourceA
|
||||
.mergeAll(Arrays.asList(sourceB, sourceC), false)
|
||||
.runForeach(System.out::println, system);
|
||||
// merging is not deterministic, can for example print 1, 2, 3, 4, 5, 6, 7, 8, 9, 10
|
||||
// #merge-all
|
||||
}
|
||||
|
||||
void mergePreferredExample() {
|
||||
// #mergePreferred
|
||||
Source<Integer, NotUsed> sourceA = Source.from(Arrays.asList(1, 2, 3, 4));
|
||||
Source<Integer, NotUsed> sourceB = Source.from(Arrays.asList(10, 20, 30, 40));
|
||||
|
||||
sourceA.mergePreferred(sourceB, false, false).runForeach(System.out::println, system);
|
||||
// prints 1, 10, ... since both sources have their first element ready and the left source is
|
||||
// preferred
|
||||
|
||||
sourceA.mergePreferred(sourceB, true, false).runForeach(System.out::println, system);
|
||||
// prints 10, 1, ... since both sources have their first element ready and the right source is
|
||||
// preferred
|
||||
// #mergePreferred
|
||||
}
|
||||
|
||||
void mergePrioritizedExample() {
|
||||
// #mergePrioritized
|
||||
Source<Integer, NotUsed> sourceA = Source.from(Arrays.asList(1, 2, 3, 4));
|
||||
Source<Integer, NotUsed> sourceB = Source.from(Arrays.asList(10, 20, 30, 40));
|
||||
|
||||
sourceA.mergePrioritized(sourceB, 99, 1, false).runForeach(System.out::println, system);
|
||||
// prints e.g. 1, 10, 2, 3, 4, 20, 30, 40 since both sources have their first element ready and
|
||||
// the left source has higher priority – if both sources have elements ready, sourceA has a
|
||||
// 99% chance of being picked next while sourceB has a 1% chance
|
||||
// #mergePrioritized
|
||||
}
|
||||
|
||||
void mergePrioritizedNExample() {
|
||||
// #mergePrioritizedN
|
||||
Source<Integer, NotUsed> sourceA = Source.from(Arrays.asList(1, 2, 3, 4));
|
||||
Source<Integer, NotUsed> sourceB = Source.from(Arrays.asList(10, 20, 30, 40));
|
||||
Source<Integer, NotUsed> sourceC = Source.from(Arrays.asList(100, 200, 300, 400));
|
||||
List<Pair<Source<Integer, ?>, Integer>> sourcesAndPriorities =
|
||||
Arrays.asList(new Pair<>(sourceA, 9900), new Pair<>(sourceB, 99), new Pair<>(sourceC, 1));
|
||||
Source.mergePrioritizedN(sourcesAndPriorities, false).runForeach(System.out::println, system);
|
||||
// prints e.g. 1, 100, 2, 3, 4, 10, 20, 30, 40, 200, 300, 400 since both sources have their
|
||||
// first element ready and
|
||||
// the left sourceA has higher priority - if both sources have elements ready, sourceA has a 99%
|
||||
// chance of being picked next
|
||||
// while sourceB has a 0.99% chance and sourceC has a 0.01% chance
|
||||
// #mergePrioritizedN
|
||||
}
|
||||
|
||||
void mergeSortedExample() {
|
||||
// #merge-sorted
|
||||
Source<Integer, NotUsed> sourceA = Source.from(Arrays.asList(1, 3, 5, 7));
|
||||
Source<Integer, NotUsed> sourceB = Source.from(Arrays.asList(2, 4, 6, 8));
|
||||
sourceA
|
||||
.mergeSorted(sourceB, Comparator.<Integer>naturalOrder())
|
||||
.runForeach(System.out::println, system);
|
||||
// prints 1, 2, 3, 4, 5, 6, 7, 8
|
||||
|
||||
Source<Integer, NotUsed> sourceC = Source.from(Arrays.asList(20, 1, 1, 1));
|
||||
sourceA
|
||||
.mergeSorted(sourceC, Comparator.<Integer>naturalOrder())
|
||||
.runForeach(System.out::println, system);
|
||||
// prints 1, 3, 5, 7, 20, 1, 1, 1
|
||||
// #merge-sorted
|
||||
}
|
||||
|
||||
void orElseExample() {
|
||||
// #or-else
|
||||
Source<String, NotUsed> source1 = Source.from(Arrays.asList("First source"));
|
||||
Source<String, NotUsed> source2 = Source.from(Arrays.asList("Second source"));
|
||||
Source<String, NotUsed> emptySource = Source.empty();
|
||||
|
||||
source1.orElse(source2).runForeach(System.out::println, system);
|
||||
// this will print "First source"
|
||||
|
||||
emptySource.orElse(source2).runForeach(System.out::println, system);
|
||||
// this will print "Second source"
|
||||
|
||||
// #or-else
|
||||
}
|
||||
|
||||
void conflateExample() {
|
||||
// #conflate
|
||||
Source.cycle(() -> Arrays.asList(1, 10, 100).iterator())
|
||||
.throttle(10, Duration.ofSeconds(1)) // fast upstream
|
||||
.conflate((Integer acc, Integer el) -> acc + el)
|
||||
.throttle(1, Duration.ofSeconds(1)); // slow downstream
|
||||
// #conflate
|
||||
}
|
||||
|
||||
void scanExample() {
|
||||
// #scan
|
||||
Source<Integer, NotUsed> source = Source.range(1, 5);
|
||||
source.scan(0, (acc, x) -> acc + x).runForeach(System.out::println, system);
|
||||
// 0 (= 0)
|
||||
// 1 (= 0 + 1)
|
||||
// 3 (= 0 + 1 + 2)
|
||||
// 6 (= 0 + 1 + 2 + 3)
|
||||
// 10 (= 0 + 1 + 2 + 3 + 4)
|
||||
// 15 (= 0 + 1 + 2 + 3 + 4 + 5)
|
||||
// #scan
|
||||
}
|
||||
|
||||
// #scan-async
|
||||
CompletionStage<Integer> asyncFunction(int acc, int next) {
|
||||
return CompletableFuture.supplyAsync(() -> acc + next);
|
||||
}
|
||||
// #scan-async
|
||||
|
||||
void scanAsyncExample() {
|
||||
// #scan-async
|
||||
Source<Integer, NotUsed> source = Source.range(1, 5);
|
||||
source.scanAsync(0, (acc, x) -> asyncFunction(acc, x)).runForeach(System.out::println, system);
|
||||
// 0 (= 0)
|
||||
// 1 (= 0 + 1)
|
||||
// 3 (= 0 + 1 + 2)
|
||||
// 6 (= 0 + 1 + 2 + 3)
|
||||
// 10 (= 0 + 1 + 2 + 3 + 4)
|
||||
// 15 (= 0 + 1 + 2 + 3 + 4 + 5)
|
||||
// #scan-async
|
||||
}
|
||||
|
||||
static // #conflateWithSeed-type
|
||||
class Summed {
|
||||
|
||||
private final Integer el;
|
||||
|
||||
public Summed(Integer el) {
|
||||
this.el = el;
|
||||
}
|
||||
|
||||
public Summed sum(Summed other) {
|
||||
return new Summed(this.el + other.el);
|
||||
}
|
||||
}
|
||||
// #conflateWithSeed-type
|
||||
|
||||
void conflateWithSeedExample() {
|
||||
// #conflateWithSeed
|
||||
|
||||
Source.cycle(() -> Arrays.asList(1, 10, 100).iterator())
|
||||
.throttle(10, Duration.ofSeconds(1)) // fast upstream
|
||||
.conflateWithSeed(Summed::new, (Summed acc, Integer el) -> acc.sum(new Summed(el)))
|
||||
.throttle(1, Duration.ofSeconds(1)); // slow downstream
|
||||
// #conflateWithSeed
|
||||
}
|
||||
|
||||
// #collect-elements
|
||||
static interface Message {}
|
||||
|
||||
static class Ping implements Message {
|
||||
final int id;
|
||||
|
||||
Ping(int id) {
|
||||
this.id = id;
|
||||
}
|
||||
}
|
||||
|
||||
static class Pong {
|
||||
final int id;
|
||||
|
||||
Pong(int id) {
|
||||
this.id = id;
|
||||
}
|
||||
}
|
||||
// #collect-elements
|
||||
|
||||
void collectExample() {
|
||||
// #collect
|
||||
Flow<Message, Pong, NotUsed> flow =
|
||||
Flow.of(Message.class)
|
||||
.collect(
|
||||
new PFBuilder<Message, Pong>()
|
||||
.match(Ping.class, p -> p.id != 0, p -> new Pong(p.id))
|
||||
.build());
|
||||
// #collect
|
||||
}
|
||||
|
||||
void collectTypeExample() {
|
||||
// #collectType
|
||||
Flow<Message, Pong, NotUsed> flow =
|
||||
Flow.of(Message.class)
|
||||
.collectType(Ping.class)
|
||||
.filter(p -> p.id != 0)
|
||||
.map(p -> new Pong(p.id));
|
||||
// #collectType
|
||||
}
|
||||
|
||||
void groupedExample() {
|
||||
// #grouped
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7))
|
||||
.grouped(3)
|
||||
.runForeach(System.out::println, system);
|
||||
// [1, 2, 3]
|
||||
// [4, 5, 6]
|
||||
// [7]
|
||||
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7))
|
||||
.grouped(3)
|
||||
.map(g -> g.stream().reduce(0, Integer::sum))
|
||||
.runForeach(System.out::println, system);
|
||||
// 6 (= 1 + 2 + 3)
|
||||
// 15 (= 4 + 5 + 6)
|
||||
// 7 (= 7)
|
||||
// #grouped
|
||||
}
|
||||
|
||||
void groupedWeightedExample() {
|
||||
// #groupedWeighted
|
||||
Source.from(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4), Arrays.asList(5, 6)))
|
||||
.groupedWeighted(4, x -> (long) x.size())
|
||||
.runForeach(System.out::println, system);
|
||||
// [[1, 2], [3, 4]]
|
||||
// [[5, 6]]
|
||||
|
||||
Source.from(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4), Arrays.asList(5, 6)))
|
||||
.groupedWeighted(3, x -> (long) x.size())
|
||||
.runForeach(System.out::println, system);
|
||||
// [[1, 2], [3, 4]]
|
||||
// [[5, 6]]
|
||||
// #groupedWeighted
|
||||
}
|
||||
|
||||
static
|
||||
// #fold // #foldAsync
|
||||
class Histogram {
|
||||
final long low;
|
||||
final long high;
|
||||
|
||||
private Histogram(long low, long high) {
|
||||
this.low = low;
|
||||
this.high = high;
|
||||
}
|
||||
|
||||
// Immutable start value
|
||||
public static Histogram INSTANCE = new Histogram(0L, 0L);
|
||||
|
||||
// #foldAsync
|
||||
|
||||
public Histogram add(int number) {
|
||||
if (number < 100) {
|
||||
return new Histogram(low + 1L, high);
|
||||
} else {
|
||||
return new Histogram(low, high + 1L);
|
||||
}
|
||||
}
|
||||
// #fold
|
||||
|
||||
// #foldAsync
|
||||
public CompletionStage<Histogram> addAsync(Integer n) {
|
||||
if (n < 100) {
|
||||
return CompletableFuture.supplyAsync(() -> new Histogram(low + 1L, high));
|
||||
} else {
|
||||
return CompletableFuture.supplyAsync(() -> new Histogram(low, high + 1L));
|
||||
}
|
||||
}
|
||||
// #fold
|
||||
}
|
||||
// #fold // #foldAsync
|
||||
|
||||
void foldExample() {
|
||||
// #fold
|
||||
|
||||
// Folding over the numbers from 1 to 150:
|
||||
Source.range(1, 150)
|
||||
.fold(Histogram.INSTANCE, Histogram::add)
|
||||
.runForeach(h -> System.out.println("Histogram(" + h.low + ", " + h.high + ")"), system);
|
||||
|
||||
// Prints: Histogram(99, 51)
|
||||
// #fold
|
||||
}
|
||||
|
||||
void foldAsyncExample() {
|
||||
// #foldAsync
|
||||
|
||||
// Folding over the numbers from 1 to 150:
|
||||
Source.range(1, 150)
|
||||
.foldAsync(Histogram.INSTANCE, Histogram::addAsync)
|
||||
.runForeach(h -> System.out.println("Histogram(" + h.low + ", " + h.high + ")"), system);
|
||||
|
||||
// Prints: Histogram(99, 51)
|
||||
// #foldAsync
|
||||
}
|
||||
|
||||
void takeExample() {
|
||||
// #take
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5)).take(3).runForeach(System.out::println, system);
|
||||
// this will print:
|
||||
// 1
|
||||
// 2
|
||||
// 3
|
||||
// #take
|
||||
}
|
||||
|
||||
void takeWhileExample() {
|
||||
// #take-while
|
||||
Source.from(Arrays.asList(1, 2, 3, 4, 5))
|
||||
.takeWhile(i -> i < 3)
|
||||
.runForeach(System.out::println, system);
|
||||
// this will print:
|
||||
// 1
|
||||
// 2
|
||||
// #take-while
|
||||
}
|
||||
|
||||
void filterExample() {
|
||||
// #filter
|
||||
Source<String, NotUsed> words =
|
||||
Source.from(
|
||||
Arrays.asList(
|
||||
("Lorem ipsum dolor sit amet, consectetur adipiscing elit, sed do eiusmod tempor incididunt "
|
||||
+ "ut labore et dolore magna aliqua")
|
||||
.split(" ")));
|
||||
|
||||
Source<String, NotUsed> longWords = words.filter(w -> w.length() > 6);
|
||||
|
||||
longWords.runForeach(System.out::println, system);
|
||||
// consectetur
|
||||
// adipiscing
|
||||
// eiusmod
|
||||
// incididunt
|
||||
// #filter
|
||||
}
|
||||
|
||||
void filterNotExample() {
|
||||
// #filterNot
|
||||
Source<String, NotUsed> words =
|
||||
Source.from(
|
||||
Arrays.asList(
|
||||
("Lorem ipsum dolor sit amet, consectetur adipiscing elit, sed do eiusmod tempor incididunt "
|
||||
+ "ut labore et dolore magna aliqua")
|
||||
.split(" ")));
|
||||
|
||||
Source<String, NotUsed> longWords = words.filterNot(w -> w.length() <= 6);
|
||||
|
||||
longWords.runForeach(System.out::println, system);
|
||||
// consectetur
|
||||
// adipiscing
|
||||
// eiusmod
|
||||
// incididunt
|
||||
// #filterNot
|
||||
}
|
||||
|
||||
void dropExample() {
|
||||
// #drop
|
||||
Source<Integer, NotUsed> fiveIntegers = Source.from(Arrays.asList(1, 2, 3, 4, 5));
|
||||
Source<Integer, NotUsed> droppedThreeInts = fiveIntegers.drop(3);
|
||||
|
||||
droppedThreeInts.runForeach(System.out::println, system);
|
||||
// 4
|
||||
// 5
|
||||
// #drop
|
||||
}
|
||||
|
||||
void dropWhileExample() {
|
||||
// #dropWhile
|
||||
Source<Integer, NotUsed> droppedWhileNegative =
|
||||
Source.from(Arrays.asList(-3, -2, -1, 0, 1, 2, 3, -1)).dropWhile(integer -> integer < 0);
|
||||
|
||||
droppedWhileNegative.runForeach(System.out::println, system);
|
||||
// 1
|
||||
// 2
|
||||
// 3
|
||||
// -1
|
||||
// #dropWhile
|
||||
}
|
||||
|
||||
static void reduceExample() {
|
||||
// #reduceExample
|
||||
Source<Integer, NotUsed> source = Source.range(1, 100).reduce((acc, element) -> acc + element);
|
||||
CompletionStage<Integer> result = source.runWith(Sink.head(), system);
|
||||
result.thenAccept(System.out::println);
|
||||
// 5050
|
||||
// #reduceExample
|
||||
}
|
||||
|
||||
void watchExample() {
|
||||
// #watch
|
||||
final ActorRef ref = someActor();
|
||||
Flow<String, String, NotUsed> flow =
|
||||
Flow.of(String.class)
|
||||
.watch(ref)
|
||||
.recover(
|
||||
org.apache.pekko.stream.WatchedActorTerminatedException.class,
|
||||
() -> ref + " terminated");
|
||||
// #watch
|
||||
}
|
||||
|
||||
void groupByExample() {
|
||||
// #groupBy
|
||||
Source.range(1, 10)
|
||||
.groupBy(2, i -> i % 2 == 0) // create two sub-streams with odd and even numbers
|
||||
.reduce(Integer::sum) // for each sub-stream, sum its elements
|
||||
.mergeSubstreams() // merge back into a stream
|
||||
.runForeach(System.out::println, system);
|
||||
// 25
|
||||
// 30
|
||||
// #groupBy
|
||||
}
|
||||
|
||||
void watchTerminationExample() {
|
||||
// #watchTermination
|
||||
Source.range(1, 5)
|
||||
.watchTermination(
|
||||
(prevMatValue, completionStage) -> {
|
||||
completionStage.whenComplete(
|
||||
(done, exc) -> {
|
||||
if (done != null)
|
||||
System.out.println("The stream materialized " + prevMatValue.toString());
|
||||
else System.out.println(exc.getMessage());
|
||||
});
|
||||
return prevMatValue;
|
||||
})
|
||||
.runForeach(System.out::println, system);
|
||||
|
||||
/*
|
||||
Prints:
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
5
|
||||
The stream materialized NotUsed
|
||||
*/
|
||||
|
||||
Source.range(1, 5)
|
||||
.watchTermination(
|
||||
(prevMatValue, completionStage) -> {
|
||||
// this function will be run when the stream terminates
|
||||
// the CompletionStage provided as a second parameter indicates whether
|
||||
// the stream completed successfully or failed
|
||||
completionStage.whenComplete(
|
||||
(done, exc) -> {
|
||||
if (done != null)
|
||||
System.out.println("The stream materialized " + prevMatValue.toString());
|
||||
else System.out.println(exc.getMessage());
|
||||
});
|
||||
return prevMatValue;
|
||||
})
|
||||
.runForeach(
|
||||
element -> {
|
||||
if (element == 3) throw new Exception("Boom");
|
||||
else System.out.println(element);
|
||||
},
|
||||
system);
|
||||
/*
|
||||
Prints:
|
||||
1
|
||||
2
|
||||
Boom
|
||||
*/
|
||||
// #watchTermination
|
||||
}
|
||||
|
||||
static CompletionStage<Done> completionTimeoutExample() {
|
||||
// #completionTimeout
|
||||
Source<Integer, NotUsed> source = Source.range(1, 100000).map(number -> number * number);
|
||||
CompletionStage<Done> result = source.completionTimeout(Duration.ofMillis(10)).run(system);
|
||||
return result;
|
||||
// #completionTimeout
|
||||
}
|
||||
|
||||
private ActorRef someActor() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
115
docs/src/test/java/jdocs/stream/operators/WithContextTest.java
Normal file
115
docs/src/test/java/jdocs/stream/operators/WithContextTest.java
Normal file
|
|
@ -0,0 +1,115 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
// #imports
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
// #imports
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.hasItems;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
|
||||
public class WithContextTest extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("WithContextTest");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void documentAsSourceWithContext() throws Exception {
|
||||
// #asSourceWithContext
|
||||
|
||||
// values with their contexts as pairs
|
||||
Collection<Pair<String, Integer>> values =
|
||||
Arrays.asList(Pair.create("eins", 1), Pair.create("zwei", 2), Pair.create("drei", 3));
|
||||
|
||||
// a regular source with pairs as elements
|
||||
Source<Pair<String, Integer>, NotUsed> source = Source.from(values);
|
||||
|
||||
// split the pair into stream elements and their context
|
||||
SourceWithContext<String, Integer, NotUsed> sourceWithContext =
|
||||
source
|
||||
.asSourceWithContext(Pair::second) // pick the second pair element as context
|
||||
.map(Pair::first); // keep the first pair element as stream element
|
||||
|
||||
SourceWithContext<String, Integer, NotUsed> mapped =
|
||||
sourceWithContext
|
||||
// regular operators apply to the element without seeing the context
|
||||
.map(s -> s.replace('e', 'y'));
|
||||
|
||||
// running the source and asserting the outcome
|
||||
CompletionStage<List<Pair<String, Integer>>> result = mapped.runWith(Sink.seq(), system);
|
||||
List<Pair<String, Integer>> list = result.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
assertThat(
|
||||
list, hasItems(Pair.create("yins", 1), Pair.create("zwyi", 2), Pair.create("dryi", 3)));
|
||||
// #asSourceWithContext
|
||||
}
|
||||
|
||||
@Test
|
||||
public void documentAsFlowWithContext() throws Exception {
|
||||
// #asFlowWithContext
|
||||
|
||||
// a regular flow with pairs as elements
|
||||
Flow<Pair<String, Integer>, Pair<String, Integer>, NotUsed> flow = // ...
|
||||
// #asFlowWithContext
|
||||
Flow.create();
|
||||
// #asFlowWithContext
|
||||
|
||||
// Declare the "flow with context"
|
||||
// ingoing: String and Integer
|
||||
// outgoing: String and Integer
|
||||
FlowWithContext<String, Integer, String, Integer, NotUsed> flowWithContext =
|
||||
// convert the flow of pairs into a "flow with context"
|
||||
flow.<String, Integer, Integer>asFlowWithContext(
|
||||
// at the end of this flow: put the elements and the context back into a pair
|
||||
Pair::create,
|
||||
// pick the second element of the incoming pair as context
|
||||
Pair::second)
|
||||
// keep the first pair element as stream element
|
||||
.map(Pair::first);
|
||||
|
||||
FlowWithContext<String, Integer, String, Integer, NotUsed> mapped =
|
||||
flowWithContext
|
||||
// regular operators apply to the element without seeing the context
|
||||
.map(s -> s.replace('e', 'y'));
|
||||
|
||||
// running the flow with some sample data and asserting the outcome
|
||||
Collection<Pair<String, Integer>> values =
|
||||
Arrays.asList(Pair.create("eins", 1), Pair.create("zwei", 2), Pair.create("drei", 3));
|
||||
|
||||
SourceWithContext<String, Integer, NotUsed> source =
|
||||
Source.from(values).asSourceWithContext(Pair::second).map(Pair::first);
|
||||
|
||||
CompletionStage<List<Pair<String, Integer>>> result =
|
||||
source.via(mapped).runWith(Sink.seq(), system);
|
||||
List<Pair<String, Integer>> list = result.toCompletableFuture().get(1, TimeUnit.SECONDS);
|
||||
assertThat(
|
||||
list, hasItems(Pair.create("yins", 1), Pair.create("zwyi", 2), Pair.create("dryi", 3)));
|
||||
// #asFlowWithContext
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.converters;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
// #import
|
||||
import org.apache.pekko.japi.function.Creator;
|
||||
import org.apache.pekko.stream.Materializer;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.StreamConverters;
|
||||
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.stream.BaseStream;
|
||||
import java.util.stream.IntStream;
|
||||
import java.util.stream.Stream;
|
||||
// #import
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
/** */
|
||||
public class StreamConvertersToJava extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("StreamConvertersToJava");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateConverterToJava8Stream() {
|
||||
// #asJavaStream
|
||||
|
||||
Source<Integer, NotUsed> source = Source.range(0, 9).filter(i -> i % 2 == 0);
|
||||
|
||||
Sink<Integer, java.util.stream.Stream<Integer>> sink = StreamConverters.<Integer>asJavaStream();
|
||||
|
||||
Stream<Integer> jStream = source.runWith(sink, system);
|
||||
|
||||
// #asJavaStream
|
||||
assertEquals(5, jStream.count());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateCreatingASourceFromJava8Stream()
|
||||
throws InterruptedException, ExecutionException, TimeoutException {
|
||||
// #fromJavaStream
|
||||
|
||||
Creator<BaseStream<Integer, IntStream>> creator = () -> IntStream.rangeClosed(0, 9);
|
||||
Source<Integer, NotUsed> source = StreamConverters.fromJavaStream(creator);
|
||||
|
||||
Sink<Integer, CompletionStage<Integer>> sink = Sink.last();
|
||||
|
||||
CompletionStage<Integer> integerCompletionStage = source.runWith(sink, system);
|
||||
// #fromJavaStream
|
||||
assertEquals(
|
||||
9, integerCompletionStage.toCompletableFuture().get(5, TimeUnit.SECONDS).intValue());
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,135 @@
|
|||
/*
|
||||
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.converters;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.IOResult;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.stream.javadsl.Keep;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.stream.javadsl.StreamConverters;
|
||||
import org.apache.pekko.testkit.javadsl.TestKit;
|
||||
import org.apache.pekko.util.ByteString;
|
||||
import jdocs.AbstractJavaTest;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
import static org.apache.pekko.util.ByteString.emptyByteString;
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
public class ToFromJavaIOStreams extends AbstractJavaTest {
|
||||
|
||||
static ActorSystem system;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
system = ActorSystem.create("ToFromJavaIOStreams");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
TestKit.shutdownActorSystem(system);
|
||||
system = null;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateFromJavaIOStreams()
|
||||
throws InterruptedException, ExecutionException, TimeoutException {
|
||||
Charset charset = Charset.defaultCharset();
|
||||
byte[] bytes = "Some random input".getBytes(charset);
|
||||
Flow<ByteString, ByteString, NotUsed> toUpperCase =
|
||||
Flow.<ByteString>create()
|
||||
.map(
|
||||
bs -> {
|
||||
String str = bs.decodeString(charset).toUpperCase();
|
||||
return ByteString.fromString(str, charset);
|
||||
});
|
||||
|
||||
// #tofromJavaIOStream
|
||||
java.io.InputStream inputStream = new ByteArrayInputStream(bytes);
|
||||
Source<ByteString, CompletionStage<IOResult>> source =
|
||||
StreamConverters.fromInputStream(() -> inputStream);
|
||||
|
||||
// Given a ByteString produces a ByteString with the upperCase representation
|
||||
// Removed from the sample for brevity.
|
||||
// Flow<ByteString, ByteString, NotUsed> toUpperCase = ...
|
||||
|
||||
java.io.OutputStream outputStream = new ByteArrayOutputStream();
|
||||
Sink<ByteString, CompletionStage<IOResult>> sink =
|
||||
StreamConverters.fromOutputStream(() -> outputStream);
|
||||
|
||||
CompletionStage<IOResult> ioResultCompletionStage =
|
||||
source.via(toUpperCase).runWith(sink, system);
|
||||
// When the ioResultCompletionStage completes, the byte array backing the outputStream
|
||||
// will contain the uppercase representation of the bytes read from the inputStream.
|
||||
// #tofromJavaIOStream
|
||||
ioResultCompletionStage.toCompletableFuture().get(5, TimeUnit.SECONDS);
|
||||
assertEquals(
|
||||
"SOME RANDOM INPUT",
|
||||
new String(((ByteArrayOutputStream) outputStream).toByteArray(), charset));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateAsJavaInputStream() throws Exception {
|
||||
|
||||
// #asJavaInputStream
|
||||
Charset charset = Charset.defaultCharset();
|
||||
Flow<ByteString, ByteString, NotUsed> toUpperCase =
|
||||
Flow.<ByteString>create()
|
||||
.map(
|
||||
bs -> {
|
||||
String str = bs.decodeString(charset).toUpperCase();
|
||||
return ByteString.fromString(str, charset);
|
||||
});
|
||||
|
||||
final Sink<ByteString, InputStream> sink = StreamConverters.asInputStream();
|
||||
final InputStream stream =
|
||||
Source.single(ByteString.fromString("Some random input"))
|
||||
.via(toUpperCase)
|
||||
.runWith(sink, system);
|
||||
|
||||
// #asJavaInputStream
|
||||
byte[] a = new byte[17];
|
||||
stream.read(a);
|
||||
assertArrayEquals("SOME RANDOM INPUT".getBytes(), a);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateAsJavaOutputStream() throws Exception {
|
||||
|
||||
// #asJavaOutputStream
|
||||
final Source<ByteString, OutputStream> source = StreamConverters.asOutputStream();
|
||||
final Sink<ByteString, CompletionStage<ByteString>> sink =
|
||||
Sink.fold(emptyByteString(), (ByteString arg1, ByteString arg2) -> arg1.concat(arg2));
|
||||
|
||||
final Pair<OutputStream, CompletionStage<ByteString>> output =
|
||||
source.toMat(sink, Keep.both()).run(system);
|
||||
|
||||
// #asJavaOutputStream
|
||||
byte[] bytesArray = new byte[3];
|
||||
output.first().write(bytesArray);
|
||||
output.first().close();
|
||||
|
||||
final byte[] expected =
|
||||
output.second().toCompletableFuture().get(5, TimeUnit.SECONDS).toArray();
|
||||
|
||||
assertArrayEquals(expected, bytesArray);
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.flow;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.actor.Cancellable;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
import org.apache.pekko.stream.testkit.TestPublisher;
|
||||
import org.apache.pekko.stream.testkit.TestSubscriber;
|
||||
import org.apache.pekko.util.ByteString;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.Collections;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
public class FromSinkAndSource {
|
||||
|
||||
void halfClosedTcpServer() {
|
||||
ActorSystem system = null;
|
||||
// #halfClosedTcpServer
|
||||
// close in immediately
|
||||
Sink<ByteString, NotUsed> sink = Sink.cancelled();
|
||||
// periodic tick out
|
||||
Source<ByteString, Cancellable> source =
|
||||
Source.tick(Duration.ofSeconds(1), Duration.ofSeconds(1), "tick")
|
||||
.map(tick -> ByteString.fromString(System.currentTimeMillis() + "\n"));
|
||||
|
||||
Flow<ByteString, ByteString, NotUsed> serverFlow = Flow.fromSinkAndSource(sink, source);
|
||||
|
||||
Source<Tcp.IncomingConnection, CompletionStage<Tcp.ServerBinding>> connectionStream =
|
||||
Tcp.get(system)
|
||||
.bind(
|
||||
"127.0.0.1", // interface
|
||||
9999, // port
|
||||
100, // backlog
|
||||
Collections.emptyList(), // socket options
|
||||
true, // Important: half close enabled
|
||||
Optional.empty() // idle timeout
|
||||
);
|
||||
|
||||
connectionStream.runForeach(
|
||||
incomingConnection -> incomingConnection.handleWith(serverFlow, system), system);
|
||||
// #halfClosedTcpServer
|
||||
}
|
||||
|
||||
void chat() {
|
||||
ActorSystem system = null;
|
||||
// #chat
|
||||
Pair<Sink<String, NotUsed>, Source<String, NotUsed>> pair =
|
||||
MergeHub.of(String.class).toMat(BroadcastHub.of(String.class), Keep.both()).run(system);
|
||||
Sink<String, NotUsed> sink = pair.first();
|
||||
Source<String, NotUsed> source = pair.second();
|
||||
|
||||
Flow<ByteString, ByteString, NotUsed> framing =
|
||||
Framing.delimiter(ByteString.fromString("\n"), 1024);
|
||||
|
||||
Sink<ByteString, NotUsed> sinkWithFraming =
|
||||
framing.map(bytes -> bytes.utf8String()).to(pair.first());
|
||||
Source<ByteString, NotUsed> sourceWithFraming =
|
||||
source.map(text -> ByteString.fromString(text + "\n"));
|
||||
|
||||
Flow<ByteString, ByteString, NotUsed> serverFlow =
|
||||
Flow.fromSinkAndSource(sinkWithFraming, sourceWithFraming);
|
||||
|
||||
Tcp.get(system)
|
||||
.bind("127.0.0.1", 9999)
|
||||
.runForeach(
|
||||
incomingConnection -> incomingConnection.handleWith(serverFlow, system), system);
|
||||
// #chat
|
||||
}
|
||||
|
||||
<In, Out> void myApiThatTakesAFlow(Flow<In, Out, NotUsed> flow) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
void testing() {
|
||||
ActorSystem system = null;
|
||||
// #testing
|
||||
TestSubscriber.Probe<String> inProbe = TestSubscriber.probe(system);
|
||||
TestPublisher.Probe<String> outProbe = TestPublisher.probe(0, system);
|
||||
Flow<String, String, NotUsed> testFlow =
|
||||
Flow.fromSinkAndSource(Sink.fromSubscriber(inProbe), Source.fromPublisher(outProbe));
|
||||
|
||||
myApiThatTakesAFlow(testFlow);
|
||||
inProbe.expectNext("first");
|
||||
outProbe.expectRequest();
|
||||
outProbe.sendError(new RuntimeException("test error"));
|
||||
// ...
|
||||
// #testing
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.flow;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
public class FutureFlow {
|
||||
|
||||
private ActorSystem system = null;
|
||||
|
||||
// #base-on-first-element
|
||||
CompletionStage<Flow<Integer, String, NotUsed>> processingFlow(int id) {
|
||||
return CompletableFuture.completedFuture(
|
||||
Flow.of(Integer.class).map(n -> "id: " + id + " value: " + n));
|
||||
}
|
||||
// #base-on-first-element
|
||||
|
||||
public void compileOnlyBaseOnFirst() {
|
||||
// #base-on-first-element
|
||||
|
||||
Source<String, NotUsed> source =
|
||||
Source.range(1, 10)
|
||||
.prefixAndTail(1)
|
||||
.flatMapConcat(
|
||||
(pair) -> {
|
||||
List<Integer> head = pair.first();
|
||||
Source<Integer, NotUsed> tail = pair.second();
|
||||
|
||||
int id = head.get(0);
|
||||
|
||||
return tail.via(Flow.completionStageFlow(processingFlow(id)));
|
||||
});
|
||||
// #base-on-first-element
|
||||
}
|
||||
}
|
||||
84
docs/src/test/java/jdocs/stream/operators/flow/Lazy.java
Normal file
84
docs/src/test/java/jdocs/stream/operators/flow/Lazy.java
Normal file
|
|
@ -0,0 +1,84 @@
|
|||
/*
|
||||
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.flow;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.stream.javadsl.RunnableGraph;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
public class Lazy {
|
||||
private ActorSystem system = null;
|
||||
|
||||
void example() {
|
||||
// #simple-example
|
||||
Source<Integer, NotUsed> numbers =
|
||||
Source.unfold(
|
||||
0,
|
||||
n -> {
|
||||
int next = n + 1;
|
||||
System.out.println("Source producing " + next);
|
||||
return Optional.of(Pair.create(next, next));
|
||||
})
|
||||
.take(3);
|
||||
|
||||
Flow<Integer, Integer, CompletionStage<NotUsed>> flow =
|
||||
Flow.lazyFlow(
|
||||
() -> {
|
||||
System.out.println("Creating the actual flow");
|
||||
return Flow.fromFunction(
|
||||
element -> {
|
||||
System.out.println("Actual flow mapped " + element);
|
||||
return element;
|
||||
});
|
||||
});
|
||||
|
||||
numbers.via(flow).run(system);
|
||||
// prints:
|
||||
// Source producing 1
|
||||
// Creating the actual flow
|
||||
// Actual flow mapped 1
|
||||
// Source producing 2
|
||||
// Actual flow mapped 2
|
||||
// #simple-example
|
||||
}
|
||||
|
||||
void statefulMap() {
|
||||
// #mutable-example
|
||||
Flow<Integer, List<Integer>, CompletionStage<NotUsed>> mutableFold =
|
||||
Flow.lazyFlow(
|
||||
() -> {
|
||||
List<Integer> zero = new ArrayList<>();
|
||||
|
||||
return Flow.of(Integer.class)
|
||||
.fold(
|
||||
zero,
|
||||
(list, element) -> {
|
||||
list.add(element);
|
||||
return list;
|
||||
});
|
||||
});
|
||||
|
||||
RunnableGraph<NotUsed> stream =
|
||||
Source.range(1, 3).via(mutableFold).to(Sink.foreach(System.out::println));
|
||||
|
||||
stream.run(system);
|
||||
stream.run(system);
|
||||
stream.run(system);
|
||||
// prints:
|
||||
// [1, 2, 3]
|
||||
// [1, 2, 3]
|
||||
// [1, 2, 3]
|
||||
// #mutable-example
|
||||
}
|
||||
}
|
||||
112
docs/src/test/java/jdocs/stream/operators/flow/StatefulMap.java
Normal file
112
docs/src/test/java/jdocs/stream/operators/flow/StatefulMap.java
Normal file
|
|
@ -0,0 +1,112 @@
|
|||
/*
|
||||
* Copyright (C) 2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.flow;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
public class StatefulMap {
|
||||
static final ActorSystem system = null;
|
||||
|
||||
public void indexed() {
|
||||
// #zipWithIndex
|
||||
Source.from(Arrays.asList("A", "B", "C", "D"))
|
||||
.statefulMap(
|
||||
() -> 0L,
|
||||
(index, element) -> Pair.create(index + 1, Pair.create(element, index)),
|
||||
indexOnComplete -> Optional.empty())
|
||||
.runForeach(System.out::println, system);
|
||||
// prints
|
||||
// Pair(A,0)
|
||||
// Pair(B,1)
|
||||
// Pair(C,2)
|
||||
// Pair(D,3)
|
||||
// #zipWithIndex
|
||||
}
|
||||
|
||||
public void bufferUntilChanged() {
|
||||
// #bufferUntilChanged
|
||||
Source.from(Arrays.asList("A", "B", "B", "C", "C", "C", "D"))
|
||||
.statefulMap(
|
||||
() -> (List<String>) new LinkedList<String>(),
|
||||
(buffer, element) -> {
|
||||
if (buffer.size() > 0 && (!buffer.get(0).equals(element))) {
|
||||
return Pair.create(
|
||||
new LinkedList<>(Collections.singletonList(element)),
|
||||
Collections.unmodifiableList(buffer));
|
||||
} else {
|
||||
buffer.add(element);
|
||||
return Pair.create(buffer, Collections.<String>emptyList());
|
||||
}
|
||||
},
|
||||
Optional::ofNullable)
|
||||
.filterNot(List::isEmpty)
|
||||
.runForeach(System.out::println, system);
|
||||
// prints
|
||||
// [A]
|
||||
// [B, B]
|
||||
// [C, C, C]
|
||||
// [D]
|
||||
// #bufferUntilChanged
|
||||
}
|
||||
|
||||
public void distinctUntilChanged() {
|
||||
// #distinctUntilChanged
|
||||
Source.from(Arrays.asList("A", "B", "B", "C", "C", "C", "D"))
|
||||
.statefulMap(
|
||||
Optional::<String>empty,
|
||||
(lastElement, element) -> {
|
||||
if (lastElement.isPresent() && lastElement.get().equals(element)) {
|
||||
return Pair.create(lastElement, Optional.<String>empty());
|
||||
} else {
|
||||
return Pair.create(Optional.of(element), Optional.of(element));
|
||||
}
|
||||
},
|
||||
listOnComplete -> Optional.empty())
|
||||
.via(Flow.flattenOptional())
|
||||
.runForeach(System.out::println, system);
|
||||
// prints
|
||||
// A
|
||||
// B
|
||||
// C
|
||||
// D
|
||||
// #distinctUntilChanged
|
||||
}
|
||||
|
||||
public void statefulMapConcatLike() {
|
||||
// #statefulMapConcatLike
|
||||
Source.fromJavaStream(() -> IntStream.rangeClosed(1, 10))
|
||||
.statefulMap(
|
||||
() -> new ArrayList<Integer>(3),
|
||||
(list, element) -> {
|
||||
list.add(element);
|
||||
if (list.size() == 3) {
|
||||
return Pair.create(new ArrayList<Integer>(3), Collections.unmodifiableList(list));
|
||||
} else {
|
||||
return Pair.create(list, Collections.<Integer>emptyList());
|
||||
}
|
||||
},
|
||||
Optional::ofNullable)
|
||||
.mapConcat(list -> list)
|
||||
.runForeach(System.out::println, system);
|
||||
// prints
|
||||
// 1
|
||||
// 2
|
||||
// 3
|
||||
// 4
|
||||
// 5
|
||||
// 6
|
||||
// 7
|
||||
// 8
|
||||
// 9
|
||||
// 10
|
||||
// #statefulMapConcatLike
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,119 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.flow;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.typed.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.*;
|
||||
|
||||
public class StatefulMapConcat {
|
||||
|
||||
static final ActorSystem<?> system = null;
|
||||
|
||||
static void zipWithIndex() {
|
||||
// #zip-with-index
|
||||
Source<Pair<String, Long>, NotUsed> letterAndIndex =
|
||||
Source.from(Arrays.asList("a", "b", "c", "d"))
|
||||
.statefulMapConcat(
|
||||
() -> {
|
||||
// variables we close over with lambdas must be final, so we use a container,
|
||||
// a 1 element array, for the actual value.
|
||||
final long[] index = {0L};
|
||||
|
||||
// we return the function that will be invoked for each element
|
||||
return (element) -> {
|
||||
final Pair<String, Long> zipped = new Pair<>(element, index[0]);
|
||||
index[0] += 1;
|
||||
// we return an iterable with the single element
|
||||
return Collections.singletonList(zipped);
|
||||
};
|
||||
});
|
||||
|
||||
letterAndIndex.runForeach(System.out::println, system);
|
||||
// prints
|
||||
// Pair(a,0)
|
||||
// Pair(b,1)
|
||||
// Pair(c,2)
|
||||
// Pair(d,3)
|
||||
// #zip-with-index
|
||||
}
|
||||
|
||||
static void denylist() {
|
||||
// #denylist
|
||||
Source<String, NotUsed> fruitsAndDenyCommands =
|
||||
Source.from(
|
||||
Arrays.asList("banana", "pear", "orange", "deny:banana", "banana", "pear", "banana"));
|
||||
|
||||
Flow<String, String, NotUsed> denyFilterFlow =
|
||||
Flow.of(String.class)
|
||||
.statefulMapConcat(
|
||||
() -> {
|
||||
Set<String> denyList = new HashSet<>();
|
||||
|
||||
return (element) -> {
|
||||
if (element.startsWith("deny:")) {
|
||||
denyList.add(element.substring("deny:".length()));
|
||||
return Collections
|
||||
.emptyList(); // no element downstream when adding a deny listed keyword
|
||||
} else if (denyList.contains(element)) {
|
||||
return Collections
|
||||
.emptyList(); // no element downstream if element is deny listed
|
||||
} else {
|
||||
return Collections.singletonList(element);
|
||||
}
|
||||
};
|
||||
});
|
||||
|
||||
fruitsAndDenyCommands.via(denyFilterFlow).runForeach(System.out::println, system);
|
||||
// prints
|
||||
// banana
|
||||
// pear
|
||||
// orange
|
||||
// pear
|
||||
// #denylist
|
||||
}
|
||||
|
||||
static void reactOnEnd() {
|
||||
// #bs-last
|
||||
Source<String, NotUsed> words =
|
||||
Source.from(Arrays.asList("baboon", "crocodile", "bat", "flamingo", "hedgehog", "beaver"));
|
||||
|
||||
Flow<String, String, NotUsed> bWordsLast =
|
||||
Flow.of(String.class)
|
||||
.concat(Source.single("-end-"))
|
||||
.statefulMapConcat(
|
||||
() -> {
|
||||
List<String> stashedBWords = new ArrayList<>();
|
||||
|
||||
return (element) -> {
|
||||
if (element.startsWith("b")) {
|
||||
// add to stash and emit no element
|
||||
stashedBWords.add(element);
|
||||
return Collections.emptyList();
|
||||
} else if (element.equals("-end-")) {
|
||||
// return in the stashed words in the order they got stashed
|
||||
return stashedBWords;
|
||||
} else {
|
||||
// emit the element as is
|
||||
return Collections.singletonList(element);
|
||||
}
|
||||
};
|
||||
});
|
||||
|
||||
words.via(bWordsLast).runForeach(System.out::println, system);
|
||||
// prints
|
||||
// crocodile
|
||||
// flamingo
|
||||
// hedgehog
|
||||
// baboon
|
||||
// bat
|
||||
// beaver
|
||||
// #bs-last
|
||||
}
|
||||
}
|
||||
47
docs/src/test/java/jdocs/stream/operators/sink/Lazy.java
Normal file
47
docs/src/test/java/jdocs/stream/operators/sink/Lazy.java
Normal file
|
|
@ -0,0 +1,47 @@
|
|||
/*
|
||||
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.sink;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Keep;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
public class Lazy {
|
||||
|
||||
private ActorSystem system = null;
|
||||
|
||||
void example() {
|
||||
// #simple-example
|
||||
CompletionStage<Optional<String>> matVal =
|
||||
Source.<String>maybe()
|
||||
.map(
|
||||
element -> {
|
||||
System.out.println("mapped " + element);
|
||||
return element;
|
||||
})
|
||||
.toMat(
|
||||
Sink.lazySink(
|
||||
() -> {
|
||||
System.out.println("Sink created");
|
||||
return Sink.foreach(elem -> System.out.println("foreach " + elem));
|
||||
}),
|
||||
Keep.left())
|
||||
.run(system);
|
||||
|
||||
// some time passes
|
||||
// nothing has been printed
|
||||
matVal.toCompletableFuture().complete(Optional.of("one"));
|
||||
// now prints:
|
||||
// mapped one
|
||||
// Sink created
|
||||
// foreach one
|
||||
|
||||
// #simple-example
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,67 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.source;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
// #imports
|
||||
import org.apache.pekko.stream.javadsl.Concat;
|
||||
import org.apache.pekko.stream.javadsl.Merge;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
// ...
|
||||
|
||||
// #imports
|
||||
import java.util.Collections;
|
||||
|
||||
public class Combine {
|
||||
|
||||
private static ActorSystem system;
|
||||
|
||||
public void merge() throws Exception {
|
||||
// #source-combine-merge
|
||||
Source<Integer, NotUsed> source1 = Source.range(1, 3);
|
||||
Source<Integer, NotUsed> source2 = Source.range(8, 10);
|
||||
Source<Integer, NotUsed> source3 = Source.range(12, 14);
|
||||
|
||||
final Source<Integer, NotUsed> combined =
|
||||
Source.combine(source1, source2, Collections.singletonList(source3), Merge::create);
|
||||
|
||||
combined.runForeach(System.out::println, system);
|
||||
// could print (order between sources is not deterministic)
|
||||
// 1
|
||||
// 12
|
||||
// 8
|
||||
// 9
|
||||
// 13
|
||||
// 14
|
||||
// 2
|
||||
// 10
|
||||
// 3
|
||||
// #source-combine-merge
|
||||
}
|
||||
|
||||
public void concat() throws Exception {
|
||||
// #source-combine-concat
|
||||
Source<Integer, NotUsed> source1 = Source.range(1, 3);
|
||||
Source<Integer, NotUsed> source2 = Source.range(8, 10);
|
||||
Source<Integer, NotUsed> source3 = Source.range(12, 14);
|
||||
|
||||
final Source<Integer, NotUsed> sources =
|
||||
Source.combine(source1, source2, Collections.singletonList(source3), Concat::create);
|
||||
|
||||
sources.runForeach(System.out::println, system);
|
||||
// prints (order is deterministic)
|
||||
// 1
|
||||
// 2
|
||||
// 3
|
||||
// 8
|
||||
// 9
|
||||
// 10
|
||||
// 12
|
||||
// 13
|
||||
// 14
|
||||
// #source-combine-concat
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,28 @@
|
|||
/*
|
||||
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.source;
|
||||
|
||||
// #sourceCompletionStageSource
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
public class CompletionStageSource {
|
||||
|
||||
public static void sourceCompletionStageSource() {
|
||||
UserRepository userRepository = null; // an abstraction over the remote service
|
||||
Source<User, CompletionStage<NotUsed>> userCompletionStageSource =
|
||||
Source.completionStageSource(userRepository.loadUsers());
|
||||
// ...
|
||||
}
|
||||
|
||||
interface UserRepository {
|
||||
CompletionStage<Source<User, NotUsed>> loadUsers();
|
||||
}
|
||||
|
||||
static class User {}
|
||||
}
|
||||
// #sourceCompletionStageSource
|
||||
38
docs/src/test/java/jdocs/stream/operators/source/From.java
Normal file
38
docs/src/test/java/jdocs/stream/operators/source/From.java
Normal file
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.source;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
public class From {
|
||||
|
||||
private ActorSystem system = null;
|
||||
|
||||
void fromIteratorSample() {
|
||||
// #from-iterator
|
||||
Source.fromIterator(() -> Arrays.asList(1, 2, 3).iterator())
|
||||
.runForeach(System.out::println, system);
|
||||
// could print
|
||||
// 1
|
||||
// 2
|
||||
// 3
|
||||
// #from-iterator
|
||||
}
|
||||
|
||||
void fromJavaStreamSample() {
|
||||
// #from-javaStream
|
||||
Source.fromJavaStream(() -> IntStream.rangeClosed(1, 3))
|
||||
.runForeach(System.out::println, system);
|
||||
// could print
|
||||
// 1
|
||||
// 2
|
||||
// 3
|
||||
// #from-javaStream
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,34 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.source;
|
||||
|
||||
// #sourceFromCompletionStage
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.Done;
|
||||
import org.apache.pekko.actor.typed.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.*;
|
||||
|
||||
// #sourceFromCompletionStage
|
||||
|
||||
class FromCompletionStage {
|
||||
|
||||
public static void sourceFromCompletionStage() {
|
||||
// Use one ActorSystem per application
|
||||
ActorSystem system = null;
|
||||
|
||||
// #sourceFromCompletionStage
|
||||
CompletionStage<Integer> stage = CompletableFuture.completedFuture(10);
|
||||
|
||||
Source<Integer, NotUsed> source = Source.completionStage(stage);
|
||||
|
||||
Sink<Integer, CompletionStage<Done>> sink = Sink.foreach(i -> System.out.println(i.toString()));
|
||||
|
||||
source.runWith(sink, system); // 10
|
||||
// #sourceFromCompletionStage
|
||||
}
|
||||
}
|
||||
76
docs/src/test/java/jdocs/stream/operators/source/Lazy.java
Normal file
76
docs/src/test/java/jdocs/stream/operators/source/Lazy.java
Normal file
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.source;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.javadsl.RunnableGraph;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.SinkQueueWithCancel;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
public class Lazy {
|
||||
|
||||
private ActorSystem system = null;
|
||||
|
||||
private Source<String, NotUsed> createExpensiveSource() {
|
||||
throw new UnsupportedOperationException("Not implemented in sample");
|
||||
}
|
||||
|
||||
void notReallyThatLazy() {
|
||||
// #not-a-good-example
|
||||
Source<String, CompletionStage<NotUsed>> source =
|
||||
Source.lazySource(
|
||||
() -> {
|
||||
System.out.println("Creating the actual source");
|
||||
return createExpensiveSource();
|
||||
});
|
||||
|
||||
SinkQueueWithCancel<String> queue = source.runWith(Sink.queue(), system);
|
||||
|
||||
// ... time passes ...
|
||||
// at some point in time we pull the first time
|
||||
// but the source creation may already have been triggered
|
||||
queue.pull();
|
||||
// #not-a-good-example
|
||||
}
|
||||
|
||||
static class IteratorLikeThing {
|
||||
boolean thereAreMore() {
|
||||
throw new UnsupportedOperationException("Not implemented in sample");
|
||||
}
|
||||
|
||||
String extractNext() {
|
||||
throw new UnsupportedOperationException("Not implemented in sample");
|
||||
}
|
||||
}
|
||||
|
||||
void safeMutableSource() {
|
||||
// #one-per-materialization
|
||||
RunnableGraph<CompletionStage<NotUsed>> stream =
|
||||
Source.lazySource(
|
||||
() -> {
|
||||
IteratorLikeThing instance = new IteratorLikeThing();
|
||||
return Source.unfold(
|
||||
instance,
|
||||
sameInstance -> {
|
||||
if (sameInstance.thereAreMore())
|
||||
return Optional.of(Pair.create(sameInstance, sameInstance.extractNext()));
|
||||
else return Optional.empty();
|
||||
});
|
||||
})
|
||||
.to(Sink.foreach(System.out::println));
|
||||
|
||||
// each of the three materializations will have their own instance of IteratorLikeThing
|
||||
stream.run(system);
|
||||
stream.run(system);
|
||||
stream.run(system);
|
||||
// #one-per-materialization
|
||||
}
|
||||
}
|
||||
115
docs/src/test/java/jdocs/stream/operators/source/Restart.java
Normal file
115
docs/src/test/java/jdocs/stream/operators/source/Restart.java
Normal file
|
|
@ -0,0 +1,115 @@
|
|||
/*
|
||||
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.source;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.Cancellable;
|
||||
import org.apache.pekko.japi.Creator;
|
||||
import org.apache.pekko.stream.KillSwitches;
|
||||
import org.apache.pekko.stream.RestartSettings;
|
||||
import org.apache.pekko.stream.UniqueKillSwitch;
|
||||
import org.apache.pekko.stream.javadsl.Keep;
|
||||
import org.apache.pekko.stream.javadsl.RestartSource;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.Arrays;
|
||||
|
||||
public class Restart {
|
||||
static org.apache.pekko.actor.ActorSystem system = org.apache.pekko.actor.ActorSystem.create();
|
||||
|
||||
public static void onRestartWithBackoffInnerFailure() {
|
||||
// #restart-failure-inner-failure
|
||||
// could throw if for example it used a database connection to get rows
|
||||
Source<Creator<Integer>, NotUsed> flakySource =
|
||||
Source.from(
|
||||
Arrays.<Creator<Integer>>asList(
|
||||
() -> 1,
|
||||
() -> 2,
|
||||
() -> 3,
|
||||
() -> {
|
||||
throw new RuntimeException("darn");
|
||||
}));
|
||||
Source<Creator<Integer>, NotUsed> forever =
|
||||
RestartSource.onFailuresWithBackoff(
|
||||
RestartSettings.create(Duration.ofSeconds(1), Duration.ofSeconds(10), 0.1),
|
||||
() -> flakySource);
|
||||
forever.runWith(
|
||||
Sink.foreach((Creator<Integer> nr) -> system.log().info("{}", nr.create())), system);
|
||||
// logs
|
||||
// [INFO] [12/10/2019 13:51:58.300] [default-akka.test.stream-dispatcher-7]
|
||||
// [akka.actor.ActorSystemImpl(default)] 1
|
||||
// [INFO] [12/10/2019 13:51:58.301] [default-akka.test.stream-dispatcher-7]
|
||||
// [akka.actor.ActorSystemImpl(default)] 2
|
||||
// [INFO] [12/10/2019 13:51:58.302] [default-akka.test.stream-dispatcher-7]
|
||||
// [akka.actor.ActorSystemImpl(default)] 3
|
||||
// [WARN] [12/10/2019 13:51:58.310] [default-akka.test.stream-dispatcher-7]
|
||||
// [RestartWithBackoffSource(akka://default)] Restarting graph due to failure. stack_trace:
|
||||
// (RuntimeException: darn)
|
||||
// --> 1 second gap
|
||||
// [INFO] [12/10/2019 13:51:59.379] [default-akka.test.stream-dispatcher-8]
|
||||
// [akka.actor.ActorSystemImpl(default)] 1
|
||||
// [INFO] [12/10/2019 13:51:59.382] [default-akka.test.stream-dispatcher-8]
|
||||
// [akka.actor.ActorSystemImpl(default)] 2
|
||||
// [INFO] [12/10/2019 13:51:59.383] [default-akka.test.stream-dispatcher-8]
|
||||
// [akka.actor.ActorSystemImpl(default)] 3
|
||||
// [WARN] [12/10/2019 13:51:59.386] [default-akka.test.stream-dispatcher-8]
|
||||
// [RestartWithBackoffSource(akka://default)] Restarting graph due to failure. stack_trace:
|
||||
// (RuntimeException: darn)
|
||||
// --> 2 second gap
|
||||
// [INFO] [12/10/2019 13:52:01.594] [default-akka.test.stream-dispatcher-8]
|
||||
// [akka.actor.ActorSystemImpl(default)] 1
|
||||
// [INFO] [12/10/2019 13:52:01.595] [default-akka.test.stream-dispatcher-8]
|
||||
// [akka.actor.ActorSystemImpl(default)] 2
|
||||
// [INFO] [12/10/2019 13:52:01.595] [default-akka.test.stream-dispatcher-8]
|
||||
// [akka.actor.ActorSystemImpl(default)] 3
|
||||
// [WARN] [12/10/2019 13:52:01.596] [default-akka.test.stream-dispatcher-8]
|
||||
// [RestartWithBackoffSource(akka://default)] Restarting graph due to failure. stack_trace:
|
||||
// (RuntimeException: darn)
|
||||
// #restart-failure-inner-failure
|
||||
|
||||
}
|
||||
|
||||
public static void onRestartWithBackoffInnerComplete() {
|
||||
// #restart-failure-inner-complete
|
||||
Source<String, Cancellable> finiteSource =
|
||||
Source.tick(Duration.ofSeconds(1), Duration.ofSeconds(1), "tick").take(3);
|
||||
Source<String, NotUsed> forever =
|
||||
RestartSource.onFailuresWithBackoff(
|
||||
RestartSettings.create(Duration.ofSeconds(1), Duration.ofSeconds(10), 0.1),
|
||||
() -> finiteSource);
|
||||
forever.runWith(Sink.foreach(System.out::println), system);
|
||||
// prints
|
||||
// tick
|
||||
// tick
|
||||
// tick
|
||||
// #restart-failure-inner-complete
|
||||
}
|
||||
|
||||
public static void onRestartWitFailureKillSwitch() {
|
||||
// #restart-failure-inner-complete-kill-switch
|
||||
Source<Creator<Integer>, NotUsed> flakySource =
|
||||
Source.from(
|
||||
Arrays.<Creator<Integer>>asList(
|
||||
() -> 1,
|
||||
() -> 2,
|
||||
() -> 3,
|
||||
() -> {
|
||||
throw new RuntimeException("darn");
|
||||
}));
|
||||
UniqueKillSwitch stopRestarting =
|
||||
RestartSource.onFailuresWithBackoff(
|
||||
RestartSettings.create(Duration.ofSeconds(1), Duration.ofSeconds(10), 0.1),
|
||||
() -> flakySource)
|
||||
.viaMat(KillSwitches.single(), Keep.right())
|
||||
.toMat(Sink.foreach(nr -> System.out.println("nr " + nr.create())), Keep.left())
|
||||
.run(system);
|
||||
// ... from some where else
|
||||
// stop the source from restarting
|
||||
stopRestarting.shutdown();
|
||||
// #restart-failure-inner-complete-kill-switch
|
||||
}
|
||||
}
|
||||
76
docs/src/test/java/jdocs/stream/operators/source/Tick.java
Normal file
76
docs/src/test/java/jdocs/stream/operators/source/Tick.java
Normal file
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.source;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.Cancellable;
|
||||
import org.apache.pekko.actor.typed.ActorRef;
|
||||
import org.apache.pekko.actor.typed.ActorSystem;
|
||||
import org.apache.pekko.actor.typed.javadsl.AskPattern;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.time.Duration;
|
||||
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
public class Tick {
|
||||
|
||||
// not really a runnable example, these are just pretend
|
||||
private ActorSystem<Void> system = null;
|
||||
private ActorRef<MyActor.Command> myActor = null;
|
||||
|
||||
static class MyActor {
|
||||
interface Command {}
|
||||
|
||||
static class Query implements Command {
|
||||
public final ActorRef<Response> replyTo;
|
||||
|
||||
public Query(ActorRef<Response> replyTo) {
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
}
|
||||
|
||||
static class Response {
|
||||
public final String text;
|
||||
|
||||
public Response(String text) {
|
||||
this.text = text;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void simple() {
|
||||
// #simple
|
||||
Source.tick(
|
||||
Duration.ofSeconds(1), // delay of first tick
|
||||
Duration.ofSeconds(1), // delay of subsequent ticks
|
||||
"tick" // element emitted each tick
|
||||
)
|
||||
.runForeach(System.out::println, system);
|
||||
// #simple
|
||||
}
|
||||
|
||||
void pollSomething() {
|
||||
// #poll-actor
|
||||
Source<String, Cancellable> periodicActorResponse =
|
||||
Source.tick(Duration.ofSeconds(1), Duration.ofSeconds(1), "tick")
|
||||
.mapAsync(
|
||||
1,
|
||||
notUsed -> {
|
||||
CompletionStage<MyActor.Response> response =
|
||||
AskPattern.ask(
|
||||
myActor, MyActor.Query::new, Duration.ofSeconds(3), system.scheduler());
|
||||
return response;
|
||||
})
|
||||
.map(response -> response.text);
|
||||
// #poll-actor
|
||||
|
||||
// #zip-latest
|
||||
Flow<Integer, Pair<Integer, String>, NotUsed> zipWithLatestResponse =
|
||||
Flow.of(Integer.class).zipLatest(periodicActorResponse);
|
||||
// #zip-latest
|
||||
}
|
||||
}
|
||||
40
docs/src/test/java/jdocs/stream/operators/source/Unfold.java
Normal file
40
docs/src/test/java/jdocs/stream/operators/source/Unfold.java
Normal file
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.source;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.math.BigInteger;
|
||||
import java.util.Optional;
|
||||
|
||||
interface Unfold {
|
||||
|
||||
// #countdown
|
||||
public static Source<Integer, NotUsed> countDown(Integer from) {
|
||||
return Source.unfold(
|
||||
from,
|
||||
current -> {
|
||||
if (current == 0) return Optional.empty();
|
||||
else return Optional.of(Pair.create(current - 1, current));
|
||||
});
|
||||
}
|
||||
// #countdown
|
||||
|
||||
// #fibonacci
|
||||
public static Source<BigInteger, NotUsed> fibonacci() {
|
||||
return Source.unfold(
|
||||
Pair.create(BigInteger.ZERO, BigInteger.ONE),
|
||||
current -> {
|
||||
BigInteger a = current.first();
|
||||
BigInteger b = current.second();
|
||||
Pair<BigInteger, BigInteger> next = Pair.create(b, a.add(b));
|
||||
return Optional.of(Pair.create(next, a));
|
||||
});
|
||||
}
|
||||
// #fibonacci
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,74 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.source;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.typed.ActorRef;
|
||||
import org.apache.pekko.actor.typed.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.util.ByteString;
|
||||
import org.apache.pekko.actor.typed.javadsl.AskPattern;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
interface UnfoldAsync {
|
||||
|
||||
// #unfoldAsync-actor-protocol
|
||||
class DataActor {
|
||||
interface Command {}
|
||||
|
||||
static final class FetchChunk implements Command {
|
||||
public final long offset;
|
||||
public final ActorRef<Chunk> replyTo;
|
||||
|
||||
public FetchChunk(long offset, ActorRef<Chunk> replyTo) {
|
||||
this.offset = offset;
|
||||
this.replyTo = replyTo;
|
||||
}
|
||||
}
|
||||
|
||||
static final class Chunk {
|
||||
public final ByteString bytes;
|
||||
|
||||
public Chunk(ByteString bytes) {
|
||||
this.bytes = bytes;
|
||||
}
|
||||
}
|
||||
// #unfoldAsync-actor-protocol
|
||||
}
|
||||
|
||||
default void unfoldAsyncSample() {
|
||||
ActorSystem<Void> system = null;
|
||||
// #unfoldAsync
|
||||
ActorRef<DataActor.Command> dataActor = null; // let's say we got it from somewhere
|
||||
|
||||
Duration askTimeout = Duration.ofSeconds(3);
|
||||
long startOffset = 0L;
|
||||
Source<ByteString, NotUsed> byteSource =
|
||||
Source.unfoldAsync(
|
||||
startOffset,
|
||||
currentOffset -> {
|
||||
// ask for next chunk
|
||||
CompletionStage<DataActor.Chunk> nextChunkCS =
|
||||
AskPattern.ask(
|
||||
dataActor,
|
||||
(ActorRef<DataActor.Chunk> ref) ->
|
||||
new DataActor.FetchChunk(currentOffset, ref),
|
||||
askTimeout,
|
||||
system.scheduler());
|
||||
|
||||
return nextChunkCS.thenApply(
|
||||
chunk -> {
|
||||
ByteString bytes = chunk.bytes;
|
||||
if (bytes.isEmpty()) return Optional.empty();
|
||||
else return Optional.of(Pair.create(currentOffset + bytes.size(), bytes));
|
||||
});
|
||||
});
|
||||
// #unfoldAsync
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,55 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.source;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.Optional;
|
||||
|
||||
interface UnfoldResource {
|
||||
// imaginary blocking API we need to use
|
||||
// #unfoldResource-blocking-api
|
||||
interface Database {
|
||||
// blocking query
|
||||
QueryResult doQuery();
|
||||
}
|
||||
|
||||
interface QueryResult {
|
||||
boolean hasMore();
|
||||
// potentially blocking retrieval of each element
|
||||
DatabaseEntry nextEntry();
|
||||
|
||||
void close();
|
||||
}
|
||||
|
||||
interface DatabaseEntry {}
|
||||
|
||||
// #unfoldResource-blocking-api
|
||||
|
||||
default void unfoldResourceExample() {
|
||||
ActorSystem system = null;
|
||||
|
||||
// #unfoldResource
|
||||
// we don't actually have one, it was just made up for the sample
|
||||
Database database = null;
|
||||
|
||||
Source<DatabaseEntry, NotUsed> queryResultSource =
|
||||
Source.unfoldResource(
|
||||
// open
|
||||
() -> database.doQuery(),
|
||||
// read
|
||||
(queryResult) -> {
|
||||
if (queryResult.hasMore()) return Optional.of(queryResult.nextEntry());
|
||||
else return Optional.empty();
|
||||
},
|
||||
// close
|
||||
QueryResult::close);
|
||||
|
||||
queryResultSource.runForeach(entry -> System.out.println(entry.toString()), system);
|
||||
// #unfoldResource
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,73 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.source;
|
||||
|
||||
import org.apache.pekko.Done;
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
public class UnfoldResourceAsync {
|
||||
// imaginary async API we need to use
|
||||
// #unfoldResource-async-api
|
||||
interface Database {
|
||||
// async query
|
||||
CompletionStage<QueryResult> doQuery();
|
||||
}
|
||||
|
||||
interface QueryResult {
|
||||
|
||||
// are there more results
|
||||
CompletionStage<Boolean> hasMore();
|
||||
|
||||
// async retrieval of each element
|
||||
CompletionStage<DatabaseEntry> nextEntry();
|
||||
|
||||
CompletionStage<Void> close();
|
||||
}
|
||||
|
||||
interface DatabaseEntry {}
|
||||
|
||||
// #unfoldResource-async-api
|
||||
|
||||
void unfoldResourceExample() {
|
||||
ActorSystem system = null;
|
||||
|
||||
// #unfoldResourceAsync
|
||||
// we don't actually have one, it was just made up for the sample
|
||||
Database database = null;
|
||||
|
||||
Source<DatabaseEntry, NotUsed> queryResultSource =
|
||||
Source.unfoldResourceAsync(
|
||||
// open
|
||||
database::doQuery,
|
||||
// read
|
||||
this::readQueryResult,
|
||||
// close
|
||||
queryResult -> queryResult.close().thenApply(__ -> Done.done()));
|
||||
|
||||
queryResultSource.runForeach(entry -> System.out.println(entry.toString()), system);
|
||||
// #unfoldResourceAsync
|
||||
}
|
||||
|
||||
// #unfoldResourceAsync
|
||||
private CompletionStage<Optional<DatabaseEntry>> readQueryResult(QueryResult queryResult) {
|
||||
return queryResult
|
||||
.hasMore()
|
||||
.thenCompose(
|
||||
more -> {
|
||||
if (more) {
|
||||
return queryResult.nextEntry().thenApply(Optional::of);
|
||||
} else {
|
||||
return CompletableFuture.completedFuture(Optional.empty());
|
||||
}
|
||||
});
|
||||
}
|
||||
// #unfoldResourceAsync
|
||||
}
|
||||
73
docs/src/test/java/jdocs/stream/operators/source/Zip.java
Normal file
73
docs/src/test/java/jdocs/stream/operators/source/Zip.java
Normal file
|
|
@ -0,0 +1,73 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.source;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
public class Zip {
|
||||
|
||||
void zipNSample() {
|
||||
ActorSystem system = null;
|
||||
|
||||
// #zipN-simple
|
||||
Source<Object, NotUsed> chars = Source.from(Arrays.asList("a", "b", "c", "e", "f"));
|
||||
Source<Object, NotUsed> numbers = Source.from(Arrays.asList(1, 2, 3, 4, 5, 6));
|
||||
Source<Object, NotUsed> colors =
|
||||
Source.from(Arrays.asList("red", "green", "blue", "yellow", "purple"));
|
||||
|
||||
Source.zipN(Arrays.asList(chars, numbers, colors)).runForeach(System.out::println, system);
|
||||
// prints:
|
||||
// [a, 1, red]
|
||||
// [b, 2, green]
|
||||
// [c, 3, blue]
|
||||
// [e, 4, yellow]
|
||||
// [f, 5, purple]
|
||||
|
||||
// #zipN-simple
|
||||
}
|
||||
|
||||
void zipWithNSample() {
|
||||
ActorSystem system = null;
|
||||
|
||||
// #zipWithN-simple
|
||||
Source<Integer, NotUsed> numbers = Source.from(Arrays.asList(1, 2, 3, 4, 5, 6));
|
||||
Source<Integer, NotUsed> otherNumbers = Source.from(Arrays.asList(5, 2, 1, 4, 10, 4));
|
||||
Source<Integer, NotUsed> andSomeOtherNumbers = Source.from(Arrays.asList(3, 7, 2, 1, 1));
|
||||
|
||||
Source.zipWithN(
|
||||
(List<Integer> seq) -> seq.stream().mapToInt(i -> i).max().getAsInt(),
|
||||
Arrays.asList(numbers, otherNumbers, andSomeOtherNumbers))
|
||||
.runForeach(System.out::println, system);
|
||||
// prints:
|
||||
// 5
|
||||
// 7
|
||||
// 3
|
||||
// 4
|
||||
// 10
|
||||
|
||||
// #zipWithN-simple
|
||||
}
|
||||
|
||||
void zipAllSample() {
|
||||
ActorSystem system = null;
|
||||
// #zipAll-simple
|
||||
|
||||
Source<Integer, NotUsed> numbers = Source.from(Arrays.asList(1, 2, 3, 4));
|
||||
Source<String, NotUsed> letters = Source.from(Arrays.asList("a", "b", "c"));
|
||||
|
||||
numbers.zipAll(letters, -1, "default").runForeach(System.out::println, system);
|
||||
// prints:
|
||||
// Pair(1,a)
|
||||
// Pair(2,b)
|
||||
// Pair(3,c)
|
||||
// Pair(4,default)
|
||||
// #zipAll-simple
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,99 @@
|
|||
/*
|
||||
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.sourceorflow;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.actor.Cancellable;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.japi.function.Function;
|
||||
import org.apache.pekko.stream.javadsl.Flow;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.util.ByteString;
|
||||
import docs.stream.operators.sourceorflow.ExtrapolateAndExpandCommon;
|
||||
import docs.stream.operators.sourceorflow.ExtrapolateAndExpandCommon.Frame;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/** */
|
||||
public class ExtrapolateAndExpand {
|
||||
public static Function<ByteString, Frame> decodeAsFrame =
|
||||
ExtrapolateAndExpandCommon.Frame$.MODULE$::decode;
|
||||
|
||||
public static Frame BLACK_FRAME = ExtrapolateAndExpandCommon.Frame$.MODULE$.blackFrame();
|
||||
|
||||
public static long nowInSeconds() {
|
||||
return ExtrapolateAndExpand.nowInSeconds();
|
||||
}
|
||||
|
||||
public static void main(String[] args) {
|
||||
ActorSystem actorSystem = ActorSystem.create("25fps-stream");
|
||||
|
||||
Source<ByteString, NotUsed> networkSource = ExtrapolateAndExpandCommon.networkSource().asJava();
|
||||
|
||||
Flow<ByteString, Frame, NotUsed> decode = Flow.of(ByteString.class).<Frame>map(decodeAsFrame);
|
||||
|
||||
// #extrapolate
|
||||
// if upstream is too slow, produce copies of the last frame but grayed out.
|
||||
Flow<Frame, Frame, NotUsed> rateControl =
|
||||
Flow.of(Frame.class)
|
||||
.extrapolate(
|
||||
lastFrame -> {
|
||||
Frame gray =
|
||||
new Frame(
|
||||
ByteString.fromString(
|
||||
"gray frame!! - " + lastFrame.pixels().utf8String()));
|
||||
return Stream.iterate(gray, i -> i).iterator();
|
||||
},
|
||||
BLACK_FRAME // initial value
|
||||
);
|
||||
|
||||
Source<Frame, NotUsed> videoSource = networkSource.via(decode).via(rateControl);
|
||||
|
||||
// let's create a 25fps stream (a Frame every 40.millis)
|
||||
Source<String, Cancellable> tickSource =
|
||||
Source.tick(Duration.ZERO, Duration.ofMillis(40), "tick");
|
||||
|
||||
Source<Frame, Cancellable> videoAt25Fps = tickSource.zip(videoSource).map(Pair::second);
|
||||
|
||||
// #extrapolate
|
||||
|
||||
// #expand
|
||||
// each element flowing through the stream is expanded to a watermark copy
|
||||
// of the upstream frame and grayed out copies. The grayed out copies should
|
||||
// only be used downstream if the producer is too slow.
|
||||
Flow<Frame, Frame, NotUsed> watermarkerRateControl =
|
||||
Flow.of(Frame.class)
|
||||
.expand(
|
||||
lastFrame -> {
|
||||
Frame watermarked =
|
||||
new Frame(
|
||||
lastFrame.pixels().$plus$plus(ByteString.fromString(" - watermark")));
|
||||
Frame gray =
|
||||
new Frame(lastFrame.pixels().$plus$plus(ByteString.fromString(" - gray")));
|
||||
return Stream.concat(Stream.of(watermarked), Stream.iterate(gray, i -> i))
|
||||
.iterator();
|
||||
});
|
||||
|
||||
Source<Frame, NotUsed> watermakedVideoSource =
|
||||
networkSource.via(decode).via(watermarkerRateControl);
|
||||
|
||||
// let's create a 25fps stream (a Frame every 40.millis)
|
||||
Source<String, Cancellable> ticks = Source.tick(Duration.ZERO, Duration.ofMillis(40), "tick");
|
||||
|
||||
Source<Frame, Cancellable> watermarkedVideoAt25Fps =
|
||||
ticks.zip(watermakedVideoSource).map(Pair::second);
|
||||
|
||||
// #expand
|
||||
videoAt25Fps
|
||||
.map(Frame::pixels)
|
||||
.map(ByteString::utf8String)
|
||||
.map(pixels -> nowInSeconds() + " - " + pixels)
|
||||
.to(Sink.foreach(System.out::println))
|
||||
.run(actorSystem);
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,35 @@
|
|||
/*
|
||||
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.sourceorflow;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class FlatMapConcat {
|
||||
private static ActorSystem system = null;
|
||||
|
||||
// #flatmap-concat
|
||||
// e.g. could be a query to a database
|
||||
private Source<String, NotUsed> lookupCustomerEvents(String customerId) {
|
||||
return Source.from(Arrays.asList(customerId + "-event-1", customerId + "-event-2"));
|
||||
}
|
||||
// #flatmap-concat
|
||||
|
||||
void example() {
|
||||
// #flatmap-concat
|
||||
Source.from(Arrays.asList("customer-1", "customer-2"))
|
||||
.flatMapConcat(this::lookupCustomerEvents)
|
||||
.runForeach(System.out::println, system);
|
||||
// prints - events from each customer consecutively
|
||||
// customer-1-event-1
|
||||
// customer-1-event-2
|
||||
// customer-2-event-1
|
||||
// customer-2-event-2
|
||||
// #flatmap-concat
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,35 @@
|
|||
/*
|
||||
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.sourceorflow;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class FlatMapMerge {
|
||||
private static ActorSystem system = null;
|
||||
|
||||
// #flatmap-merge
|
||||
// e.g. could be a query to a database
|
||||
private Source<String, NotUsed> lookupCustomerEvents(String customerId) {
|
||||
return Source.from(Arrays.asList(customerId + "-evt-1", customerId + "-evt-2"));
|
||||
}
|
||||
// #flatmap-merge
|
||||
|
||||
void example() {
|
||||
// #flatmap-merge
|
||||
Source.from(Arrays.asList("customer-1", "customer-2"))
|
||||
.flatMapMerge(10, this::lookupCustomerEvents)
|
||||
.runForeach(System.out::println, system);
|
||||
// prints - events from different customers could interleave
|
||||
// customer-1-evt-1
|
||||
// customer-2-evt-1
|
||||
// customer-1-evt-2
|
||||
// customer-2-evt-2
|
||||
// #flatmap-merge
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,25 @@
|
|||
/*
|
||||
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.sourceorflow;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class Intersperse {
|
||||
public static void main(String[] args) {
|
||||
ActorSystem system = ActorSystem.create();
|
||||
// #intersperse
|
||||
Source.from(Arrays.asList(1, 2, 3))
|
||||
.map(String::valueOf)
|
||||
.intersperse("[", ", ", "]")
|
||||
.runForeach(System.out::print, system);
|
||||
// prints
|
||||
// [1, 2, 3]
|
||||
// #intersperse
|
||||
system.terminate();
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,25 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.sourceorflow;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.typed.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
public class Limit {
|
||||
public void simple() {
|
||||
ActorSystem<?> system = null;
|
||||
// #simple
|
||||
Source<String, NotUsed> untrustedSource = Source.repeat("element");
|
||||
|
||||
CompletionStage<List<String>> elements =
|
||||
untrustedSource.limit(10000).runWith(Sink.seq(), system);
|
||||
// #simple
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,29 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.sourceorflow;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.typed.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
import org.apache.pekko.util.ByteString;
|
||||
|
||||
import java.util.concurrent.CompletionStage;
|
||||
|
||||
public class LimitWeighted {
|
||||
public void simple() {
|
||||
ActorSystem<?> system = null;
|
||||
// #simple
|
||||
Source<ByteString, NotUsed> untrustedSource = Source.repeat(ByteString.fromString("element"));
|
||||
|
||||
CompletionStage<ByteString> allBytes =
|
||||
untrustedSource
|
||||
.limitWeighted(
|
||||
10000, // max bytes
|
||||
bytes -> (long) bytes.length() // bytes of each chunk
|
||||
)
|
||||
.runReduce(ByteString::concat, system);
|
||||
// #simple
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,118 @@
|
|||
/*
|
||||
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.sourceorflow;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.pattern.Patterns;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class MapAsyncs {
|
||||
|
||||
private final Random random = new Random();
|
||||
|
||||
// #mapasync-strict-order
|
||||
// #mapasync-concurrent
|
||||
// #mapasyncunordered
|
||||
|
||||
private final Source<Event, NotUsed> events =
|
||||
Source.fromIterator(() -> Stream.iterate(1, i -> i + 1).iterator())
|
||||
.throttle(1, Duration.ofMillis(50))
|
||||
.map(Event::new);
|
||||
// #mapasync-strict-order
|
||||
// #mapasync-concurrent
|
||||
// #mapasyncunordered
|
||||
|
||||
private final ActorSystem system = ActorSystem.create("mapAsync-operator-examples");
|
||||
|
||||
public MapAsyncs() {}
|
||||
|
||||
// #mapasync-strict-order
|
||||
// #mapasync-concurrent
|
||||
// #mapasyncunordered
|
||||
|
||||
public CompletionStage<Integer> eventHandler(Event in) throws InterruptedException {
|
||||
System.out.println("Processing event number " + in + "...");
|
||||
// ...
|
||||
// #mapasync-strict-order
|
||||
// #mapasync-concurrent
|
||||
// #mapasyncunordered
|
||||
CompletionStage<Integer> cs;
|
||||
if (random.nextInt(5) == 0) {
|
||||
cs =
|
||||
Patterns.after(
|
||||
Duration.ofMillis(500),
|
||||
system,
|
||||
() -> CompletableFuture.completedFuture(in.sequenceNumber));
|
||||
} else {
|
||||
cs = CompletableFuture.completedFuture(in.sequenceNumber);
|
||||
}
|
||||
return cs.thenApply(
|
||||
i -> {
|
||||
System.out.println("Completed processing " + i.intValue());
|
||||
return i;
|
||||
});
|
||||
// #mapasync-strict-order
|
||||
// #mapasync-concurrent
|
||||
// #mapasyncunordered
|
||||
}
|
||||
// #mapasync-strict-order
|
||||
// #mapasync-concurrent
|
||||
// #mapasyncunordered
|
||||
|
||||
private void runStrictOrder() {
|
||||
// #mapasync-strict-order
|
||||
|
||||
events
|
||||
.mapAsync(1, this::eventHandler)
|
||||
.map(in -> "`mapSync` emitted event number " + in.intValue())
|
||||
.runWith(Sink.foreach(str -> System.out.println(str)), system);
|
||||
// #mapasync-strict-order
|
||||
}
|
||||
|
||||
private void run() {
|
||||
// #mapasync-concurrent
|
||||
|
||||
events
|
||||
.mapAsync(10, this::eventHandler)
|
||||
.map(in -> "`mapSync` emitted event number " + in.intValue())
|
||||
.runWith(Sink.foreach(str -> System.out.println(str)), system);
|
||||
// #mapasync-concurrent
|
||||
}
|
||||
|
||||
private void runUnordered() {
|
||||
// #mapasyncunordered
|
||||
|
||||
events
|
||||
.mapAsyncUnordered(10, this::eventHandler)
|
||||
.map(in -> "`mapSync` emitted event number " + in.intValue())
|
||||
.runWith(Sink.foreach(str -> System.out.println(str)), system);
|
||||
// #mapasyncunordered
|
||||
}
|
||||
|
||||
public static void main(String[] args) {
|
||||
new MapAsyncs().run();
|
||||
}
|
||||
|
||||
static class Event {
|
||||
public final int sequenceNumber;
|
||||
|
||||
public Event(int sequenceNumber) {
|
||||
this.sequenceNumber = sequenceNumber;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Event(" + sequenceNumber + ')';
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,36 @@
|
|||
/*
|
||||
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.sourceorflow;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class MapConcat {
|
||||
private static ActorSystem system = null;
|
||||
|
||||
// #map-concat
|
||||
Iterable<Integer> duplicate(int i) {
|
||||
return Arrays.asList(i, i);
|
||||
}
|
||||
|
||||
// #map-concat
|
||||
|
||||
void example() {
|
||||
// #map-concat
|
||||
Source.from(Arrays.asList(1, 2, 3))
|
||||
.mapConcat(i -> duplicate(i))
|
||||
.runForeach(System.out::println, system);
|
||||
// prints:
|
||||
// 1
|
||||
// 1
|
||||
// 2
|
||||
// 2
|
||||
// 3
|
||||
// 3
|
||||
// #map-concat
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,36 @@
|
|||
/*
|
||||
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.sourceorflow;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class MapError {
|
||||
|
||||
public static void main(String[] args) {
|
||||
|
||||
// #map-error
|
||||
|
||||
final ActorSystem system = ActorSystem.create("mapError-operator-example");
|
||||
Source.from(Arrays.asList(-1, 0, 1))
|
||||
.map(x -> 1 / x)
|
||||
.mapError(
|
||||
ArithmeticException.class,
|
||||
(ArithmeticException e) ->
|
||||
new UnsupportedOperationException("Divide by Zero Operation is not supported."))
|
||||
.runWith(Sink.seq(), system)
|
||||
.whenComplete(
|
||||
(result, exception) -> {
|
||||
if (result != null) System.out.println(result.toString());
|
||||
else System.out.println(exception.getMessage());
|
||||
});
|
||||
|
||||
// prints "Divide by Zero Operation is not supported."
|
||||
// #map-error
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,37 @@
|
|||
/*
|
||||
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.sourceorflow;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.typed.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class MergeLatest {
|
||||
|
||||
private static final ActorSystem<Void> system = null;
|
||||
|
||||
public static void example() {
|
||||
// #mergeLatest
|
||||
Source<Integer, NotUsed> prices = Source.from(Arrays.asList(100, 101, 99, 103));
|
||||
Source<Integer, NotUsed> quantities = Source.from(Arrays.asList(1, 3, 4, 2));
|
||||
|
||||
prices
|
||||
.mergeLatest(quantities, true)
|
||||
.map(priceAndQuantity -> priceAndQuantity.get(0) * priceAndQuantity.get(1))
|
||||
.runForeach(System.out::println, system);
|
||||
|
||||
// prints something like:
|
||||
// 100
|
||||
// 101
|
||||
// 303
|
||||
// 297
|
||||
// 396
|
||||
// 412
|
||||
// 206
|
||||
// #mergeLatest
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,66 @@
|
|||
/*
|
||||
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.sourceorflow;
|
||||
|
||||
import org.apache.pekko.Done;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.stream.FlowMonitor;
|
||||
import org.apache.pekko.stream.FlowMonitorState;
|
||||
import org.apache.pekko.stream.javadsl.Keep;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
/** */
|
||||
public class Monitor {
|
||||
|
||||
// #monitor
|
||||
private static <T> void printMonitorState(FlowMonitorState.StreamState<T> state) {
|
||||
if (state == FlowMonitorState.finished()) {
|
||||
System.out.println("Stream is initialized but hasn't processed any element");
|
||||
} else if (state instanceof FlowMonitorState.Received) {
|
||||
FlowMonitorState.Received msg = (FlowMonitorState.Received) state;
|
||||
System.out.println("Last message received: " + msg.msg());
|
||||
} else if (state instanceof FlowMonitorState.Failed) {
|
||||
Throwable cause = ((FlowMonitorState.Failed) state).cause();
|
||||
System.out.println("Stream failed with cause: " + cause.getMessage());
|
||||
} else {
|
||||
System.out.println("Stream completed already");
|
||||
}
|
||||
}
|
||||
// #monitor
|
||||
|
||||
public static void main(String[] args)
|
||||
throws InterruptedException, TimeoutException, ExecutionException {
|
||||
ActorSystem actorSystem = ActorSystem.create("25fps-stream");
|
||||
|
||||
// #monitor
|
||||
Source<Integer, FlowMonitor<Integer>> monitoredSource =
|
||||
Source.fromIterator(() -> Arrays.asList(0, 1, 2, 3, 4, 5).iterator())
|
||||
.throttle(5, Duration.ofSeconds(1))
|
||||
.monitorMat(Keep.right());
|
||||
|
||||
Pair<FlowMonitor<Integer>, CompletionStage<Done>> run =
|
||||
monitoredSource.toMat(Sink.foreach(System.out::println), Keep.both()).run(actorSystem);
|
||||
|
||||
FlowMonitor<Integer> monitor = run.first();
|
||||
|
||||
// If we peek in the monitor too early, it's possible it was not initialized yet.
|
||||
printMonitorState(monitor.state());
|
||||
|
||||
// Periodically check the monitor
|
||||
Source.tick(Duration.ofMillis(200), Duration.ofMillis(400), "")
|
||||
.runForeach(__ -> printMonitorState(monitor.state()), actorSystem);
|
||||
// #monitor
|
||||
|
||||
run.second().toCompletableFuture().whenComplete((x, t) -> actorSystem.terminate());
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,55 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.sourceorflow;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.typed.ActorSystem;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class Sliding {
|
||||
|
||||
private final ActorSystem<Void> system = null;
|
||||
|
||||
public void slidingExample1() {
|
||||
// #sliding-1
|
||||
Source<Integer, NotUsed> source = Source.range(1, 4);
|
||||
source.sliding(2, 1).runForeach(n -> System.out.println(n), system);
|
||||
// prints:
|
||||
// [1, 2]
|
||||
// [2, 3]
|
||||
// [3, 4]
|
||||
// #sliding-1
|
||||
}
|
||||
|
||||
public void slidingExample2() {
|
||||
// #sliding-2
|
||||
Source<Integer, NotUsed> source = Source.range(1, 4);
|
||||
source.sliding(3, 2).runForeach(n -> System.out.println(n), system);
|
||||
// prints:
|
||||
// Vector(1, 2, 3)
|
||||
// [1, 2, 3]
|
||||
// [3, 4] - shorter because stream ended before we got 3 elements
|
||||
// #sliding-2
|
||||
}
|
||||
|
||||
public void slidingExample3() {
|
||||
// #moving-average
|
||||
Source<Integer, NotUsed> numbers = Source.from(Arrays.asList(1, 3, 10, 2, 3, 4, 2, 10, 11));
|
||||
Source<Float, NotUsed> movingAverage =
|
||||
numbers
|
||||
.sliding(5, 1)
|
||||
.map(window -> ((float) window.stream().mapToInt(i -> i).sum()) / window.size());
|
||||
movingAverage.runForeach(n -> System.out.println(n), system);
|
||||
// prints
|
||||
// 3.8 = average of 1, 3, 10, 2, 3
|
||||
// 4.4 = average of 3, 10, 2, 3, 4
|
||||
// 4.2 = average of 10, 2, 3, 4, 2
|
||||
// 4.2 = average of 2, 3, 4, 2, 10
|
||||
// 6.0 = average of 3, 4, 2, 10, 11
|
||||
// #moving-average
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,107 @@
|
|||
/*
|
||||
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.sourceorflow;
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.japi.Pair;
|
||||
import org.apache.pekko.japi.function.Function;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.time.LocalDateTime;
|
||||
import java.time.ZoneOffset;
|
||||
import java.util.Collections;
|
||||
|
||||
public class Split {
|
||||
public static void splitWhenExample(String[] args) {
|
||||
ActorSystem system = ActorSystem.create();
|
||||
|
||||
// #splitWhen
|
||||
Source.range(1, 100)
|
||||
.throttle(1, Duration.ofMillis(100))
|
||||
.map(elem -> new Pair<>(elem, Instant.now()))
|
||||
.statefulMapConcat(
|
||||
() -> {
|
||||
return new Function<Pair<Integer, Instant>, Iterable<Pair<Integer, Boolean>>>() {
|
||||
// stateful decision in statefulMapConcat
|
||||
// keep track of time bucket (one per second)
|
||||
LocalDateTime currentTimeBucket =
|
||||
LocalDateTime.ofInstant(Instant.ofEpochMilli(0), ZoneOffset.UTC);
|
||||
|
||||
@Override
|
||||
public Iterable<Pair<Integer, Boolean>> apply(
|
||||
Pair<Integer, Instant> elemTimestamp) {
|
||||
LocalDateTime time =
|
||||
LocalDateTime.ofInstant(elemTimestamp.second(), ZoneOffset.UTC);
|
||||
LocalDateTime bucket = time.withNano(0);
|
||||
boolean newBucket = !bucket.equals(currentTimeBucket);
|
||||
if (newBucket) currentTimeBucket = bucket;
|
||||
return Collections.singleton(new Pair<>(elemTimestamp.first(), newBucket));
|
||||
}
|
||||
};
|
||||
})
|
||||
.splitWhen(elemDecision -> elemDecision.second()) // split when time bucket changes
|
||||
.map(elemDecision -> elemDecision.first())
|
||||
.fold(0, (acc, notUsed) -> acc + 1) // sum
|
||||
.to(Sink.foreach(System.out::println))
|
||||
.run(system);
|
||||
// 3
|
||||
// 10
|
||||
// 10
|
||||
// 10
|
||||
// 10
|
||||
// 10
|
||||
// 10
|
||||
// 10
|
||||
// 10
|
||||
// 10
|
||||
// 7
|
||||
// #splitWhen
|
||||
}
|
||||
|
||||
public static void splitAfterExample(String[] args) {
|
||||
ActorSystem system = ActorSystem.create();
|
||||
|
||||
// #splitAfter
|
||||
Source.range(1, 100)
|
||||
.throttle(1, Duration.ofMillis(100))
|
||||
.map(elem -> new Pair<>(elem, Instant.now()))
|
||||
.sliding(2, 1)
|
||||
.splitAfter(
|
||||
slidingElements -> {
|
||||
if (slidingElements.size() == 2) {
|
||||
Pair<Integer, Instant> current = slidingElements.get(0);
|
||||
Pair<Integer, Instant> next = slidingElements.get(1);
|
||||
LocalDateTime currentBucket =
|
||||
LocalDateTime.ofInstant(current.second(), ZoneOffset.UTC).withNano(0);
|
||||
LocalDateTime nextBucket =
|
||||
LocalDateTime.ofInstant(next.second(), ZoneOffset.UTC).withNano(0);
|
||||
return !currentBucket.equals(nextBucket);
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
})
|
||||
.map(slidingElements -> slidingElements.get(0).first())
|
||||
.fold(0, (acc, notUsed) -> acc + 1) // sum
|
||||
.to(Sink.foreach(System.out::println))
|
||||
.run(system);
|
||||
// 3
|
||||
// 10
|
||||
// 10
|
||||
// 10
|
||||
// 10
|
||||
// 10
|
||||
// 10
|
||||
// 10
|
||||
// 10
|
||||
// 10
|
||||
// 6
|
||||
// note that the very last element is never included due to sliding,
|
||||
// but that would not be problem for an infinite stream
|
||||
// #splitAfter
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,47 @@
|
|||
/*
|
||||
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package jdocs.stream.operators.sourceorflow;
|
||||
|
||||
import org.apache.pekko.NotUsed;
|
||||
import org.apache.pekko.actor.ActorSystem;
|
||||
import org.apache.pekko.stream.Materializer;
|
||||
import org.apache.pekko.stream.ThrottleMode;
|
||||
import org.apache.pekko.stream.javadsl.Sink;
|
||||
import org.apache.pekko.stream.javadsl.Source;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.stream.Stream;
|
||||
import docs.stream.operators.sourceorflow.ThrottleCommon.Frame;
|
||||
|
||||
/** */
|
||||
public class Throttle {
|
||||
|
||||
public static void main(String[] args) {
|
||||
ActorSystem actorSystem = ActorSystem.create("25fps-throttled-stream");
|
||||
Materializer mat = Materializer.matFromSystem(actorSystem);
|
||||
|
||||
Source<Frame, NotUsed> frameSource =
|
||||
Source.fromIterator(() -> Stream.iterate(0, i -> i + 1).iterator())
|
||||
.map(i -> new Frame(i.intValue()));
|
||||
|
||||
// #throttle
|
||||
int framesPerSecond = 24;
|
||||
|
||||
Source<Frame, NotUsed> videoThrottling =
|
||||
frameSource.throttle(framesPerSecond, Duration.ofSeconds(1));
|
||||
// serialize `Frame` and send over the network.
|
||||
// #throttle
|
||||
|
||||
// #throttle-with-burst
|
||||
Source<Frame, NotUsed> throttlingWithBurst =
|
||||
frameSource.throttle(
|
||||
framesPerSecond, Duration.ofSeconds(1), framesPerSecond * 30, ThrottleMode.shaping());
|
||||
// serialize `Frame` and send over the network.
|
||||
// #throttle-with-burst
|
||||
|
||||
videoThrottling.map(f -> f.i()).to(Sink.foreach(System.out::println)).run(mat);
|
||||
throttlingWithBurst.take(1000L).map(f -> f.i()).to(Sink.foreach(System.out::println)).run(mat);
|
||||
}
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue