also fix FlowInterleaveSpec

Also-by: Johan Andrén <johan@markatta.com>
Also-by: Roland Kuhn <rk@rkuhn.info>
Also-by: Martynas Mickevičius <mmartynas@gmail.com>
This commit is contained in:
Endre Sándor Varga 2016-01-20 10:00:37 +02:00 committed by Martynas Mickevičius
parent ef77b56e66
commit 60497f6561
195 changed files with 1110 additions and 857 deletions

View file

@ -0,0 +1,19 @@
/**
* Copyright (C) 2016 Typesafe Inc. <http://www.typesafe.com>
*/
package akka
/**
* Typically used together with `Future` to signal completion
* but there is no actual value completed. More clearly signals intent
* than `Unit` and is available both from Scala and Java (which `Unit` is not).
*/
sealed abstract class Done
case object Done extends Done {
/**
* Java API: the singleton instance
*/
def getInstance() = this
}

View file

@ -0,0 +1,19 @@
/**
* Copyright (C) 2016 Typesafe Inc. <http://www.typesafe.com>
*/
package akka
/**
* This type is used in generic type signatures wherever the actual value is of no importance.
* It is a combination of Scalas `Unit` and Javas `Void`, which both have different issues when
* used from the other language. An example use-case is the materialized value of an Akka Stream for cases
* where no result shall be returned from materialization.
*/
sealed abstract class NotUsed
case object NotUsed extends NotUsed {
/**
* Java API: the singleton instance
*/
def getInstance(): NotUsed = this
}

View file

@ -4,6 +4,7 @@
package akka.stream
import akka.{Done, NotUsed}
import akka.actor.ActorSystem
import akka.stream.scaladsl._
import java.util.concurrent.TimeUnit
@ -24,9 +25,9 @@ class FlatMapMergeBenchmark {
@Param(Array("0", "1", "10"))
val NumberOfStreams = 0
var graph: RunnableGraph[Future[Unit]] = _
var graph: RunnableGraph[Future[Done]] = _
def createSource(count: Int): Graph[SourceShape[Int], Unit] = akka.stream.Fusing.aggressive(Source.repeat(1).take(count))
def createSource(count: Int): Graph[SourceShape[Int], NotUsed] = akka.stream.Fusing.aggressive(Source.repeat(1).take(count))
@Setup
def setup() {

View file

@ -5,6 +5,7 @@
package akka.stream
import java.util.concurrent.TimeUnit
import akka.NotUsed
import akka.actor.ActorSystem
import akka.stream.scaladsl._
import com.typesafe.config.ConfigFactory
@ -57,7 +58,7 @@ class FlowMapBenchmark {
final val successFailure = Success(new Exception)
// safe to be benchmark scoped because the flows we construct in this bench are stateless
var flow: Source[Int, Unit] = _
var flow: Source[Int, NotUsed] = _
@Param(Array("8", "32", "128"))
val initialInputBufferSize = 0

View file

@ -5,6 +5,7 @@
package akka.stream
import java.util.concurrent.TimeUnit
import akka.NotUsed
import akka.actor.ActorSystem
import akka.stream.scaladsl._
import org.openjdk.jmh.annotations._
@ -38,7 +39,7 @@ object MaterializationBenchmark {
})
val graphWithNestedImportsBuilder = (numOfNestedGraphs: Int) => {
var flow: Graph[FlowShape[Unit, Unit], Unit] = Flow[Unit].map(identity)
var flow: Graph[FlowShape[Unit, Unit], NotUsed] = Flow[Unit].map(identity)
for (_ <- 1 to numOfNestedGraphs) {
flow = GraphDSL.create(flow) { b
flow
@ -78,10 +79,10 @@ class MaterializationBenchmark {
implicit val system = ActorSystem("MaterializationBenchmark")
implicit val materializer = ActorMaterializer()
var flowWithMap: RunnableGraph[Unit] = _
var graphWithJunctions: RunnableGraph[Unit] = _
var graphWithNestedImports: RunnableGraph[Unit] = _
var graphWithImportedFlow: RunnableGraph[Unit] = _
var flowWithMap: RunnableGraph[NotUsed] = _
var graphWithJunctions: RunnableGraph[NotUsed] = _
var graphWithNestedImports: RunnableGraph[NotUsed] = _
var graphWithImportedFlow: RunnableGraph[NotUsed] = _
@Param(Array("1", "10", "100", "1000"))
val complexity = 0

View file

@ -7,6 +7,7 @@ package akka.stream.io
import java.io.{FileInputStream, File}
import java.util.concurrent.TimeUnit
import akka.{Done, NotUsed}
import akka.actor.ActorSystem
import akka.stream.{Attributes, ActorMaterializer}
import akka.stream.scaladsl._
@ -47,7 +48,7 @@ class FileSourcesBenchmark {
var fileChannelSource: Source[ByteString, Future[Long]] = _
var fileInputStreamSource: Source[ByteString, Future[Long]] = _
var ioSourceLinesIterator: Source[ByteString, Unit] = _
var ioSourceLinesIterator: Source[ByteString, NotUsed] = _
@Setup
def setup() {
@ -95,7 +96,7 @@ class FileSourcesBenchmark {
*/
@Benchmark
def naive_ioSourceLinesIterator() = {
val p = Promise[Unit]()
val p = Promise[Done]()
ioSourceLinesIterator.to(Sink.onComplete(p.complete(_))).run()
Await.result(p.future, 30.seconds)

View file

@ -4,6 +4,7 @@
package docs.http.javadsl.server;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.dispatch.OnFailure;
import akka.http.impl.util.Util;
@ -98,7 +99,7 @@ public class HttpServerExampleDocTest {
Source<IncomingConnection, Future<ServerBinding>> serverSource =
Http.get(system).bind("localhost", 8080, materializer);
Flow<IncomingConnection, IncomingConnection, BoxedUnit> failureDetection =
Flow<IncomingConnection, IncomingConnection, NotUsed> failureDetection =
Flow.of(IncomingConnection.class).transform(() ->
new PushStage<IncomingConnection, IncomingConnection>() {
@Override
@ -136,7 +137,7 @@ public class HttpServerExampleDocTest {
Source<IncomingConnection, Future<ServerBinding>> serverSource =
Http.get(system).bind("localhost", 8080, materializer);
Flow<HttpRequest, HttpRequest, BoxedUnit> failureDetection =
Flow<HttpRequest, HttpRequest, NotUsed> failureDetection =
Flow.of(HttpRequest.class).transform(() ->
new PushStage<HttpRequest, HttpRequest>() {
@Override
@ -151,7 +152,7 @@ public class HttpServerExampleDocTest {
}
});
Flow<HttpRequest, HttpResponse, BoxedUnit> httpEcho =
Flow<HttpRequest, HttpResponse, NotUsed> httpEcho =
Flow.of(HttpRequest.class)
.via(failureDetection)
.map(request -> {

View file

@ -9,6 +9,7 @@ import java.io.BufferedReader;
import java.io.InputStreamReader;
import java.util.concurrent.TimeUnit;
import akka.NotUsed;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
@ -71,7 +72,7 @@ public class WebsocketCoreExample {
* A handler that treats incoming messages as a name,
* and responds with a greeting to that name
*/
public static Flow<Message, Message, BoxedUnit> greeter() {
public static Flow<Message, Message, NotUsed> greeter() {
return
Flow.<Message>create()
.collect(new JavaPartialFunction<Message, Message>() {

View file

@ -8,6 +8,8 @@ import static akka.pattern.Patterns.ask;
import java.util.HashSet;
import java.util.Set;
import java.util.Iterator;
import akka.NotUsed;
import com.typesafe.config.Config;
import akka.actor.*;
@ -113,27 +115,27 @@ public class PersistenceQueryDocTest {
}
@Override
public Source<EventEnvelope, BoxedUnit> eventsByTag(String tag, long offset) {
public Source<EventEnvelope, NotUsed> eventsByTag(String tag, long offset) {
final Props props = MyEventsByTagPublisher.props(tag, offset, refreshInterval);
return Source.<EventEnvelope>actorPublisher(props).
mapMaterializedValue(m -> BoxedUnit.UNIT);
mapMaterializedValue(m -> NotUsed.getInstance());
}
@Override
public Source<EventEnvelope, BoxedUnit> eventsByPersistenceId(String persistenceId,
public Source<EventEnvelope, NotUsed> eventsByPersistenceId(String persistenceId,
long fromSequenceNr, long toSequenceNr) {
// implement in a similar way as eventsByTag
throw new UnsupportedOperationException("Not implemented yet");
}
@Override
public Source<String, BoxedUnit> allPersistenceIds() {
public Source<String, NotUsed> allPersistenceIds() {
// implement in a similar way as eventsByTag
throw new UnsupportedOperationException("Not implemented yet");
}
@Override
public Source<String, BoxedUnit> currentPersistenceIds() {
public Source<String, NotUsed> currentPersistenceIds() {
// implement in a similar way as eventsByTag
throw new UnsupportedOperationException("Not implemented yet");
}
@ -166,25 +168,25 @@ public class PersistenceQueryDocTest {
}
@Override
public akka.stream.scaladsl.Source<EventEnvelope, BoxedUnit> eventsByTag(
public akka.stream.scaladsl.Source<EventEnvelope, NotUsed> eventsByTag(
String tag, long offset) {
return javadslReadJournal.eventsByTag(tag, offset).asScala();
}
@Override
public akka.stream.scaladsl.Source<EventEnvelope, BoxedUnit> eventsByPersistenceId(
public akka.stream.scaladsl.Source<EventEnvelope, NotUsed> eventsByPersistenceId(
String persistenceId, long fromSequenceNr, long toSequenceNr) {
return javadslReadJournal.eventsByPersistenceId(persistenceId, fromSequenceNr,
toSequenceNr).asScala();
}
@Override
public akka.stream.scaladsl.Source<String, BoxedUnit> allPersistenceIds() {
public akka.stream.scaladsl.Source<String, NotUsed> allPersistenceIds() {
return javadslReadJournal.allPersistenceIds().asScala();
}
@Override
public akka.stream.scaladsl.Source<String, BoxedUnit> currentPersistenceIds() {
public akka.stream.scaladsl.Source<String, NotUsed> currentPersistenceIds() {
return javadslReadJournal.currentPersistenceIds().asScala();
}
@ -209,7 +211,7 @@ public class PersistenceQueryDocTest {
"akka.persistence.query.my-read-journal");
// issue query to journal
Source<EventEnvelope, BoxedUnit> source =
Source<EventEnvelope, NotUsed> source =
readJournal.eventsByPersistenceId("user-1337", 0, Long.MAX_VALUE);
// materialize stream, consuming events
@ -262,7 +264,7 @@ public class PersistenceQueryDocTest {
//#events-by-tag
// assuming journal is able to work with numeric offsets we can:
final Source<EventEnvelope, BoxedUnit> blueThings =
final Source<EventEnvelope, NotUsed> blueThings =
readJournal.eventsByTag("blue", 0L);
// find top 10 blue things:
@ -276,7 +278,7 @@ public class PersistenceQueryDocTest {
}, mat);
// start another query, from the known offset
Source<EventEnvelope, BoxedUnit> blue = readJournal.eventsByTag("blue", 10);
Source<EventEnvelope, NotUsed> blue = readJournal.eventsByTag("blue", 10);
//#events-by-tag
}

View file

@ -6,8 +6,8 @@ package docs.persistence.query;
import java.util.HashSet;
import java.util.Set;
import scala.runtime.BoxedUnit;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.persistence.journal.WriteEventAdapter;
import akka.persistence.journal.Tagged;
@ -37,7 +37,7 @@ public class LeveldbPersistenceQueryDocTest {
PersistenceQuery.get(system).getReadJournalFor(LeveldbReadJournal.class,
LeveldbReadJournal.Identifier());
Source<EventEnvelope, BoxedUnit> source =
Source<EventEnvelope, NotUsed> source =
queries.eventsByPersistenceId("some-persistence-id", 0, Long.MAX_VALUE);
//#EventsByPersistenceId
}
@ -48,7 +48,7 @@ public class LeveldbPersistenceQueryDocTest {
PersistenceQuery.get(system).getReadJournalFor(LeveldbReadJournal.class,
LeveldbReadJournal.Identifier());
Source<String, BoxedUnit> source = queries.allPersistenceIds();
Source<String, NotUsed> source = queries.allPersistenceIds();
//#AllPersistenceIds
}
@ -58,7 +58,7 @@ public class LeveldbPersistenceQueryDocTest {
PersistenceQuery.get(system).getReadJournalFor(LeveldbReadJournal.class,
LeveldbReadJournal.Identifier());
Source<EventEnvelope, BoxedUnit> source =
Source<EventEnvelope, NotUsed> source =
queries.eventsByTag("green", 0);
//#EventsByTag
}

View file

@ -8,6 +8,7 @@ import java.util.Arrays;
import java.util.List;
import java.util.concurrent.TimeUnit;
import akka.NotUsed;
import akka.stream.javadsl.GraphDSL;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -26,7 +27,6 @@ import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
import static org.junit.Assert.assertArrayEquals;
public class BidiFlowDocTest {
@ -110,7 +110,7 @@ public class BidiFlowDocTest {
//#codec
@SuppressWarnings("unused")
//#codec
public final BidiFlow<Message, ByteString, ByteString, Message, BoxedUnit> codecVerbose =
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));
@ -119,7 +119,7 @@ public class BidiFlowDocTest {
return BidiShape.fromFlows(top, bottom);
}));
public final BidiFlow<Message, ByteString, ByteString, Message, BoxedUnit> codec =
public final BidiFlow<Message, ByteString, ByteString, Message, NotUsed> codec =
BidiFlow.fromFunctions(BidiFlowDocTest::toBytes, BidiFlowDocTest::fromBytes);
//#codec
@ -186,7 +186,7 @@ public class BidiFlowDocTest {
}
}
public final BidiFlow<ByteString, ByteString, ByteString, ByteString, BoxedUnit> framing =
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));
@ -210,16 +210,16 @@ public class BidiFlowDocTest {
* | +-------+ +---------+ |
* +------------------------------------+
*/
final BidiFlow<Message, ByteString, ByteString, Message, BoxedUnit> stack =
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, BoxedUnit> pingpong =
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, BoxedUnit> flow =
final Flow<Message, Message, NotUsed> flow =
stack.atop(stack.reversed()).join(pingpong);
final Future<List<Message>> result = Source
.from(Arrays.asList(0, 1, 2))

View file

@ -6,6 +6,7 @@ package docs.stream;
import java.util.Arrays;
import java.util.Optional;
import akka.NotUsed;
import akka.stream.ClosedShape;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -20,7 +21,7 @@ import akka.stream.javadsl.Tcp.OutgoingConnection;
import akka.testkit.JavaTestKit;
import akka.util.ByteString;
import scala.concurrent.*;
import scala.runtime.BoxedUnit;
import scala.Option;
public class CompositionDocTest {
@ -56,47 +57,47 @@ public class CompositionDocTest {
@Test
public void nestedFlow() throws Exception {
//#nested-flow
final Source<Integer, BoxedUnit> nestedSource =
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, BoxedUnit> nestedFlow =
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, BoxedUnit> nestedSink =
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<BoxedUnit> runnableGraph = nestedSource.to(nestedSink);
final RunnableGraph<NotUsed> runnableGraph = nestedSource.to(nestedSink);
//#nested-flow
}
@Test
public void reusingComponents() throws Exception {
final Source<Integer, BoxedUnit> nestedSource =
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, BoxedUnit> nestedFlow =
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, BoxedUnit> nestedSink =
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<BoxedUnit> runnableGraph = nestedSource.to(nestedSink);
final RunnableGraph<NotUsed> runnableGraph = nestedSource.to(nestedSink);
// Usage is uniform, no matter if modules are composite or atomic
final RunnableGraph<BoxedUnit> runnableGraph2 =
final RunnableGraph<NotUsed> runnableGraph2 =
Source.single(0).to(Sink.fold(0, (acc, i) -> acc + i));
//#reuse
}
@ -150,7 +151,7 @@ public class CompositionDocTest {
@Test
public void partialGraph() throws Exception {
//#partial-graph
final Graph<FlowShape<Integer, Integer>, BoxedUnit> partial =
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));
@ -173,10 +174,10 @@ public class CompositionDocTest {
//#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, BoxedUnit> flow = Flow.fromGraph(partial);
final Flow<Integer, Integer, NotUsed> flow = Flow.fromGraph(partial);
// Simple way to create a graph backed Source
final Source<Integer, BoxedUnit> source = Source.fromGraph(
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);
@ -187,23 +188,23 @@ public class CompositionDocTest {
);
// Building a Sink with a nested Flow, using the fluid DSL
final Sink<Integer, BoxedUnit> sink = Flow.of(Integer.class)
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<BoxedUnit> closed = source.via(flow.filter(i -> i > 1)).to(sink);
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<BoxedUnit> closed1 =
final RunnableGraph<NotUsed> closed1 =
Source.single(0).to(Sink.foreach(System.out::println));
final RunnableGraph<BoxedUnit> closed2 =
final RunnableGraph<NotUsed> closed2 =
RunnableGraph.fromGraph(
GraphDSL.create(builder -> {
final ClosedShape embeddedClosed = builder.add(closed1);
@ -246,7 +247,7 @@ public class CompositionDocTest {
final Source<Integer, Promise<Optional<Integer>>> source = Source.<Integer>maybe();
// Materializes to BoxedUnit (black)
final Flow<Integer, Integer, BoxedUnit> flow1 = Flow.of(Integer.class).take(100);
final Flow<Integer, Integer, NotUsed> flow1 = Flow.of(Integer.class).take(100);
// Materializes to Promise<Option<>> (red)
final Source<Integer, Promise<Optional<Integer>>> nestedSource =
@ -255,7 +256,7 @@ public class CompositionDocTest {
//#mat-combine-2
// Materializes to BoxedUnit (orange)
final Flow<Integer, ByteString, BoxedUnit> flow2 = Flow.of(Integer.class)
final Flow<Integer, ByteString, NotUsed> flow2 = Flow.of(Integer.class)
.map(i -> ByteString.fromString(i.toString()));
// Materializes to Future<OutgoingConnection> (yellow)
@ -287,19 +288,19 @@ public class CompositionDocTest {
@Test
public void attributes() throws Exception {
//#attributes-inheritance
final Source<Integer, BoxedUnit> nestedSource =
final Source<Integer, NotUsed> nestedSource =
Source.single(0)
.map(i -> i + 1)
.named("nestedSource"); // Wrap, no inputBuffer set
final Flow<Integer, Integer, BoxedUnit> nestedFlow =
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, BoxedUnit> nestedSink =
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

View file

@ -9,6 +9,7 @@ import java.util.*;
import java.util.concurrent.TimeUnit;
import java.util.stream.Stream;
import akka.NotUsed;
import akka.japi.Pair;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -19,7 +20,6 @@ import scala.concurrent.Future;
import scala.concurrent.Promise;
import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
import scala.Option;
import akka.actor.ActorSystem;
import akka.actor.Cancellable;
@ -48,13 +48,13 @@ public class FlowDocTest {
@Test
public void sourceIsImmutable() throws Exception {
//#source-immutable
final Source<Integer, BoxedUnit> source =
final Source<Integer, NotUsed> source =
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
source.map(x -> 0); // has no effect on source, since it's immutable
source.runWith(Sink.fold(0, (agg, next) -> agg + next), mat); // 55
// returns new Source<Integer>, with `map()` appended
final Source<Integer, BoxedUnit> zeroes = source.map(x -> 0);
final Source<Integer, NotUsed> zeroes = source.map(x -> 0);
final Sink<Integer, Future<Integer>> fold =
Sink.fold(0, (agg, next) -> agg + next);
zeroes.runWith(fold, mat); // 0
@ -70,7 +70,7 @@ public class FlowDocTest {
@Test
public void materializationInSteps() throws Exception {
//#materialization-in-steps
final Source<Integer, BoxedUnit> source =
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, Future<Integer>> sink =
@ -91,7 +91,7 @@ public class FlowDocTest {
@Test
public void materializationRunWith() throws Exception {
//#materialization-runWith
final Source<Integer, BoxedUnit> source =
final Source<Integer, NotUsed> source =
Source.from(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
final Sink<Integer, Future<Integer>> sink =
Sink.fold(0, (aggr, next) -> aggr + next);
@ -195,12 +195,12 @@ public class FlowDocTest {
.to(Sink.foreach(System.out::println));
// Starting from a Source
final Source<Integer, BoxedUnit> source = Source.from(Arrays.asList(1, 2, 3, 4))
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, BoxedUnit> sink = Flow.of(Integer.class)
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
@ -276,9 +276,9 @@ public class FlowDocTest {
public void fusingAndAsync() {
//#explicit-fusing
Flow<Integer, Integer, BoxedUnit> flow =
Flow<Integer, Integer, NotUsed> flow =
Flow.of(Integer.class).map(x -> x * 2).filter(x -> x > 500);
Graph<FlowShape<Integer, Integer>, BoxedUnit> fused =
Graph<FlowShape<Integer, Integer>, NotUsed> fused =
akka.stream.Fusing.aggressive(flow);
Source.fromIterator(() -> Stream.iterate(0, x -> x + 1).iterator())

View file

@ -7,13 +7,14 @@ import static org.junit.Assert.assertEquals;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.TimeUnit;
import akka.NotUsed;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.Duration;
import scala.runtime.BoxedUnit;
import akka.actor.ActorSystem;
import akka.stream.ActorMaterializer;
@ -46,7 +47,7 @@ public class FlowErrorDocTest {
public void demonstrateFailStream() throws Exception {
//#stop
final Materializer mat = ActorMaterializer.create(system);
final Source<Integer, BoxedUnit> source = Source.from(Arrays.asList(0, 1, 2, 3, 4, 5))
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(0, 1, 2, 3, 4, 5))
.map(elem -> 100 / elem);
final Sink<Integer, Future<Integer>> fold =
Sink.fold(0, (acc, elem) -> acc + elem);
@ -70,7 +71,7 @@ public class FlowErrorDocTest {
final Materializer mat = ActorMaterializer.create(
ActorMaterializerSettings.create(system).withSupervisionStrategy(decider),
system);
final Source<Integer, BoxedUnit> source = Source.from(Arrays.asList(0, 1, 2, 3, 4, 5))
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(0, 1, 2, 3, 4, 5))
.map(elem -> 100 / elem);
final Sink<Integer, Future<Integer>> fold =
Sink.fold(0, (acc, elem) -> acc + elem);
@ -92,10 +93,10 @@ public class FlowErrorDocTest {
else
return Supervision.stop();
};
final Flow<Integer, Integer, BoxedUnit> flow =
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, BoxedUnit> source = Source.from(Arrays.asList(0, 1, 2, 3, 4, 5))
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(0, 1, 2, 3, 4, 5))
.via(flow);
final Sink<Integer, Future<Integer>> fold =
Sink.fold(0, (acc, elem) -> acc + elem);
@ -117,13 +118,13 @@ public class FlowErrorDocTest {
else
return Supervision.stop();
};
final Flow<Integer, Integer, BoxedUnit> flow =
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, BoxedUnit> source = Source.from(Arrays.asList(1, 3, -1, 5, 7))
final Source<Integer, NotUsed> source = Source.from(Arrays.asList(1, 3, -1, 5, 7))
.via(flow);
final Future<List<Integer>> result = source.grouped(1000)
.runWith(Sink.<List<Integer>>head(), mat);

View file

@ -9,6 +9,7 @@ import java.util.Arrays;
import java.util.List;
import java.util.concurrent.TimeUnit;
import akka.NotUsed;
import akka.stream.ClosedShape;
import akka.stream.SourceShape;
import org.junit.AfterClass;
@ -18,7 +19,6 @@ import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.Duration;
import scala.runtime.BoxedUnit;
import akka.actor.ActorSystem;
import akka.japi.Pair;
import akka.stream.*;
@ -45,13 +45,13 @@ public class FlowGraphDocTest {
@Test
public void demonstrateBuildSimpleGraph() throws Exception {
//#simple-flow-graph
final Source<Integer, BoxedUnit> in = Source.from(Arrays.asList(1, 2, 3, 4, 5));
final Source<Integer, NotUsed> in = Source.from(Arrays.asList(1, 2, 3, 4, 5));
final Sink<List<String>, Future<List<String>>> sink = Sink.head();
final Sink<List<Integer>, Future<List<Integer>>> sink2 = Sink.head();
final Flow<Integer, Integer, BoxedUnit> f1 = Flow.of(Integer.class).map(elem -> elem + 10);
final Flow<Integer, Integer, BoxedUnit> f2 = Flow.of(Integer.class).map(elem -> elem + 20);
final Flow<Integer, String, BoxedUnit> f3 = Flow.of(Integer.class).map(elem -> elem.toString());
final Flow<Integer, Integer, BoxedUnit> f4 = Flow.of(Integer.class).map(elem -> elem + 30);
final Flow<Integer, Integer, NotUsed> f1 = Flow.of(Integer.class).map(elem -> elem + 10);
final Flow<Integer, Integer, NotUsed> f2 = Flow.of(Integer.class).map(elem -> elem + 20);
final Flow<Integer, String, NotUsed> f3 = Flow.of(Integer.class).map(elem -> elem.toString());
final Flow<Integer, Integer, NotUsed> f4 = Flow.of(Integer.class).map(elem -> elem + 30);
final RunnableGraph<Future<List<String>>> result =
RunnableGraph.<Future<List<String>>>fromGraph(
@ -81,8 +81,8 @@ public class FlowGraphDocTest {
public void demonstrateConnectErrors() {
try {
//#simple-graph
final RunnableGraph<BoxedUnit> g =
RunnableGraph.<BoxedUnit>fromGraph(
final RunnableGraph<NotUsed> g =
RunnableGraph.<NotUsed>fromGraph(
GraphDSL
.create((b) -> {
final SourceShape<Integer> source1 = b.add(Source.from(Arrays.asList(1, 2, 3, 4, 5)));
@ -107,7 +107,7 @@ public class FlowGraphDocTest {
//#flow-graph-reusing-a-flow
final Sink<Integer, Future<Integer>> topHeadSink = Sink.head();
final Sink<Integer, Future<Integer>> bottomHeadSink = Sink.head();
final Flow<Integer, Integer, BoxedUnit> sharedDoubler = Flow.of(Integer.class).map(elem -> elem * 2);
final Flow<Integer, Integer, NotUsed> sharedDoubler = Flow.of(Integer.class).map(elem -> elem * 2);
final RunnableGraph<Pair<Future<Integer>, Future<Integer>>> g =
RunnableGraph.<Pair<Future<Integer>, Future<Integer>>>fromGraph(
@ -139,7 +139,7 @@ public class FlowGraphDocTest {
return a + b;
});
final Flow<Future<Integer>, Integer, BoxedUnit> flatten = Flow.<Future<Integer>>create()
final Flow<Future<Integer>, Integer, NotUsed> flatten = Flow.<Future<Integer>>create()
.mapAsync(4, x -> {
return x;
});

View file

@ -3,10 +3,10 @@
*/
package docs.stream;
import scala.runtime.BoxedUnit;
import static org.junit.Assert.assertEquals;
import akka.NotUsed;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -37,10 +37,10 @@ public class FlowParallelismDocTest {
static class Pancake {}
//#pipelining
Flow<ScoopOfBatter, HalfCookedPancake, BoxedUnit> fryingPan1 =
Flow<ScoopOfBatter, HalfCookedPancake, NotUsed> fryingPan1 =
Flow.of(ScoopOfBatter.class).map(batter -> new HalfCookedPancake());
Flow<HalfCookedPancake, Pancake, BoxedUnit> fryingPan2 =
Flow<HalfCookedPancake, Pancake, NotUsed> fryingPan2 =
Flow.of(HalfCookedPancake.class).map(halfCooked -> new Pancake());
//#pipelining
@ -49,17 +49,17 @@ public class FlowParallelismDocTest {
//#pipelining
// With the two frying pans we can fully cook pancakes
Flow<ScoopOfBatter, Pancake, BoxedUnit> pancakeChef = fryingPan1.via(fryingPan2);
Flow<ScoopOfBatter, Pancake, NotUsed> pancakeChef = fryingPan1.via(fryingPan2);
//#pipelining
}
@Test
public void demonstrateParallelism() {
//#parallelism
Flow<ScoopOfBatter, Pancake, BoxedUnit> fryingPan =
Flow<ScoopOfBatter, Pancake, NotUsed> fryingPan =
Flow.of(ScoopOfBatter.class).map(batter -> new Pancake());
Flow<ScoopOfBatter, Pancake, BoxedUnit> pancakeChef =
Flow<ScoopOfBatter, Pancake, NotUsed> pancakeChef =
Flow.fromGraph(GraphDSL.create(b -> {
final UniformFanInShape<Pancake, Pancake> mergePancakes =
b.add(Merge.create(2));
@ -82,7 +82,7 @@ public class FlowParallelismDocTest {
@Test
public void parallelPipeline() {
//#parallel-pipeline
Flow<ScoopOfBatter, Pancake, BoxedUnit> pancakeChef =
Flow<ScoopOfBatter, Pancake, NotUsed> pancakeChef =
Flow.fromGraph(GraphDSL.create(b -> {
final UniformFanInShape<Pancake, Pancake> mergePancakes =
b.add(Merge.create(2));
@ -109,7 +109,7 @@ public class FlowParallelismDocTest {
@Test
public void pipelinedParallel() {
//#pipelined-parallel
Flow<ScoopOfBatter, HalfCookedPancake, BoxedUnit> pancakeChefs1 =
Flow<ScoopOfBatter, HalfCookedPancake, NotUsed> pancakeChefs1 =
Flow.fromGraph(GraphDSL.create(b -> {
final UniformFanInShape<HalfCookedPancake, HalfCookedPancake> mergeHalfCooked =
b.add(Merge.create(2));
@ -124,7 +124,7 @@ public class FlowParallelismDocTest {
return FlowShape.of(dispatchBatter.in(), mergeHalfCooked.out());
}));
Flow<HalfCookedPancake, Pancake, BoxedUnit> pancakeChefs2 =
Flow<HalfCookedPancake, Pancake, NotUsed> pancakeChefs2 =
Flow.fromGraph(GraphDSL.create(b -> {
final UniformFanInShape<Pancake, Pancake> mergePancakes =
b.add(Merge.create(2));
@ -139,7 +139,7 @@ public class FlowParallelismDocTest {
return FlowShape.of(dispatchHalfCooked.in(), mergePancakes.out());
}));
Flow<ScoopOfBatter, Pancake, BoxedUnit> kitchen =
Flow<ScoopOfBatter, Pancake, NotUsed> kitchen =
pancakeChefs1.via(pancakeChefs2);
//#pipelined-parallel
}

View file

@ -2,11 +2,11 @@ package docs.stream;
import java.util.Arrays;
import akka.NotUsed;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.runtime.BoxedUnit;
import akka.actor.ActorSystem;
import akka.stream.*;
import akka.stream.javadsl.*;
@ -34,13 +34,13 @@ public class GraphCyclesDocTest {
final static SilenceSystemOut.System System = SilenceSystemOut.get();
final Source<Integer, BoxedUnit> source = Source.from(Arrays.asList(1, 2, 3, 4, 5));
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, BoxedUnit> printFlow =
final Flow<Integer, Integer, NotUsed> printFlow =
Flow.of(Integer.class).map(s -> {
System.out.println(s);
return s;
@ -62,7 +62,7 @@ public class GraphCyclesDocTest {
@Test
public void demonstrateUnfairCycle() {
final Flow<Integer, Integer, BoxedUnit> printFlow =
final Flow<Integer, Integer, NotUsed> printFlow =
Flow.of(Integer.class).map(s -> {
System.out.println(s);
return s;
@ -85,7 +85,7 @@ public class GraphCyclesDocTest {
@Test
public void demonstrateDroppingCycle() {
final Flow<Integer, Integer, BoxedUnit> printFlow =
final Flow<Integer, Integer, NotUsed> printFlow =
Flow.of(Integer.class).map(s -> {
System.out.println(s);
return s;
@ -109,7 +109,7 @@ public class GraphCyclesDocTest {
@Test
public void demonstrateZippingCycle() {
final Flow<Integer, Integer, BoxedUnit> printFlow =
final Flow<Integer, Integer, NotUsed> printFlow =
Flow.of(Integer.class).map(s -> {
System.out.println(s);
return s;
@ -133,7 +133,7 @@ public class GraphCyclesDocTest {
@Test
public void demonstrateLiveZippingCycle() {
final Flow<Integer, Integer, BoxedUnit> printFlow =
final Flow<Integer, Integer, NotUsed> printFlow =
Flow.of(Integer.class).map(s -> {
System.out.println(s);
return s;

View file

@ -1,5 +1,7 @@
package docs.stream;
import akka.Done;
import akka.NotUsed;
import akka.actor.ActorSystem;
//#imports
import akka.dispatch.Futures;
@ -7,7 +9,6 @@ import akka.dispatch.Mapper;
import akka.dispatch.OnSuccess;
import akka.japi.Option;
import akka.japi.Predicate;
import akka.japi.function.Effect;
import akka.japi.function.Procedure;
import akka.stream.*;
import akka.stream.javadsl.*;
@ -20,7 +21,6 @@ import akka.japi.Function;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.reactivestreams.Subscriber;
import org.reactivestreams.Subscription;
import scala.Tuple2;
import scala.concurrent.Await;
@ -29,7 +29,6 @@ import scala.concurrent.Future;
import scala.concurrent.Promise;
import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
import java.util.*;
@ -96,10 +95,10 @@ public class GraphStageDocTest {
public void demonstrateCustomSourceUsage() throws Exception {
//#simple-source-usage
// A GraphStage is a proper Graph, just like what GraphDSL.create would return
Graph<SourceShape<Integer>, BoxedUnit> sourceGraph = new NumbersSource();
Graph<SourceShape<Integer>, NotUsed> sourceGraph = new NumbersSource();
// Create a Source from the Graph to access the DSL
Source<Integer, BoxedUnit> mySource = Source.fromGraph(sourceGraph);
Source<Integer, NotUsed> mySource = Source.fromGraph(sourceGraph);
// Returns 55
Future<Integer> result1 = mySource.take(10).runFold(0, (sum, next) -> sum + next, mat);
@ -162,7 +161,7 @@ public class GraphStageDocTest {
@Test
public void demonstrateOneToOne() throws Exception {
// tests:
final Graph<FlowShape<String, Integer>, BoxedUnit> stringLength =
final Graph<FlowShape<String, Integer>, NotUsed> stringLength =
Flow.fromGraph(new Map<String, Integer>(new Function<String, Integer>() {
@Override
public Integer apply(String str) {
@ -229,7 +228,7 @@ public class GraphStageDocTest {
public void demonstrateAManyToOneElementGraphStage() throws Exception {
// tests:
Graph<FlowShape<Integer, Integer>, BoxedUnit> evenFilter =
Graph<FlowShape<Integer, Integer>, NotUsed> evenFilter =
Flow.fromGraph(new Filter<Integer>(n -> n % 2 == 0));
Future<Integer> result =
@ -298,7 +297,7 @@ public class GraphStageDocTest {
@Test
public void demonstrateAOneToManyElementGraphStage() throws Exception {
// tests:
Graph<FlowShape<Integer, Integer>, BoxedUnit> duplicator =
Graph<FlowShape<Integer, Integer>, NotUsed> duplicator =
Flow.fromGraph(new Duplicator<Integer>());
Future<Integer> result =
@ -355,7 +354,7 @@ public class GraphStageDocTest {
@Test
public void demonstrateASimplerOneToManyStage() throws Exception {
// tests:
Graph<FlowShape<Integer, Integer>, BoxedUnit> duplicator =
Graph<FlowShape<Integer, Integer>, NotUsed> duplicator =
Flow.fromGraph(new Duplicator2<Integer>());
Future<Integer> result =
@ -387,9 +386,9 @@ public class GraphStageDocTest {
// will close upstream when the future completes
public class KillSwitch<A> extends GraphStage<FlowShape<A, A>> {
private final Future<BoxedUnit> switchF;
private final Future<Done> switchF;
public KillSwitch(Future<BoxedUnit> switchF) {
public KillSwitch(Future<Done> switchF) {
this.switchF = switchF;
}
@ -423,18 +422,18 @@ public class GraphStageDocTest {
@Override
public void preStart() {
AsyncCallback<BoxedUnit> callback = createAsyncCallback(new Procedure<BoxedUnit>() {
AsyncCallback<Done> callback = createAsyncCallback(new Procedure<Done>() {
@Override
public void apply(BoxedUnit param) throws Exception {
public void apply(Done param) throws Exception {
completeStage();
}
});
ExecutionContext ec = system.dispatcher();
switchF.onSuccess(new OnSuccess<BoxedUnit>() {
switchF.onSuccess(new OnSuccess<Done>() {
@Override
public void onSuccess(BoxedUnit result) throws Throwable {
callback.invoke(BoxedUnit.UNIT);
public void onSuccess(Done result) throws Throwable {
callback.invoke(Done.getInstance());
}
}, ec);
}
@ -447,16 +446,16 @@ public class GraphStageDocTest {
public void demonstrateAnAsynchronousSideChannel() throws Exception{
// tests:
Promise<BoxedUnit> switchF = Futures.promise();
Graph<FlowShape<Integer, Integer>, BoxedUnit> killSwitch =
Promise<Done> switchF = Futures.promise();
Graph<FlowShape<Integer, Integer>, NotUsed> killSwitch =
Flow.fromGraph(new KillSwitch<>(switchF.future()));
ExecutionContext ec = system.dispatcher();
// TODO this is probably racey, is there a way to make sure it happens after?
Future<Integer> valueAfterKill = switchF.future().flatMap(new Mapper<BoxedUnit, Future<Integer>>() {
Future<Integer> valueAfterKill = switchF.future().flatMap(new Mapper<Done, Future<Integer>>() {
@Override
public Future<Integer> apply(BoxedUnit parameter) {
public Future<Integer> apply(Done parameter) {
return Futures.successful(4);
}
}, ec);
@ -467,7 +466,7 @@ public class GraphStageDocTest {
.via(killSwitch)
.runFold(0, (n, sum) -> n + sum, mat);
switchF.success(BoxedUnit.UNIT);
switchF.success(Done.getInstance());
assertEquals(new Integer(6), Await.result(result, Duration.create(3, "seconds")));
}
@ -694,7 +693,7 @@ public class GraphStageDocTest {
TestSubscriber.ManualProbe<Integer> subscriber = TestSubscriber.manualProbe(system);
TestPublisher.Probe<Integer> publisher = TestPublisher.probe(0, system);
RunnableGraph<BoxedUnit> flow2 =
RunnableGraph<NotUsed> flow2 =
Source.fromPublisher(publisher)
.via(new TwoBuffer<>())
.to(Sink.fromSubscriber(subscriber));

View file

@ -4,6 +4,7 @@
package docs.stream;
import akka.NotUsed;
import akka.actor.*;
import akka.dispatch.Futures;
import akka.dispatch.MessageDispatcher;
@ -22,7 +23,6 @@ import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.ExecutionContext;
import scala.concurrent.Future;
import scala.runtime.BoxedUnit;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Optional;
@ -290,14 +290,14 @@ public class IntegrationDocTest {
{
//#tweet-authors
final Source<Author, BoxedUnit> authors = tweets
final Source<Author, NotUsed> authors = tweets
.filter(t -> t.hashtags().contains(AKKA))
.map(t -> t.author);
//#tweet-authors
//#email-addresses-mapAsync
final Source<String, BoxedUnit> emailAddresses = authors
final Source<String, NotUsed> emailAddresses = authors
.mapAsync(4, author -> addressSystem.lookupEmail(author.handle))
.filter(o -> o.isPresent())
.map(o -> o.get());
@ -305,7 +305,7 @@ public class IntegrationDocTest {
//#email-addresses-mapAsync
//#send-emails
final RunnableGraph<BoxedUnit> sendEmails = emailAddresses
final RunnableGraph<NotUsed> sendEmails = emailAddresses
.mapAsync(4, address ->
emailServer.send(new Email(address, "Akka", "I like your tweet")))
.to(Sink.ignore());
@ -331,18 +331,18 @@ public class IntegrationDocTest {
final AddressSystem2 addressSystem = new AddressSystem2();
{
final Source<Author, BoxedUnit> authors = tweets
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, BoxedUnit> lookupEmail =
final Flow<Author, String, NotUsed> lookupEmail =
Flow.of(Author.class)
.mapAsync(4, author -> addressSystem.lookupEmail(author.handle))
.withAttributes(resumeAttrib);
final Source<String, BoxedUnit> emailAddresses = authors.via(lookupEmail);
final Source<String, NotUsed> emailAddresses = authors.via(lookupEmail);
//#email-addresses-mapAsync-supervision
}
@ -358,18 +358,18 @@ public class IntegrationDocTest {
{
//#external-service-mapAsyncUnordered
final Source<Author, BoxedUnit> authors =
final Source<Author, NotUsed> authors =
tweets
.filter(t -> t.hashtags().contains(AKKA))
.map(t -> t.author);
final Source<String, BoxedUnit> emailAddresses =
final Source<String, NotUsed> emailAddresses =
authors
.mapAsyncUnordered(4, author -> addressSystem.lookupEmail(author.handle))
.filter(o -> o.isPresent())
.map(o -> o.get());
final RunnableGraph<BoxedUnit> sendEmails =
final RunnableGraph<NotUsed> sendEmails =
emailAddresses
.mapAsyncUnordered(4, address ->
emailServer.send(new Email(address, "Akka", "I like your tweet")))
@ -389,19 +389,19 @@ public class IntegrationDocTest {
final SmsServer smsServer = new SmsServer(getRef());
{
final Source<Author, BoxedUnit> authors =
final Source<Author, NotUsed> authors =
tweets
.filter(t -> t.hashtags().contains(AKKA))
.map(t -> t.author);
final Source<String, BoxedUnit> phoneNumbers = authors.mapAsync(4, author -> addressSystem.lookupPhoneNumber(author.handle))
final Source<String, NotUsed> phoneNumbers = authors.mapAsync(4, author -> addressSystem.lookupPhoneNumber(author.handle))
.filter(o -> o.isPresent())
.map(o -> o.get());
//#blocking-mapAsync
final MessageDispatcher blockingEc = system.dispatchers().lookup("blocking-dispatcher");
final RunnableGraph<BoxedUnit> sendTextMessages =
final RunnableGraph<NotUsed> sendTextMessages =
phoneNumbers
.mapAsync(4, phoneNo ->
Futures.future(() ->
@ -436,17 +436,17 @@ public class IntegrationDocTest {
final SmsServer smsServer = new SmsServer(probe.ref());
{
final Source<Author, BoxedUnit> authors =
final Source<Author, NotUsed> authors =
tweets
.filter(t -> t.hashtags().contains(AKKA))
.map(t -> t.author);
final Source<String, BoxedUnit> phoneNumbers = authors.mapAsync(4, author -> addressSystem.lookupPhoneNumber(author.handle))
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, BoxedUnit> send =
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"));
@ -477,9 +477,9 @@ public class IntegrationDocTest {
{
//#save-tweets
final Source<Tweet, BoxedUnit> akkaTweets = tweets.filter(t -> t.hashtags().contains(AKKA));
final Source<Tweet, NotUsed> akkaTweets = tweets.filter(t -> t.hashtags().contains(AKKA));
final RunnableGraph<BoxedUnit> saveTweets =
final RunnableGraph<NotUsed> saveTweets =
akkaTweets
.mapAsync(4, tweet -> ask(database, new Save(tweet), 300))
.to(Sink.ignore());

View file

@ -3,6 +3,7 @@
*/
package docs.stream;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.actor.Cancellable;
import akka.http.javadsl.model.Uri;
@ -19,7 +20,6 @@ import scala.Option;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import scala.concurrent.Promise;
import scala.runtime.BoxedUnit;
import org.reactivestreams.Publisher;
import org.reactivestreams.Subscriber;
@ -67,30 +67,30 @@ public class MigrationsJava {
Inlet<Integer> inlet1 = null;
Inlet<Integer> inlet2 = null;
Flow<Integer, Integer, BoxedUnit> flow = Flow.of(Integer.class);
Flow<Integer, Integer, BoxedUnit> flow1 = Flow.of(Integer.class);
Flow<Integer, Integer, BoxedUnit> flow2 = Flow.of(Integer.class);
Flow<Integer, Integer, NotUsed> flow = Flow.of(Integer.class);
Flow<Integer, Integer, NotUsed> flow1 = Flow.of(Integer.class);
Flow<Integer, Integer, NotUsed> flow2 = Flow.of(Integer.class);
Promise<Option<Integer>> promise = null;
{
Graph<SourceShape<Integer>, BoxedUnit> graphSource = null;
Graph<SinkShape<Integer>, BoxedUnit> graphSink = null;
Graph<FlowShape<Integer, Integer>, BoxedUnit> graphFlow = null;
Graph<SourceShape<Integer>, NotUsed> graphSource = null;
Graph<SinkShape<Integer>, NotUsed> graphSink = null;
Graph<FlowShape<Integer, Integer>, NotUsed> graphFlow = null;
//#flow-wrap
Source<Integer, BoxedUnit> source = Source.fromGraph(graphSource);
Sink<Integer, BoxedUnit> sink = Sink.fromGraph(graphSink);
Flow<Integer, Integer, BoxedUnit> aflow = Flow.fromGraph(graphFlow);
Source<Integer, NotUsed> source = Source.fromGraph(graphSource);
Sink<Integer, NotUsed> sink = Sink.fromGraph(graphSink);
Flow<Integer, Integer, NotUsed> aflow = Flow.fromGraph(graphFlow);
Flow.fromSinkAndSource(Sink.<Integer>head(), Source.single(0));
Flow.fromSinkAndSourceMat(Sink.<Integer>head(), Source.single(0), Keep.left());
//#flow-wrap
Graph<BidiShape<Integer, Integer, Integer, Integer>, BoxedUnit> bidiGraph = null;
Graph<BidiShape<Integer, Integer, Integer, Integer>, NotUsed> bidiGraph = null;
//#bidi-wrap
BidiFlow<Integer, Integer, Integer, Integer, BoxedUnit> bidiFlow =
BidiFlow<Integer, Integer, Integer, Integer, NotUsed> bidiFlow =
BidiFlow.fromGraph(bidiGraph);
BidiFlow.fromFlows(flow1, flow2);
BidiFlow.fromFlowsMat(flow1, flow2, Keep.both());
@ -158,10 +158,10 @@ public class MigrationsJava {
FiniteDuration.create(200, TimeUnit.MILLISECONDS),
"tick");
final Source<Integer, BoxedUnit> pubSource =
final Source<Integer, NotUsed> pubSource =
Source.fromPublisher(TestPublisher.<Integer>manualProbe(true, sys));
final Source<Integer, BoxedUnit> futSource =
final Source<Integer, NotUsed> futSource =
Source.fromFuture(Futures.successful(42));
final Source<Integer, Subscriber<Integer>> subSource =
@ -169,7 +169,7 @@ public class MigrationsJava {
//#source-creators
//#sink-creators
final Sink<Integer, BoxedUnit> subSink =
final Sink<Integer, NotUsed> subSink =
Sink.fromSubscriber(TestSubscriber.<Integer>manualProbe(sys));
//#sink-creators
@ -182,14 +182,14 @@ public class MigrationsJava {
//#sink-as-publisher
//#empty-flow
Flow<Integer, Integer, BoxedUnit> emptyFlow = Flow.<Integer>create();
Flow<Integer, Integer, NotUsed> emptyFlow = Flow.<Integer>create();
// or
Flow<Integer, Integer, BoxedUnit> emptyFlow2 = Flow.of(Integer.class);
Flow<Integer, Integer, NotUsed> emptyFlow2 = Flow.of(Integer.class);
//#empty-flow
//#flatMapConcat
Flow.<Source<Integer, BoxedUnit>>create().
<Integer, BoxedUnit>flatMapConcat(i -> i);
Flow.<Source<Integer, NotUsed>>create().
<Integer, NotUsed>flatMapConcat(i -> i);
//#flatMapConcat
//#group-flatten

View file

@ -12,6 +12,7 @@ import java.util.stream.Collectors;
import java.util.stream.DoubleStream;
import java.util.stream.Stream;
import akka.NotUsed;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -32,7 +33,6 @@ import scala.collection.Iterator;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.Duration;
import scala.runtime.BoxedUnit;
import scala.util.Random;
public class RateTransformationDocTest {
@ -57,7 +57,7 @@ public class RateTransformationDocTest {
@Test
public void conflateShouldSummarize() throws Exception {
//#conflate-summarize
final Flow<Double, Tuple3<Double, Double, Integer>, BoxedUnit> statsFlow =
final Flow<Double, Tuple3<Double, Double, Integer>, NotUsed> statsFlow =
Flow.of(Double.class)
.conflate(elem -> Collections.singletonList(elem), (acc, elem) -> {
return Stream
@ -85,7 +85,7 @@ public class RateTransformationDocTest {
public void conflateShouldSample() throws Exception {
//#conflate-sample
final Double p = 0.01;
final Flow<Double, Double, BoxedUnit> sampleFlow = Flow.of(Double.class)
final Flow<Double, Double, NotUsed> sampleFlow = Flow.of(Double.class)
.conflate(elem -> Collections.singletonList(elem), (acc, elem) -> {
if (r.nextDouble() < p) {
return Stream
@ -108,7 +108,7 @@ public class RateTransformationDocTest {
@Test
public void expandShouldRepeatLast() throws Exception {
//#expand-last
final Flow<Double, Double, BoxedUnit> lastFlow = Flow.of(Double.class)
final Flow<Double, Double, NotUsed> lastFlow = Flow.of(Double.class)
.expand(d -> d, s -> new Pair<>(s, s));
//#expand-last
@ -131,13 +131,13 @@ public class RateTransformationDocTest {
public void expandShouldTrackDrift() throws Exception {
@SuppressWarnings("unused")
//#expand-drift
final Flow<Double, Pair<Double, Integer>, BoxedUnit> driftFlow = Flow.of(Double.class)
final Flow<Double, Pair<Double, Integer>, NotUsed> driftFlow = Flow.of(Double.class)
.expand(d -> new Pair<Double, Integer>(d, 0), t -> {
return new Pair<>(t, new Pair<>(t.first(), t.second() + 1));
});
//#expand-drift
final TestLatch latch = new TestLatch(2, system);
final Flow<Double, Pair<Double, Integer>, BoxedUnit> realDriftFlow = Flow.of(Double.class)
final Flow<Double, Pair<Double, Integer>, NotUsed> realDriftFlow = Flow.of(Double.class)
.expand(d -> { latch.countDown(); return new Pair<Double, Integer>(d, 0); }, t -> {
return new Pair<>(t, new Pair<>(t.first(), t.second() + 1));
});

View file

@ -4,6 +4,7 @@
package docs.stream;
import akka.NotUsed;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.japi.Pair;
@ -25,7 +26,6 @@ import org.reactivestreams.Processor;
//#imports
import org.reactivestreams.Subscription;
import scala.runtime.BoxedUnit;
import java.lang.Exception;
@ -55,7 +55,7 @@ public class ReactiveStreamsDocTest {
static class Data {
static //#authors
final Flow<Tweet, Author, BoxedUnit> authors = Flow.of(Tweet.class)
final Flow<Tweet, Author, NotUsed> authors = Flow.of(Tweet.class)
.filter(t -> t.hashtags().contains(AKKA))
.map(t -> t.author);
@ -243,7 +243,7 @@ public class ReactiveStreamsDocTest {
}
};
final Flow<Integer, Integer, BoxedUnit> flow = Flow.fromProcessor(factory);
final Flow<Integer, Integer, NotUsed> flow = Flow.fromProcessor(factory);
//#use-processor
}

View file

@ -6,12 +6,12 @@ package docs.stream;
import java.util.Arrays;
import java.util.concurrent.TimeUnit;
import akka.NotUsed;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
import akka.actor.ActorSystem;
import akka.actor.Cancellable;
import akka.stream.*;
@ -60,11 +60,11 @@ public class StreamBuffersRateDocTest {
//#materializer-buffer
//#section-buffer
final Flow<Integer, Integer, BoxedUnit> flow1 =
final Flow<Integer, Integer, NotUsed> flow1 =
Flow.of(Integer.class)
.map(elem -> elem * 2) // the buffer size of this map is 1
.withAttributes(Attributes.inputBuffer(1, 1));
final Flow<Integer, Integer, BoxedUnit> flow2 =
final Flow<Integer, Integer, NotUsed> flow2 =
flow1.via(
Flow.of(Integer.class)
.map(elem -> elem / 2)); // the buffer size of this map is the default
@ -80,7 +80,7 @@ public class StreamBuffersRateDocTest {
Source.tick(oneSecond, oneSecond, "message!");
final Source<String, Cancellable> tickSource =
Source.tick(oneSecond.mul(3), oneSecond.mul(3), "tick");
final Flow<String, Integer, BoxedUnit> conflate =
final Flow<String, Integer, NotUsed> conflate =
Flow.of(String.class).conflate(
first -> 1, (count, elem) -> count + 1);
@ -97,10 +97,10 @@ public class StreamBuffersRateDocTest {
@Test
public void demonstrateExplicitBuffers() {
final Source<Job, BoxedUnit> inboundJobsConnector = Source.empty();
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, BoxedUnit> jobs = inboundJobsConnector;
final Source<Job, NotUsed> jobs = inboundJobsConnector;
jobs.buffer(1000, OverflowStrategy.backpressure());
//#explicit-buffers-backpressure

View file

@ -8,6 +8,8 @@ import static org.junit.Assert.assertEquals;
import java.util.*;
import java.util.concurrent.TimeUnit;
import akka.Done;
import akka.NotUsed;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -15,7 +17,6 @@ import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.Duration;
import scala.runtime.BoxedUnit;
import akka.actor.*;
import akka.japi.Pair;
import akka.stream.*;
@ -43,10 +44,10 @@ public class StreamPartialFlowGraphDocTest {
@Test
public void demonstrateBuildWithOpenPorts() throws Exception {
//#simple-partial-flow-graph
final Graph<FanInShape2<Integer, Integer, Integer>, BoxedUnit> zip =
final Graph<FanInShape2<Integer, Integer, Integer>, NotUsed> zip =
ZipWith.create((Integer left, Integer right) -> Math.max(left, right));
final Graph<UniformFanInShape<Integer, Integer>, BoxedUnit> pickMaxOfThree =
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);
@ -95,9 +96,9 @@ public class StreamPartialFlowGraphDocTest {
@Test
public void demonstrateBuildSourceFromPartialFlowGraphCreate() throws Exception {
//#source-from-partial-flow-graph
final Source<Integer, BoxedUnit> ints = Source.fromIterator(() -> new Ints());
final Source<Integer, NotUsed> ints = Source.fromIterator(() -> new Ints());
final Source<Pair<Integer, Integer>, BoxedUnit> pairs = Source.fromGraph(
final Source<Pair<Integer, Integer>, NotUsed> pairs = Source.fromGraph(
GraphDSL.create(
builder -> {
final FanInShape2<Integer, Integer, Pair<Integer, Integer>> zip =
@ -118,7 +119,7 @@ public class StreamPartialFlowGraphDocTest {
@Test
public void demonstrateBuildFlowFromPartialFlowGraphCreate() throws Exception {
//#flow-from-partial-flow-graph
final Flow<Integer, Pair<Integer, String>, BoxedUnit> pairs = Flow.fromGraph(GraphDSL.create(
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 =
@ -143,10 +144,10 @@ public class StreamPartialFlowGraphDocTest {
@Test
public void demonstrateBuildSourceWithCombine() throws Exception {
//#source-combine
Source<Integer, BoxedUnit> source1 = Source.single(1);
Source<Integer, BoxedUnit> source2 = Source.single(2);
Source<Integer, NotUsed> source1 = Source.single(1);
Source<Integer, NotUsed> source2 = Source.single(2);
final Source<Integer, BoxedUnit> sources = Source.combine(source1, source2, new ArrayList<>(),
final Source<Integer, NotUsed> sources = Source.combine(source1, source2, new ArrayList<>(),
i -> Merge.<Integer>create(i));
//#source-combine
final Future<Integer> result=
@ -163,9 +164,9 @@ public class StreamPartialFlowGraphDocTest {
ActorRef actorRef = probe.getRef();
//#sink-combine
Sink<Integer, BoxedUnit> sendRmotely = Sink.actorRef(actorRef, "Done");
Sink<Integer, Future<BoxedUnit>> localProcessing = Sink.<Integer>foreach(a -> { /*do something useful*/ } );
Sink<Integer, BoxedUnit> sinks = Sink.combine(sendRmotely,localProcessing, new ArrayList<>(), a -> Broadcast.create(a));
Sink<Integer, NotUsed> sendRemotely = Sink.actorRef(actorRef, "Done");
Sink<Integer, Future<Done>> localProcessing = Sink.<Integer>foreach(a -> { /*do something useful*/ } );
Sink<Integer, NotUsed> sinks = Sink.combine(sendRemotely,localProcessing, new ArrayList<>(), a -> Broadcast.create(a));
Source.<Integer>from(Arrays.asList(new Integer[]{0, 1, 2})).runWith(sinks, mat);
//#sink-combine

View file

@ -8,6 +8,7 @@ import java.util.Collections;
import java.util.List;
import java.util.concurrent.TimeUnit;
import akka.NotUsed;
import org.junit.*;
import static org.junit.Assert.assertEquals;
@ -26,7 +27,6 @@ import scala.concurrent.Future;
import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
public class StreamTestKitDocTest {
@ -62,7 +62,7 @@ public class StreamTestKitDocTest {
@Test
public void groupedPartOfInfiniteStream() throws Exception {
//#grouped-infinite
final Source<Integer, BoxedUnit> sourceUnderTest = Source.repeat(1)
final Source<Integer, NotUsed> sourceUnderTest = Source.repeat(1)
.map(i -> i * 2);
final Future<List<Integer>> future = sourceUnderTest
@ -77,7 +77,7 @@ public class StreamTestKitDocTest {
@Test
public void foldedStream() throws Exception {
//#folded-stream
final Flow<Integer, Integer, BoxedUnit> flowUnderTest = Flow.of(Integer.class)
final Flow<Integer, Integer, NotUsed> flowUnderTest = Flow.of(Integer.class)
.takeWhile(i -> i < 5);
final Future<Integer> future = Source.from(Arrays.asList(1, 2, 3, 4, 5, 6))
@ -90,7 +90,7 @@ public class StreamTestKitDocTest {
@Test
public void pipeToTestProbe() throws Exception {
//#pipeto-testprobe
final Source<List<Integer>, BoxedUnit> sourceUnderTest = Source
final Source<List<Integer>, NotUsed> sourceUnderTest = Source
.from(Arrays.asList(1, 2, 3, 4))
.grouped(2);
@ -153,7 +153,7 @@ public class StreamTestKitDocTest {
@Test
public void testSinkProbe() {
//#test-sink-probe
final Source<Integer, BoxedUnit> sourceUnderTest = Source.from(Arrays.asList(1, 2, 3, 4))
final Source<Integer, NotUsed> sourceUnderTest = Source.from(Arrays.asList(1, 2, 3, 4))
.filter(elem -> elem % 2 == 0)
.map(elem -> elem * 2);
@ -168,7 +168,7 @@ public class StreamTestKitDocTest {
@Test
public void testSourceProbe() {
//#test-source-probe
final Sink<Integer, BoxedUnit> sinkUnderTest = Sink.cancelled();
final Sink<Integer, NotUsed> sinkUnderTest = Sink.cancelled();
TestSource.<Integer>probe(system)
.toMat(sinkUnderTest, Keep.left())
@ -199,7 +199,7 @@ public class StreamTestKitDocTest {
@Test
public void testSourceAndTestSink() throws Exception {
//#test-source-and-sink
final Flow<Integer, Integer, BoxedUnit> flowUnderTest = Flow.of(Integer.class)
final Flow<Integer, Integer, NotUsed> flowUnderTest = Flow.of(Integer.class)
.mapAsyncUnordered(2, sleep -> akka.pattern.Patterns.after(
Duration.create(10, TimeUnit.MILLISECONDS),
system.scheduler(),

View file

@ -3,6 +3,8 @@
*/
package docs.stream;
import akka.Done;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.dispatch.Foreach;
import akka.japi.JavaPartialFunction;
@ -20,7 +22,6 @@ import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
import java.util.ArrayList;
import java.util.Arrays;
@ -167,7 +168,7 @@ public class TwitterStreamQuickstartDocTest {
public static final Hashtag AKKA = new Hashtag("#akka");
//#model
public static final Source<Tweet, BoxedUnit> tweets = Source.from(
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 !"),
@ -184,7 +185,7 @@ public class TwitterStreamQuickstartDocTest {
static abstract class Example0 {
//#tweet-source
Source<Tweet, BoxedUnit> tweets;
Source<Tweet, NotUsed> tweets;
//#tweet-source
}
@ -220,7 +221,7 @@ public class TwitterStreamQuickstartDocTest {
//#first-sample
//#authors-filter-map
final Source<Author, BoxedUnit> authors =
final Source<Author, NotUsed> authors =
tweets
.filter(t -> t.hashtags().contains(AKKA))
.map(t -> t.author);
@ -241,7 +242,7 @@ public class TwitterStreamQuickstartDocTest {
}
};
final Source<Author, BoxedUnit> authors =
final Source<Author, NotUsed> authors =
tweets.collect(collectFunction);
//#authors-collect
};
@ -261,22 +262,22 @@ public class TwitterStreamQuickstartDocTest {
@Test
public void demonstrateMapConcat() {
//#hashtags-mapConcat
final Source<Hashtag, BoxedUnit> hashtags =
final Source<Hashtag, NotUsed> hashtags =
tweets.mapConcat(t -> new ArrayList<Hashtag>(t.hashtags()));
//#hashtags-mapConcat
}
static abstract class HiddenDefinitions {
//#flow-graph-broadcast
Sink<Author, BoxedUnit> writeAuthors;
Sink<Hashtag, BoxedUnit> writeHashtags;
Sink<Author, NotUsed> writeAuthors;
Sink<Hashtag, NotUsed> writeHashtags;
//#flow-graph-broadcast
}
@Test
public void demonstrateBroadcast() {
final Sink<Author, Future<BoxedUnit>> writeAuthors = Sink.ignore();
final Sink<Hashtag, Future<BoxedUnit>> writeHashtags = Sink.ignore();
final Sink<Author, Future<Done>> writeAuthors = Sink.ignore();
final Sink<Hashtag, Future<Done>> writeHashtags = Sink.ignore();
//#flow-graph-broadcast
RunnableGraph.fromGraph(GraphDSL.create(b -> {
@ -340,7 +341,7 @@ public class TwitterStreamQuickstartDocTest {
@Test
public void demonstrateMaterializeMultipleTimes() {
final Source<Tweet, BoxedUnit> tweetsInMinuteFromNow = tweets; // not really in second, just acting as if
final Source<Tweet, NotUsed> tweetsInMinuteFromNow = tweets; // not really in second, just acting as if
//#tweets-runnable-flow-materialized-twice
final Sink<Integer, Future<Integer>> sumSink =

View file

@ -8,6 +8,7 @@ import static org.junit.Assert.assertEquals;
import java.io.File;
import java.io.IOException;
import akka.Done;
import akka.actor.ActorSystem;
import akka.stream.ActorAttributes;
import akka.stream.javadsl.Sink;
@ -18,7 +19,6 @@ import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.Future;
import scala.runtime.BoxedUnit;
import akka.stream.*;
import akka.testkit.JavaTestKit;
@ -55,7 +55,7 @@ public class StreamFileDocTest {
try {
//#file-source
Sink<ByteString, Future<BoxedUnit>> printlnSink =
Sink<ByteString, Future<Done>> printlnSink =
Sink.foreach(chunk -> System.out.println(chunk.utf8String()));
Future<Long> bytesWritten =

View file

@ -5,6 +5,7 @@ package docs.stream.io;
import java.util.concurrent.ConcurrentLinkedQueue;
import akka.NotUsed;
import akka.stream.io.Framing;
import docs.stream.SilenceSystemOut;
import java.net.InetSocketAddress;
@ -14,7 +15,6 @@ import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.Future;
import scala.runtime.BoxedUnit;
import akka.actor.ActorSystem;
import akka.stream.*;
@ -74,7 +74,7 @@ public class StreamTcpDocTest {
connections.runForeach(connection -> {
System.out.println("New connection from: " + connection.remoteAddress());
final Flow<ByteString, ByteString, BoxedUnit> echo = Flow.of(ByteString.class)
final Flow<ByteString, ByteString, NotUsed> echo = Flow.of(ByteString.class)
.via(Framing.delimiter(ByteString.fromString("\n"), 256, false))
.map(bytes -> bytes.utf8String())
.map(s -> s + "!!!\n")
@ -110,9 +110,9 @@ public class StreamTcpDocTest {
final String welcomeMsg = "Welcome to: " + connection.localAddress() +
" you are: " + connection.remoteAddress() + "!\n";
final Source<ByteString, BoxedUnit> welcome =
final Source<ByteString, NotUsed> welcome =
Source.single(ByteString.fromString(welcomeMsg));
final Flow<ByteString, ByteString, BoxedUnit> echoFlow =
final Flow<ByteString, ByteString, NotUsed> echoFlow =
Flow.of(ByteString.class)
.via(Framing.delimiter(ByteString.fromString("\n"), 256, false))
.map(bytes -> bytes.utf8String())
@ -126,7 +126,7 @@ public class StreamTcpDocTest {
.map(s -> s + "\n")
.map(s -> ByteString.fromString(s));
final Flow<ByteString, ByteString, BoxedUnit> serverLogic =
final Flow<ByteString, ByteString, NotUsed> serverLogic =
Flow.fromGraph(GraphDSL.create(builder -> {
final UniformFanInShape<ByteString, ByteString> concat =
builder.add(Concat.create());
@ -165,7 +165,7 @@ public class StreamTcpDocTest {
}
};
final Flow<ByteString, ByteString, BoxedUnit> repl = Flow.of(ByteString.class)
final Flow<ByteString, ByteString, NotUsed> repl = Flow.of(ByteString.class)
.via(Framing.delimiter(ByteString.fromString("\n"), 256, false))
.map(bytes -> bytes.utf8String())
.map(text -> {System.out.println("Server: " + text); return "next";})

View file

@ -3,6 +3,7 @@
*/
package docs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
@ -22,7 +23,6 @@ import scala.Tuple2;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
import java.util.Arrays;
import java.util.List;
@ -47,7 +47,7 @@ public class RecipeByteStrings extends RecipeTest {
final Materializer mat = ActorMaterializer.create(system);
final Source<ByteString, BoxedUnit> rawBytes = Source.from(Arrays.asList(
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 }),
@ -93,7 +93,7 @@ public class RecipeByteStrings extends RecipeTest {
{
//#bytestring-chunker2
Source<ByteString, BoxedUnit> chunksStream =
Source<ByteString, NotUsed> chunksStream =
rawBytes.transform(() -> new Chunker(CHUNK_LIMIT));
//#bytestring-chunker2
@ -143,17 +143,17 @@ public class RecipeByteStrings extends RecipeTest {
{
//#bytes-limiter2
Flow<ByteString, ByteString, BoxedUnit> limiter =
Flow<ByteString, ByteString, NotUsed> limiter =
Flow.of(ByteString.class).transform(() -> new ByteLimiter(SIZE_LIMIT));
//#bytes-limiter2
final Source<ByteString, BoxedUnit> bytes1 = Source.from(Arrays.asList(
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, BoxedUnit> bytes2 = Source.from(Arrays.asList(
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 }),
@ -184,14 +184,14 @@ public class RecipeByteStrings extends RecipeTest {
public void compacting() throws Exception {
new JavaTestKit(system) {
{
final Source<ByteString, BoxedUnit> rawBytes = Source.from(Arrays.asList(
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, BoxedUnit> compacted = rawBytes.map(bs -> bs.compact());
Source<ByteString, NotUsed> compacted = rawBytes.map(bs -> bs.compact());
//#compacting-bytestrings
FiniteDuration timeout = FiniteDuration.create(3, TimeUnit.SECONDS);

View file

@ -3,6 +3,7 @@
*/
package docs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
@ -19,7 +20,6 @@ import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.duration.Duration;
import scala.runtime.BoxedUnit;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
@ -80,12 +80,12 @@ public class RecipeDigest extends RecipeTest {
//#calculating-digest
{
Source<ByteString, BoxedUnit> data = Source.from(Arrays.asList(
Source<ByteString, NotUsed> data = Source.from(Arrays.asList(
ByteString.fromString("abcdbcdecdef"),
ByteString.fromString("defgefghfghighijhijkijkljklmklmnlmnomnopnopq")));
//#calculating-digest2
final Source<ByteString, BoxedUnit> digest = data
final Source<ByteString, NotUsed> digest = data
.transform(() -> digestCalculator("SHA-256"));
//#calculating-digest2

View file

@ -3,6 +3,8 @@
*/
package docs.stream.javadsl.cookbook;
import akka.Done;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.*;
import akka.stream.javadsl.*;
@ -11,7 +13,6 @@ import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.Future;
import scala.runtime.BoxedUnit;
import java.util.ArrayList;
import java.util.List;
@ -37,7 +38,7 @@ public class RecipeDroppyBroadcast extends RecipeTest {
new JavaTestKit(system) {
//#droppy-bcast
// Makes a sink drop elements if too slow
public <T> Sink<T, Future<BoxedUnit>> droppySink(Sink<T, Future<BoxedUnit>> sink, int size) {
public <T> Sink<T, Future<Done>> droppySink(Sink<T, Future<Done>> sink, int size) {
return Flow.<T> create()
.buffer(size, OverflowStrategy.dropHead())
.toMat(sink, Keep.right());
@ -50,11 +51,11 @@ public class RecipeDroppyBroadcast extends RecipeTest {
nums.add(i + 1);
}
final Sink<Integer, Future<BoxedUnit>> mySink1 = Sink.ignore();
final Sink<Integer, Future<BoxedUnit>> mySink2 = Sink.ignore();
final Sink<Integer, Future<BoxedUnit>> mySink3 = Sink.ignore();
final Sink<Integer, Future<Done>> mySink1 = Sink.ignore();
final Sink<Integer, Future<Done>> mySink2 = Sink.ignore();
final Sink<Integer, Future<Done>> mySink3 = Sink.ignore();
final Source<Integer, BoxedUnit> myData = Source.from(nums);
final Source<Integer, NotUsed> myData = Source.from(nums);
//#droppy-bcast2
RunnableGraph.fromGraph(GraphDSL.create(builder -> {

View file

@ -3,6 +3,7 @@
*/
package docs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
@ -14,7 +15,6 @@ import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
import java.util.Arrays;
import java.util.List;
@ -42,12 +42,12 @@ public class RecipeFlattenList extends RecipeTest {
public void workWithMapConcat() throws Exception {
new JavaTestKit(system) {
{
Source<List<Message>, BoxedUnit> someDataSource = Source
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>, BoxedUnit> myData = someDataSource;
Source<Message, BoxedUnit> flattened = myData.mapConcat(i -> i);
Source<List<Message>, NotUsed> myData = someDataSource;
Source<Message, NotUsed> flattened = myData.mapConcat(i -> i);
//#flattening-lists
List<Message> got = Await.result(flattened.grouped(10).runWith(Sink.head(), mat),

View file

@ -3,6 +3,7 @@
*/
package docs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.*;
import akka.dispatch.Mapper;
import akka.japi.pf.ReceiveBuilder;
@ -22,6 +23,7 @@ import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
import java.util.*;
import java.util.concurrent.TimeUnit;
@ -142,9 +144,9 @@ public class RecipeGlobalRateLimit extends RecipeTest {
public void work() throws Exception {
new JavaTestKit(system) {
//#global-limiter-flow
public <T> Flow<T, T, BoxedUnit> limitGlobal(ActorRef limiter, FiniteDuration maxAllowedWait) {
public <T> Flow<T, T, NotUsed> limitGlobal(ActorRef limiter, FiniteDuration maxAllowedWait) {
final int parallelism = 4;
final Flow<T, T, BoxedUnit> f = Flow.create();
final Flow<T, T, NotUsed> f = Flow.create();
return f.mapAsync(parallelism, element -> {
final Timeout triggerTimeout = new Timeout(maxAllowedWait);

View file

@ -3,6 +3,7 @@
*/
package docs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
@ -12,7 +13,6 @@ import akka.util.ByteString;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.runtime.BoxedUnit;
import java.util.concurrent.TimeUnit;
@ -43,7 +43,7 @@ public class RecipeKeepAlive extends RecipeTest {
//@formatter:off
//#inject-keepalive
Flow<ByteString, ByteString, BoxedUnit> keepAliveInject =
Flow<ByteString, ByteString, NotUsed> keepAliveInject =
Flow.of(ByteString.class).keepAlive(
scala.concurrent.duration.Duration.create(1, TimeUnit.SECONDS),
() -> keepAliveMessage);

View file

@ -3,6 +3,7 @@
*/
package docs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.event.Logging;
import akka.event.LoggingAdapter;
@ -19,7 +20,6 @@ import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.runtime.AbstractFunction0;
import scala.runtime.BoxedUnit;
import java.util.Arrays;
@ -45,7 +45,7 @@ public class RecipeLoggingElements extends RecipeTest {
final SilenceSystemOut.System System = SilenceSystemOut.get(getTestActor());
{
final Source<String, BoxedUnit> mySource = Source.from(Arrays.asList("1", "2", "3"));
final Source<String, NotUsed> mySource = Source.from(Arrays.asList("1", "2", "3"));
//#println-debug
mySource.map(elem -> {
@ -65,7 +65,7 @@ public class RecipeLoggingElements extends RecipeTest {
}
{
final Source<String, BoxedUnit> mySource = Source.from(Arrays.asList("1", "2", "3"));
final Source<String, NotUsed> mySource = Source.from(Arrays.asList("1", "2", "3"));
final int onElement = Logging.WarningLevel();
final int onFinish = Logging.ErrorLevel();
@ -82,7 +82,8 @@ public class RecipeLoggingElements extends RecipeTest {
mySource.log("custom", adapter);
//#log-custom
new DebugFilter("customLogger", "[custom] Element: ", false, false, 3).intercept(new AbstractFunction0 () {
new DebugFilter("customLogger", "[custom] Element: ", false, false, 3).intercept(new AbstractFunction0<Object> () {
public Void apply() {
mySource.log("custom", adapter).runWith(Sink.ignore(), mat);
return null;

View file

@ -3,6 +3,7 @@
*/
package docs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.japi.Pair;
import akka.stream.ActorMaterializer;
@ -23,7 +24,6 @@ import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
import java.util.concurrent.TimeUnit;
@ -57,11 +57,11 @@ public class RecipeMissedTicks extends RecipeTest {
@SuppressWarnings("unused")
//#missed-ticks
final Flow<Tick, Integer, BoxedUnit> missedTicks =
final Flow<Tick, Integer, NotUsed> missedTicks =
Flow.of(Tick.class).conflate(tick -> 0, (missed, tick) -> missed + 1);
//#missed-ticks
final TestLatch latch = new TestLatch(3, system);
final Flow<Tick, Integer, BoxedUnit> realMissedTicks =
final Flow<Tick, Integer, NotUsed> realMissedTicks =
Flow.of(Tick.class).conflate(tick -> 0, (missed, tick) -> { latch.countDown(); return missed + 1; });
Pair<TestPublisher.Probe<Tick>, TestSubscriber.Probe<Integer>> pubSub =

View file

@ -3,6 +3,7 @@
*/
package docs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.japi.Function;
import akka.japi.Pair;
@ -18,7 +19,6 @@ import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
import java.util.ArrayList;
import java.util.Arrays;
@ -93,14 +93,14 @@ public class RecipeMultiGroupByTest extends RecipeTest {
{
final Source<Message, BoxedUnit> elems = Source
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>, BoxedUnit> messageAndTopic = elems
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
@ -112,7 +112,7 @@ public class RecipeMultiGroupByTest extends RecipeTest {
.collect(toList());
});
SubSource<Pair<Message, Topic>, BoxedUnit> multiGroups = messageAndTopic
SubSource<Pair<Message, Topic>, NotUsed> multiGroups = messageAndTopic
.groupBy(2, pair -> pair.second())
.map(pair -> {
Message message = pair.first();

View file

@ -3,6 +3,7 @@
*/
package docs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
@ -16,7 +17,6 @@ import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
import java.util.Arrays;
import java.util.concurrent.TimeUnit;
@ -40,7 +40,7 @@ public class RecipeParseLines extends RecipeTest {
@Test
public void parseLines() throws Exception {
final Source<ByteString, BoxedUnit> rawData = Source.from(Arrays.asList(
final Source<ByteString, NotUsed> rawData = Source.from(Arrays.asList(
ByteString.fromString("Hello World"),
ByteString.fromString("\r"),
ByteString.fromString("!\r"),
@ -48,7 +48,7 @@ public class RecipeParseLines extends RecipeTest {
ByteString.fromString("\r\n\r\n")));
//#parse-lines
final Source<String, BoxedUnit> lines = rawData
final Source<String, NotUsed> lines = rawData
.via(Framing.delimiter(ByteString.fromString("\r\n"), 100, true))
.map(b -> b.utf8String());
//#parse-lines

View file

@ -3,6 +3,7 @@
*/
package docs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.japi.Pair;
import akka.japi.function.Function;
@ -19,7 +20,6 @@ import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.runtime.BoxedUnit;
import java.util.Arrays;
import java.util.HashSet;
@ -47,12 +47,12 @@ public class RecipeReduceByKeyTest extends RecipeTest {
public void work() throws Exception {
new JavaTestKit(system) {
{
final Source<String, BoxedUnit> words = Source.from(Arrays.asList("hello", "world", "and", "hello", "akka"));
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>, BoxedUnit> counts = words
final Source<Pair<String, Integer>, NotUsed> counts = words
// split the words into separate streams first
.groupBy(MAXIMUM_DISTINCT_WORDS, i -> i)
// add counting logic to the streams
@ -74,7 +74,7 @@ public class RecipeReduceByKeyTest extends RecipeTest {
}
//#reduce-by-key-general
static public <In, K, Out> Flow<In, Pair<K, Out>, BoxedUnit> reduceByKey(
static public <In, K, Out> Flow<In, Pair<K, Out>, NotUsed> reduceByKey(
int maximumGroupSize,
Function<In, K> groupKey,
Function<K, Out> foldZero,
@ -96,12 +96,12 @@ public class RecipeReduceByKeyTest extends RecipeTest {
public void workGeneralised() throws Exception {
new JavaTestKit(system) {
{
final Source<String, BoxedUnit> words = Source.from(Arrays.asList("hello", "world", "and", "hello", "akka"));
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>, BoxedUnit> counts = words.via(reduceByKey(
Source<Pair<String, Integer>, NotUsed> counts = words.via(reduceByKey(
MAXIMUM_DISTINCT_WORDS,
word -> word,
key -> 0,

View file

@ -3,6 +3,7 @@
*/
package docs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.japi.Pair;
import akka.stream.ActorMaterializer;
@ -19,7 +20,6 @@ import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.duration.Duration;
import scala.runtime.BoxedUnit;
import java.util.concurrent.TimeUnit;
@ -45,11 +45,11 @@ public class RecipeSimpleDrop extends RecipeTest {
{
@SuppressWarnings("unused")
//#simple-drop
final Flow<Message, Message, BoxedUnit> droppyStream =
final Flow<Message, Message, NotUsed> droppyStream =
Flow.of(Message.class).conflate(i -> i, (lastMessage, newMessage) -> newMessage);
//#simple-drop
final TestLatch latch = new TestLatch(2, system);
final Flow<Message, Message, BoxedUnit> realDroppyStream =
final Flow<Message, Message, NotUsed> realDroppyStream =
Flow.of(Message.class).conflate(i -> i, (lastMessage, newMessage) -> { latch.countDown(); return newMessage; });
final Pair<TestPublisher.Probe<Message>, TestSubscriber.Probe<Message>> pubSub = TestSource

View file

@ -3,6 +3,7 @@
*/
package docs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
@ -15,7 +16,6 @@ import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
import java.util.Arrays;
import java.util.List;
@ -41,7 +41,7 @@ public class RecipeToStrict extends RecipeTest {
public void workWithPrintln() throws Exception {
new JavaTestKit(system) {
{
final Source<String, BoxedUnit> myData = Source.from(Arrays.asList("1", "2", "3"));
final Source<String, NotUsed> myData = Source.from(Arrays.asList("1", "2", "3"));
final int MAX_ALLOWED_SIZE = 100;
//#draining-to-list

View file

@ -3,6 +3,7 @@
*/
package docs.stream.javadsl.cookbook;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.stream.*;
import akka.stream.javadsl.*;
@ -13,7 +14,6 @@ import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
import java.util.Arrays;
import java.util.List;
@ -38,8 +38,8 @@ public class RecipeWorkerPool extends RecipeTest {
final Materializer mat = ActorMaterializer.create(system);
//#worker-pool
public static <In, Out> Flow<In, Out, BoxedUnit> balancer(
Flow<In, Out, BoxedUnit> worker, int workerCount) {
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 =
@ -60,16 +60,16 @@ public class RecipeWorkerPool extends RecipeTest {
public void workForVersion1() throws Exception {
new JavaTestKit(system) {
{
Source<Message, BoxedUnit> data =
Source<Message, NotUsed> data =
Source
.from(Arrays.asList("1", "2", "3", "4", "5"))
.map(t -> new Message(t));
Flow<Message, Message, BoxedUnit> worker = Flow.of(Message.class).map(m -> new Message(m.msg + " done"));
Flow<Message, Message, NotUsed> worker = Flow.of(Message.class).map(m -> new Message(m.msg + " done"));
//#worker-pool2
Flow<Message, Message, BoxedUnit> balancer = balancer(worker, 3);
Source<Message, BoxedUnit> processedJobs = data.via(balancer);
Flow<Message, Message, NotUsed> balancer = balancer(worker, 3);
Source<Message, NotUsed> processedJobs = data.via(balancer);
//#worker-pool2
FiniteDuration timeout = FiniteDuration.create(200, TimeUnit.MILLISECONDS);

View file

@ -73,7 +73,7 @@ to "run" the HTTP layer (and, potentially, higher-layers) against data that do n
some other source. Potential scenarios where this might be useful include tests, debugging or low-level event-sourcing
(e.g by replaying network traffic).
On the client-side the stand-alone HTTP layer forms a ``BidiFlow<HttpRequest, SslTlsOutbound, SslTlsInbound, HttpResponse, BoxedUnit>``,
On the client-side the stand-alone HTTP layer forms a ``BidiFlow<HttpRequest, SslTlsOutbound, SslTlsInbound, HttpResponse, NotUsed>``,
that is a stage that "upgrades" a potentially encrypted raw connection to the HTTP level.
You create an instance of the layer by calling one of the two overloads of the ``Http.get(system).clientLayer`` method,

View file

@ -160,7 +160,7 @@ to "run" the HTTP layer (and, potentially, higher-layers) against data that do n
some other source. Potential scenarios where this might be useful include tests, debugging or low-level event-sourcing
(e.g by replaying network traffic).
On the server-side the stand-alone HTTP layer forms a ``BidiFlow<HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest, BoxedUnit>``,
On the server-side the stand-alone HTTP layer forms a ``BidiFlow<HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest, NotUsed>``,
that is a stage that "upgrades" a potentially encrypted raw connection to the HTTP level.
You create an instance of the layer by calling one of the two overloads of the ``Http.get(system).serverLayer`` method,

View file

@ -6,3 +6,47 @@ Migration Guide Akka Streams 2.0.x to 2.4.x
General notes
=============
akka.Done and akka.NotUsed replacing Unit and BoxedUnit
-------------------------------------------------------
To provide more clear signatures and have a unified API for both
Java and Scala two new types have been introduced:
``akka.NotUsed`` is meant to be used instead of ``Unit`` in Scala
and ``BoxedUnit`` in Java to signify that the type parameter is required
but not actually used. This is commonly the case with ``Source``s, ``Flow``s and ``Sink``s
that do not materialize into any value.
``akka.Done`` is added for the use case where it is boxed inside another object to signify
completion but there is no actual value attached to the completion. It is used to replace
occurrences of ``Future<BoxedUnit>`` with ``Future<Done>`` in Java and ``Future[Unit]`` with
``Future[Done]`` in Scala.
All previous usage of ``Unit`` and ``BoxedUnit`` for these two cases in the akka streams APIs
has been updated.
This means that Java code like this::
Source<String, BoxedUnit> source = Source.from(Arrays.asList("1", "2", "3"));
Sink<String, Future<BoxedUnit>> sink = Sink.ignore()
needs to be changed into::
Source<String, NotUsed> source = Source.from(Arrays.asList("1", "2", "3"));
Sink<String, Future<Done>> sink = Sink.ignore()
And Scala code like this::
Source[Int, Unit] source = Source.from(1 to 5)
Sink[Int, Future[Unit]] sink = Sink.ignore()
needs to be changed into::
Source[Int, NotUsed] source = Source.from(1 to 5)
Sink[Int, Future[Done]] sink = Sink.ignore()
These changes apply to all the places where streams are used, which means that signatures
in the persistent query APIs also are affected.

View file

@ -5,6 +5,7 @@ package docs.http.scaladsl.server
import java.io.File
import akka.Done
import akka.actor.ActorRef
import akka.http.scaladsl.model.Multipart.FormData.BodyPart
import akka.stream.io.{ Framing }
@ -73,18 +74,18 @@ class FileUploadExamplesSpec extends RoutingSpec {
val csvUploads =
path("metadata" / LongNumber) { id =>
entity(as[Multipart.FormData]) { formData =>
val done = formData.parts.mapAsync(1) {
val done: Future[Done] = formData.parts.mapAsync(1) {
case b: BodyPart if b.filename.exists(_.endsWith(".csv")) =>
b.entity.dataBytes
.via(splitLines)
.map(_.utf8String.split(",").toVector)
.runForeach(csv =>
metadataActor ! MetadataActor.Entry(id, csv))
case _ => Future.successful(Unit)
case _ => Future.successful(Done)
}.runWith(Sink.ignore)
// when processing have finished create a response for the user
onSuccess(done) {
onSuccess(done) { _ =>
complete {
"ok!"
}

View file

@ -92,5 +92,4 @@ class FormFieldDirectivesExamplesSpec extends RoutingSpec {
}
}
}

View file

@ -3,6 +3,7 @@
*/
package docs.persistence.query
import akka.NotUsed
import akka.persistence.journal.{ EventAdapter, EventSeq }
import akka.testkit.AkkaSpec
import akka.persistence.query.PersistenceQuery
@ -56,10 +57,10 @@ class LeveldbPersistenceQueryDocSpec(config: String) extends AkkaSpec(config) {
val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](
LeveldbReadJournal.Identifier)
val src: Source[EventEnvelope, Unit] =
val src: Source[EventEnvelope, NotUsed] =
queries.eventsByPersistenceId("some-persistence-id", 0L, Long.MaxValue)
val events: Source[Any, Unit] = src.map(_.event)
val events: Source[Any, NotUsed] = src.map(_.event)
//#EventsByPersistenceId
}
@ -69,7 +70,7 @@ class LeveldbPersistenceQueryDocSpec(config: String) extends AkkaSpec(config) {
val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](
LeveldbReadJournal.Identifier)
val src: Source[String, Unit] = queries.allPersistenceIds()
val src: Source[String, NotUsed] = queries.allPersistenceIds()
//#AllPersistenceIds
}
@ -79,7 +80,7 @@ class LeveldbPersistenceQueryDocSpec(config: String) extends AkkaSpec(config) {
val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](
LeveldbReadJournal.Identifier)
val src: Source[EventEnvelope, Unit] =
val src: Source[EventEnvelope, NotUsed] =
queries.eventsByTag(tag = "green", offset = 0L)
//#EventsByTag
}

View file

@ -4,6 +4,7 @@
package docs.persistence.query
import akka.NotUsed
import akka.actor._
import akka.persistence.{ Recovery, PersistentActor }
import akka.persistence.query._
@ -53,25 +54,25 @@ object PersistenceQueryDocSpec {
config.getDuration("refresh-interval", MILLISECONDS).millis
override def eventsByTag(
tag: String, offset: Long = 0L): Source[EventEnvelope, Unit] = {
tag: String, offset: Long = 0L): Source[EventEnvelope, NotUsed] = {
val props = MyEventsByTagPublisher.props(tag, offset, refreshInterval)
Source.actorPublisher[EventEnvelope](props)
.mapMaterializedValue(_ ())
.mapMaterializedValue(_ NotUsed)
}
override def eventsByPersistenceId(
persistenceId: String, fromSequenceNr: Long = 0L,
toSequenceNr: Long = Long.MaxValue): Source[EventEnvelope, Unit] = {
toSequenceNr: Long = Long.MaxValue): Source[EventEnvelope, NotUsed] = {
// implement in a similar way as eventsByTag
???
}
override def allPersistenceIds(): Source[String, Unit] = {
override def allPersistenceIds(): Source[String, NotUsed] = {
// implement in a similar way as eventsByTag
???
}
override def currentPersistenceIds(): Source[String, Unit] = {
override def currentPersistenceIds(): Source[String, NotUsed] = {
// implement in a similar way as eventsByTag
???
}
@ -95,19 +96,19 @@ object PersistenceQueryDocSpec {
with akka.persistence.query.javadsl.CurrentPersistenceIdsQuery {
override def eventsByTag(
tag: String, offset: Long = 0L): javadsl.Source[EventEnvelope, Unit] =
tag: String, offset: Long = 0L): javadsl.Source[EventEnvelope, NotUsed] =
scaladslReadJournal.eventsByTag(tag, offset).asJava
override def eventsByPersistenceId(
persistenceId: String, fromSequenceNr: Long = 0L,
toSequenceNr: Long = Long.MaxValue): javadsl.Source[EventEnvelope, Unit] =
toSequenceNr: Long = Long.MaxValue): javadsl.Source[EventEnvelope, NotUsed] =
scaladslReadJournal.eventsByPersistenceId(
persistenceId, fromSequenceNr, toSequenceNr).asJava
override def allPersistenceIds(): javadsl.Source[String, Unit] =
override def allPersistenceIds(): javadsl.Source[String, NotUsed] =
scaladslReadJournal.allPersistenceIds().asJava
override def currentPersistenceIds(): javadsl.Source[String, Unit] =
override def currentPersistenceIds(): javadsl.Source[String, NotUsed] =
scaladslReadJournal.currentPersistenceIds().asJava
// possibility to add more plugin specific queries
@ -201,7 +202,7 @@ class PersistenceQueryDocSpec(s: String) extends AkkaSpec(s) {
"akka.persistence.query.my-read-journal")
// issue query to journal
val source: Source[EventEnvelope, Unit] =
val source: Source[EventEnvelope, NotUsed] =
readJournal.eventsByPersistenceId("user-1337", 0, Long.MaxValue)
// materialize stream, consuming events
@ -220,7 +221,7 @@ class PersistenceQueryDocSpec(s: String) extends AkkaSpec(s) {
//#events-by-tag
// assuming journal is able to work with numeric offsets we can:
val blueThings: Source[EventEnvelope, Unit] =
val blueThings: Source[EventEnvelope, NotUsed] =
readJournal.eventsByTag("blue")
// find top 10 blue things:

View file

@ -3,6 +3,7 @@
*/
package docs.stream
import akka.NotUsed
import akka.stream._
import akka.stream.scaladsl.Tcp.OutgoingConnection
import akka.stream.scaladsl._
@ -181,7 +182,7 @@ class CompositionDocSpec extends AkkaSpec {
val source: Source[Int, Promise[Option[Int]]] = Source.maybe[Int]
// Materializes to Unit (black)
val flow1: Flow[Int, Int, Unit] = Flow[Int].take(100)
val flow1: Flow[Int, Int, NotUsed] = Flow[Int].take(100)
// Materializes to Promise[Int] (red)
val nestedSource: Source[Int, Promise[Option[Int]]] =
@ -190,7 +191,7 @@ class CompositionDocSpec extends AkkaSpec {
//#mat-combine-2
// Materializes to Unit (orange)
val flow2: Flow[Int, ByteString, Unit] = Flow[Int].map { i => ByteString(i.toString) }
val flow2: Flow[Int, ByteString, NotUsed] = Flow[Int].map { i => ByteString(i.toString) }
// Materializes to Future[OutgoingConnection] (yellow)
val flow3: Flow[ByteString, ByteString, Future[OutgoingConnection]] =

View file

@ -3,6 +3,7 @@
*/
package docs.stream
import akka.NotUsed
import akka.actor.Cancellable
import akka.stream.{ ClosedShape, FlowShape }
import akka.stream.scaladsl._
@ -134,11 +135,11 @@ class FlowDocSpec extends AkkaSpec {
source.to(Sink.foreach(println(_)))
// Starting from a Sink
val sink: Sink[Int, Unit] = Flow[Int].map(_ * 2).to(Sink.foreach(println(_)))
val sink: Sink[Int, NotUsed] = Flow[Int].map(_ * 2).to(Sink.foreach(println(_)))
Source(1 to 6).to(sink)
// Broadcast to a sink inline
val otherSink: Sink[Int, Unit] =
val otherSink: Sink[Int, NotUsed] =
Flow[Int].alsoTo(Sink.foreach(println(_))).to(Sink.ignore)
Source(1 to 6).to(otherSink)

View file

@ -3,6 +3,7 @@
*/
package docs.stream
import akka.NotUsed
import akka.stream._
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
@ -21,7 +22,7 @@ class FlowGraphDocSpec extends AkkaSpec {
"build simple graph" in {
//format: OFF
//#simple-flow-graph
val g = RunnableGraph.fromGraph(GraphDSL.create() { implicit builder: GraphDSL.Builder[Unit] =>
val g = RunnableGraph.fromGraph(GraphDSL.create() { implicit builder: GraphDSL.Builder[NotUsed] =>
import GraphDSL.Implicits._
val in = Source(1 to 10)
val out = Sink.ignore
@ -131,7 +132,7 @@ class FlowGraphDocSpec extends AkkaSpec {
object PriorityWorkerPool {
def apply[In, Out](
worker: Flow[In, Out, Any],
workerCount: Int): Graph[PriorityWorkerPoolShape[In, Out], Unit] = {
workerCount: Int): Graph[PriorityWorkerPoolShape[In, Out], NotUsed] = {
GraphDSL.create() { implicit b
import GraphDSL.Implicits._

View file

@ -1,5 +1,6 @@
package docs.stream
import akka.NotUsed
import akka.stream.FlowShape
import akka.stream.scaladsl.{ GraphDSL, Merge, Balance, Source, Flow }
import akka.stream.testkit.AkkaSpec
@ -15,11 +16,11 @@ class FlowParallelismDocSpec extends AkkaSpec {
//format: OFF
//#pipelining
// Takes a scoop of batter and creates a pancake with one side cooked
val fryingPan1: Flow[ScoopOfBatter, HalfCookedPancake, Unit] =
val fryingPan1: Flow[ScoopOfBatter, HalfCookedPancake, NotUsed] =
Flow[ScoopOfBatter].map { batter => HalfCookedPancake() }
// Finishes a half-cooked pancake
val fryingPan2: Flow[HalfCookedPancake, Pancake, Unit] =
val fryingPan2: Flow[HalfCookedPancake, Pancake, NotUsed] =
Flow[HalfCookedPancake].map { halfCooked => Pancake() }
//#pipelining
//format: ON
@ -28,17 +29,17 @@ class FlowParallelismDocSpec extends AkkaSpec {
//#pipelining
// With the two frying pans we can fully cook pancakes
val pancakeChef: Flow[ScoopOfBatter, Pancake, Unit] =
val pancakeChef: Flow[ScoopOfBatter, Pancake, NotUsed] =
Flow[ScoopOfBatter].via(fryingPan1).via(fryingPan2)
//#pipelining
}
"Demonstrate parallel processing" in {
//#parallelism
val fryingPan: Flow[ScoopOfBatter, Pancake, Unit] =
val fryingPan: Flow[ScoopOfBatter, Pancake, NotUsed] =
Flow[ScoopOfBatter].map { batter => Pancake() }
val pancakeChef: Flow[ScoopOfBatter, Pancake, Unit] = Flow.fromGraph(GraphDSL.create() { implicit builder =>
val pancakeChef: Flow[ScoopOfBatter, Pancake, NotUsed] = Flow.fromGraph(GraphDSL.create() { implicit builder =>
val dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
val mergePancakes = builder.add(Merge[Pancake](2))
@ -58,7 +59,7 @@ class FlowParallelismDocSpec extends AkkaSpec {
"Demonstrate parallelized pipelines" in {
//#parallel-pipeline
val pancakeChef: Flow[ScoopOfBatter, Pancake, Unit] =
val pancakeChef: Flow[ScoopOfBatter, Pancake, NotUsed] =
Flow.fromGraph(GraphDSL.create() { implicit builder =>
val dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
@ -76,7 +77,7 @@ class FlowParallelismDocSpec extends AkkaSpec {
"Demonstrate pipelined parallel processing" in {
//#pipelined-parallel
val pancakeChefs1: Flow[ScoopOfBatter, HalfCookedPancake, Unit] =
val pancakeChefs1: Flow[ScoopOfBatter, HalfCookedPancake, NotUsed] =
Flow.fromGraph(GraphDSL.create() { implicit builder =>
val dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
val mergeHalfPancakes = builder.add(Merge[HalfCookedPancake](2))
@ -89,7 +90,7 @@ class FlowParallelismDocSpec extends AkkaSpec {
FlowShape(dispatchBatter.in, mergeHalfPancakes.out)
})
val pancakeChefs2: Flow[HalfCookedPancake, Pancake, Unit] =
val pancakeChefs2: Flow[HalfCookedPancake, Pancake, NotUsed] =
Flow.fromGraph(GraphDSL.create() { implicit builder =>
val dispatchHalfPancakes = builder.add(Balance[HalfCookedPancake](2))
val mergePancakes = builder.add(Merge[Pancake](2))
@ -102,7 +103,7 @@ class FlowParallelismDocSpec extends AkkaSpec {
FlowShape(dispatchHalfPancakes.in, mergePancakes.out)
})
val kitchen: Flow[ScoopOfBatter, Pancake, Unit] = pancakeChefs1.via(pancakeChefs2)
val kitchen: Flow[ScoopOfBatter, Pancake, NotUsed] = pancakeChefs1.via(pancakeChefs2)
//#pipelined-parallel
}

View file

@ -3,6 +3,7 @@
*/
package docs.stream
import akka.NotUsed
import akka.stream.scaladsl.{ Keep, Sink, Flow, Source }
import akka.stream.stage._
import akka.stream._
@ -68,10 +69,10 @@ class GraphStageDocSpec extends AkkaSpec {
//#simple-source-usage
// A GraphStage is a proper Graph, just like what GraphDSL.create would return
val sourceGraph: Graph[SourceShape[Int], Unit] = new NumbersSource
val sourceGraph: Graph[SourceShape[Int], NotUsed] = new NumbersSource
// Create a Source from the Graph to access the DSL
val mySource: Source[Int, Unit] = Source.fromGraph(new NumbersSource)
val mySource: Source[Int, NotUsed] = Source.fromGraph(new NumbersSource)
// Returns 55
val result1: Future[Int] = mySource.take(10).runFold(0)(_ + _)

View file

@ -3,6 +3,8 @@
*/
package docs.stream
import akka.NotUsed
import scala.concurrent.duration._
import akka.stream.testkit.AkkaSpec
import akka.stream.scaladsl._
@ -132,21 +134,21 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
val emailServer = new EmailServer(probe.ref)
//#tweet-authors
val authors: Source[Author, Unit] =
val authors: Source[Author, NotUsed] =
tweets
.filter(_.hashtags.contains(akka))
.map(_.author)
//#tweet-authors
//#email-addresses-mapAsync
val emailAddresses: Source[String, Unit] =
val emailAddresses: Source[String, NotUsed] =
authors
.mapAsync(4)(author => addressSystem.lookupEmail(author.handle))
.collect { case Some(emailAddress) => emailAddress }
//#email-addresses-mapAsync
//#send-emails
val sendEmails: RunnableGraph[Unit] =
val sendEmails: RunnableGraph[NotUsed] =
emailAddresses
.mapAsync(4)(address => {
emailServer.send(
@ -168,14 +170,14 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
"lookup email with mapAsync and supervision" in {
val addressSystem = new AddressSystem2
val authors: Source[Author, Unit] =
val authors: Source[Author, NotUsed] =
tweets.filter(_.hashtags.contains(akka)).map(_.author)
//#email-addresses-mapAsync-supervision
import ActorAttributes.supervisionStrategy
import Supervision.resumingDecider
val emailAddresses: Source[String, Unit] =
val emailAddresses: Source[String, NotUsed] =
authors.via(
Flow[Author].mapAsync(4)(author => addressSystem.lookupEmail(author.handle))
.withAttributes(supervisionStrategy(resumingDecider)))
@ -188,15 +190,15 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
val emailServer = new EmailServer(probe.ref)
//#external-service-mapAsyncUnordered
val authors: Source[Author, Unit] =
val authors: Source[Author, NotUsed] =
tweets.filter(_.hashtags.contains(akka)).map(_.author)
val emailAddresses: Source[String, Unit] =
val emailAddresses: Source[String, NotUsed] =
authors
.mapAsyncUnordered(4)(author => addressSystem.lookupEmail(author.handle))
.collect { case Some(emailAddress) => emailAddress }
val sendEmails: RunnableGraph[Unit] =
val sendEmails: RunnableGraph[NotUsed] =
emailAddresses
.mapAsyncUnordered(4)(address => {
emailServer.send(
@ -231,7 +233,7 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
//#blocking-mapAsync
val blockingExecutionContext = system.dispatchers.lookup("blocking-dispatcher")
val sendTextMessages: RunnableGraph[Unit] =
val sendTextMessages: RunnableGraph[NotUsed] =
phoneNumbers
.mapAsync(4)(phoneNo => {
Future {
@ -271,7 +273,7 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
smsServer.send(TextMessage(to = phoneNo, body = "I like your tweet"))
}
.withAttributes(ActorAttributes.dispatcher("blocking-dispatcher"))
val sendTextMessages: RunnableGraph[Unit] =
val sendTextMessages: RunnableGraph[NotUsed] =
phoneNumbers.via(send).to(Sink.ignore)
sendTextMessages.run()
@ -291,10 +293,10 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
val database = system.actorOf(Props(classOf[DatabaseService], probe.ref), "db")
//#save-tweets
val akkaTweets: Source[Tweet, Unit] = tweets.filter(_.hashtags.contains(akka))
val akkaTweets: Source[Tweet, NotUsed] = tweets.filter(_.hashtags.contains(akka))
implicit val timeout = Timeout(3.seconds)
val saveTweets: RunnableGraph[Unit] =
val saveTweets: RunnableGraph[NotUsed] =
akkaTweets
.mapAsync(4)(tweet => database ? Save(tweet))
.to(Sink.ignore)

View file

@ -3,8 +3,9 @@
*/
package docs.stream
import akka.NotUsed
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.{ RunnableGraph, Flow, Sink, Source }
import akka.stream.scaladsl.{ Flow, Sink, Source }
import akka.stream.testkit._
import org.reactivestreams.Processor
@ -139,7 +140,7 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
// An example Processor factory
def createProcessor: Processor[Int, Int] = Flow[Int].toProcessor.run()
val flow: Flow[Int, Int, Unit] = Flow.fromProcessor(() => createProcessor)
val flow: Flow[Int, Int, NotUsed] = Flow.fromProcessor(() => createProcessor)
//#use-processor
}

View file

@ -1,5 +1,6 @@
package docs.stream
import akka.NotUsed
import akka.stream._
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
@ -57,10 +58,10 @@ class StreamBuffersRateSpec extends AkkaSpec {
"explcit buffers" in {
trait Job
def inboundJobsConnector(): Source[Job, Unit] = Source.empty
def inboundJobsConnector(): Source[Job, NotUsed] = Source.empty
//#explicit-buffers-backpressure
// Getting a stream of jobs from an imaginary external system as a Source
val jobs: Source[Job, Unit] = inboundJobsConnector()
val jobs: Source[Job, NotUsed] = inboundJobsConnector()
jobs.buffer(1000, OverflowStrategy.backpressure)
//#explicit-buffers-backpressure

View file

@ -5,6 +5,7 @@ package docs.stream
//#imports
import akka.{ Done, NotUsed }
import akka.actor.ActorSystem
import akka.stream.{ ClosedShape, ActorMaterializer, OverflowStrategy }
import akka.stream.scaladsl._
@ -73,7 +74,7 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
//#first-sample
//#authors-filter-map
val authors: Source[Author, Unit] =
val authors: Source[Author, NotUsed] =
tweets
.filter(_.hashtags.contains(akka))
.map(_.author)
@ -82,7 +83,7 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
trait Example3 {
//#authors-collect
val authors: Source[Author, Unit] =
val authors: Source[Author, NotUsed] =
tweets.collect { case t if t.hashtags.contains(akka) => t.author }
//#authors-collect
}
@ -101,7 +102,7 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
"mapConcat hashtags" in {
//#hashtags-mapConcat
val hashtags: Source[Hashtag, Unit] = tweets.mapConcat(_.hashtags.toList)
val hashtags: Source[Hashtag, NotUsed] = tweets.mapConcat(_.hashtags.toList)
//#hashtags-mapConcat
}
@ -113,8 +114,8 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
}
"simple broadcast" in {
val writeAuthors: Sink[Author, Future[Unit]] = Sink.ignore
val writeHashtags: Sink[Hashtag, Future[Unit]] = Sink.ignore
val writeAuthors: Sink[Author, Future[Done]] = Sink.ignore
val writeHashtags: Sink[Hashtag, Future[Done]] = Sink.ignore
// format: OFF
//#flow-graph-broadcast
@ -151,7 +152,7 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
import scala.concurrent.duration._
//#backpressure-by-readline
val completion: Future[Unit] =
val completion: Future[Done] =
Source(1 to 10)
.map(i => { println(s"map => $i"); i })
.runForeach { i => readLine(s"Element = $i; continue reading? [press enter]\n") }
@ -163,7 +164,7 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
"count elements on finite stream" in {
//#tweets-fold-count
val count: Flow[Tweet, Int, Unit] = Flow[Tweet].map(_ => 1)
val count: Flow[Tweet, Int, NotUsed] = Flow[Tweet].map(_ => 1)
val sumSink: Sink[Int, Future[Int]] = Sink.fold[Int, Int](0)(_ + _)

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook
import akka.NotUsed
import akka.stream.scaladsl.{ Flow, Sink, Source }
import akka.util.ByteString
@ -89,7 +90,7 @@ class RecipeByteStrings extends RecipeSpec {
val data = Source(List(ByteString(1, 2), ByteString(3), ByteString(4, 5, 6), ByteString(7, 8, 9)))
//#compacting-bytestrings
val compacted: Source[ByteString, Unit] = data.map(_.compact)
val compacted: Source[ByteString, NotUsed] = data.map(_.compact)
//#compacting-bytestrings
Await.result(compacted.grouped(10).runWith(Sink.head), 3.seconds).forall(_.isCompact) should be(true)

View file

@ -2,6 +2,7 @@ package docs.stream.cookbook
import java.security.MessageDigest
import akka.NotUsed
import akka.stream.scaladsl.{ Sink, Source }
import akka.util.ByteString
@ -41,7 +42,7 @@ class RecipeDigest extends RecipeSpec {
}
}
val digest: Source[ByteString, Unit] = data.transform(() => digestCalculator("SHA-256"))
val digest: Source[ByteString, NotUsed] = data.transform(() => digestCalculator("SHA-256"))
//#calculating-digest
Await.result(digest.runWith(Sink.head), 3.seconds) should be(

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook
import akka.NotUsed
import akka.stream.scaladsl.{ Sink, Source }
import scala.collection.immutable
@ -15,8 +16,8 @@ class RecipeFlattenSeq extends RecipeSpec {
val someDataSource = Source(List(List("1"), List("2"), List("3", "4", "5"), List("6", "7")))
//#flattening-seqs
val myData: Source[List[Message], Unit] = someDataSource
val flattened: Source[Message, Unit] = myData.mapConcat(identity)
val myData: Source[List[Message], NotUsed] = someDataSource
val flattened: Source[Message, NotUsed] = myData.mapConcat(identity)
//#flattening-seqs
Await.result(flattened.grouped(8).runWith(Sink.head), 3.seconds) should be(List("1", "2", "3", "4", "5", "6", "7"))

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook
import akka.NotUsed
import akka.actor.{ Props, ActorRef, Actor }
import akka.actor.Actor.Receive
import akka.stream.ClosedShape
@ -78,7 +79,7 @@ class RecipeGlobalRateLimit extends RecipeSpec {
"work" in {
//#global-limiter-flow
def limitGlobal[T](limiter: ActorRef, maxAllowedWait: FiniteDuration): Flow[T, T, Unit] = {
def limitGlobal[T](limiter: ActorRef, maxAllowedWait: FiniteDuration): Flow[T, T, NotUsed] = {
import akka.pattern.ask
import akka.util.Timeout
Flow[T].mapAsync(4)((element: T) => {

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook
import akka.NotUsed
import akka.stream.ClosedShape
import akka.stream.scaladsl._
import akka.stream.testkit._
@ -14,7 +15,7 @@ class RecipeKeepAlive extends RecipeSpec {
//#inject-keepalive
import scala.concurrent.duration._
val injectKeepAlive: Flow[ByteString, ByteString, Unit] =
val injectKeepAlive: Flow[ByteString, ByteString, NotUsed] =
Flow[ByteString].keepAlive(1.second, () => keepaliveMessage)
//#inject-keepalive

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook
import akka.NotUsed
import akka.stream.scaladsl._
import akka.stream.testkit._
import scala.concurrent.duration._
@ -19,12 +20,12 @@ class RecipeMissedTicks extends RecipeSpec {
val sink = Sink.fromSubscriber(sub)
//#missed-ticks
val missedTicks: Flow[Tick, Int, Unit] =
val missedTicks: Flow[Tick, Int, NotUsed] =
Flow[Tick].conflate(seed = (_) => 0)(
(missedTicks, tick) => missedTicks + 1)
//#missed-ticks
val latch = TestLatch(3)
val realMissedTicks: Flow[Tick, Int, Unit] =
val realMissedTicks: Flow[Tick, Int, NotUsed] =
Flow[Tick].conflate(seed = (_) => 0)(
(missedTicks, tick) => { latch.countDown(); missedTicks + 1 })

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook
import akka.NotUsed
import akka.stream.scaladsl.{ Sink, Source }
import scala.collection.immutable
@ -23,7 +24,7 @@ class RecipeMultiGroupBy extends RecipeSpec {
//#multi-groupby
val topicMapper: (Message) => immutable.Seq[Topic] = extractTopics
val messageAndTopic: Source[(Message, Topic), Unit] = elems.mapConcat { msg: Message =>
val messageAndTopic: Source[(Message, Topic), NotUsed] = elems.mapConcat { msg: Message =>
val topicsForMessage = topicMapper(msg)
// Create a (Msg, Topic) pair for each of the topics
// the message belongs to

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook
import akka.NotUsed
import akka.stream.{ Graph, FlowShape, Inlet, Outlet, Attributes, OverflowStrategy }
import akka.stream.scaladsl._
import scala.concurrent.{ Await, Future }
@ -17,7 +18,7 @@ class RecipeReduceByKey extends RecipeSpec {
def words = Source(List("hello", "world", "and", "hello", "universe", "akka") ++ List.fill(1000)("rocks!"))
//#word-count
val counts: Source[(String, Int), Unit] = words
val counts: Source[(String, Int), NotUsed] = words
// split the words into separate streams first
.groupBy(MaximumDistinctWords, identity)
// add counting logic to the streams
@ -45,7 +46,7 @@ class RecipeReduceByKey extends RecipeSpec {
def reduceByKey[In, K, Out](
maximumGroupSize: Int,
groupKey: (In) => K,
foldZero: (K) => Out)(fold: (Out, In) => Out): Flow[In, (K, Out), Unit] = {
foldZero: (K) => Out)(fold: (Out, In) => Out): Flow[In, (K, Out), NotUsed] = {
Flow[In]
.groupBy(maximumGroupSize, groupKey)

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook
import akka.NotUsed
import akka.stream.scaladsl.{ Flow, Sink, Source }
import akka.stream.testkit._
import scala.concurrent.duration._
@ -13,7 +14,7 @@ class RecipeSimpleDrop extends RecipeSpec {
"work" in {
//#simple-drop
val droppyStream: Flow[Message, Message, Unit] =
val droppyStream: Flow[Message, Message, NotUsed] =
Flow[Message].conflate(seed = identity)((lastMessage, newMessage) => newMessage)
//#simple-drop
val latch = TestLatch(2)

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook
import akka.NotUsed
import akka.stream.FlowShape
import akka.stream.scaladsl._
import akka.testkit.TestProbe
@ -18,7 +19,7 @@ class RecipeWorkerPool extends RecipeSpec {
val worker = Flow[String].map(_ + " done")
//#worker-pool
def balancer[In, Out](worker: Flow[In, Out, Any], workerCount: Int): Flow[In, Out, Unit] = {
def balancer[In, Out](worker: Flow[In, Out, Any], workerCount: Int): Flow[In, Out, NotUsed] = {
import GraphDSL.Implicits._
Flow.fromGraph(GraphDSL.create() { implicit b =>
@ -35,7 +36,7 @@ class RecipeWorkerPool extends RecipeSpec {
})
}
val processedJobs: Source[Result, Unit] = myJobs.via(balancer(worker, 3))
val processedJobs: Source[Result, NotUsed] = myJobs.via(balancer(worker, 3))
//#worker-pool
Await.result(processedJobs.grouped(10).runWith(Sink.head), 3.seconds).toSet should be(Set(

View file

@ -4,6 +4,7 @@
package akka.http.impl.engine.client
import akka.NotUsed
import akka.stream.impl.fusing.GraphInterpreter
import language.existentials
import scala.annotation.tailrec
@ -60,7 +61,7 @@ private[http] object OutgoingConnectionBlueprint {
val requestRendererFactory = new HttpRequestRendererFactory(userAgentHeader, requestHeaderSizeHint, log)
val requestRendering: Flow[HttpRequest, ByteString, Unit] = Flow[HttpRequest]
val requestRendering: Flow[HttpRequest, ByteString, NotUsed] = Flow[HttpRequest]
.map(RequestRenderingContext(_, hostHeader))
.via(Flow[RequestRenderingContext].flatMapConcat(requestRendererFactory.renderToSource).named("renderer"))

View file

@ -5,6 +5,7 @@
package akka.http.impl.engine.client
import java.net.InetSocketAddress
import akka.NotUsed
import akka.http.ConnectionPoolSettings
import scala.concurrent.{ Promise, Future }
@ -69,7 +70,7 @@ private object PoolFlow {
*/
def apply(connectionFlow: Flow[HttpRequest, HttpResponse, Future[Http.OutgoingConnection]],
remoteAddress: InetSocketAddress, settings: ConnectionPoolSettings, log: LoggingAdapter)(
implicit system: ActorSystem, fm: Materializer): Flow[RequestContext, ResponseContext, Unit] =
implicit system: ActorSystem, fm: Materializer): Flow[RequestContext, ResponseContext, NotUsed] =
Flow.fromGraph(GraphDSL.create[FlowShape[RequestContext, ResponseContext]]() { implicit b
import settings._
import GraphDSL.Implicits._

View file

@ -4,6 +4,7 @@
package akka.http.impl.engine.parsing
import akka.NotUsed
import akka.http.ParserSettings
import akka.stream.impl.fusing.GraphInterpreter
import scala.annotation.tailrec
@ -272,7 +273,7 @@ private[http] object BodyPartParser {
sealed trait Output
sealed trait PartStart extends Output
final case class BodyPartStart(headers: List[HttpHeader], createEntity: Source[Output, Unit] BodyPartEntity) extends PartStart
final case class BodyPartStart(headers: List[HttpHeader], createEntity: Source[Output, NotUsed] BodyPartEntity) extends PartStart
final case class EntityPart(data: ByteString) extends Output
final case class ParseError(info: ErrorInfo) extends PartStart

View file

@ -4,6 +4,7 @@
package akka.http.impl.engine.parsing
import akka.NotUsed
import akka.http.scaladsl.model._
import akka.stream.impl.fusing.GraphInterpreter
import akka.stream.scaladsl.{ Sink, Source }
@ -63,17 +64,17 @@ private[http] object ParserOutput {
//////////////////////////////////////
sealed abstract class EntityCreator[-A <: ParserOutput, +B >: HttpEntity.Strict <: HttpEntity] extends (Source[A, Unit] B)
sealed abstract class EntityCreator[-A <: ParserOutput, +B >: HttpEntity.Strict <: HttpEntity] extends (Source[A, NotUsed] B)
final case class StrictEntityCreator(entity: HttpEntity.Strict) extends EntityCreator[ParserOutput, HttpEntity.Strict] {
def apply(parts: Source[ParserOutput, Unit]) = {
def apply(parts: Source[ParserOutput, NotUsed]) = {
// We might need to drain stray empty tail streams which will be read by no one.
SubSource.kill(parts)
entity
}
}
final case class StreamedEntityCreator[-A <: ParserOutput, +B >: HttpEntity.Strict <: HttpEntity](creator: Source[A, Unit] B)
final case class StreamedEntityCreator[-A <: ParserOutput, +B >: HttpEntity.Strict <: HttpEntity](creator: Source[A, NotUsed] B)
extends EntityCreator[A, B] {
def apply(parts: Source[A, Unit]) = creator(parts)
def apply(parts: Source[A, NotUsed]) = creator(parts)
}
}

View file

@ -4,6 +4,7 @@
package akka.http.impl.engine.rendering
import akka.NotUsed
import akka.http.impl.engine.ws.{ FrameEvent, UpgradeToWebsocketResponseHeader }
import akka.http.scaladsl.model.ws.Message
import akka.stream.{ Outlet, Inlet, Attributes, FlowShape, Graph }
@ -53,7 +54,7 @@ private[http] class HttpResponseRendererFactory(serverHeader: Option[headers.Ser
// split out so we can stabilize by overriding in tests
protected def currentTimeMillis(): Long = System.currentTimeMillis()
def renderer: Flow[ResponseRenderingContext, ResponseRenderingOutput, Unit] = Flow.fromGraph(HttpResponseRenderer)
def renderer: Flow[ResponseRenderingContext, ResponseRenderingOutput, NotUsed] = Flow.fromGraph(HttpResponseRenderer)
object HttpResponseRenderer extends GraphStage[FlowShape[ResponseRenderingContext, ResponseRenderingOutput]] {
val in = Inlet[ResponseRenderingContext]("in")

View file

@ -10,6 +10,7 @@ import scala.concurrent.{ Promise, Future }
import scala.concurrent.duration.{ Deadline, FiniteDuration, Duration }
import scala.collection.immutable
import scala.util.control.NonFatal
import akka.NotUsed
import akka.actor.Cancellable
import akka.japi.Function
import akka.event.LoggingAdapter
@ -67,22 +68,22 @@ private[http] object HttpServerBluePrint {
theStack.withAttributes(HttpAttributes.remoteAddress(remoteAddress))
}
val tlsSupport: BidiFlow[ByteString, SslTlsOutbound, SslTlsInbound, SessionBytes, Unit] =
val tlsSupport: BidiFlow[ByteString, SslTlsOutbound, SslTlsInbound, SessionBytes, NotUsed] =
BidiFlow.fromFlows(Flow[ByteString].map(SendBytes), Flow[SslTlsInbound].collect { case x: SessionBytes x })
def websocketSupport(settings: ServerSettings, log: LoggingAdapter): BidiFlow[ResponseRenderingOutput, ByteString, SessionBytes, SessionBytes, Unit] =
def websocketSupport(settings: ServerSettings, log: LoggingAdapter): BidiFlow[ResponseRenderingOutput, ByteString, SessionBytes, SessionBytes, NotUsed] =
BidiFlow.fromGraph(new ProtocolSwitchStage(settings, log))
def parsingRendering(settings: ServerSettings, log: LoggingAdapter): BidiFlow[ResponseRenderingContext, ResponseRenderingOutput, SessionBytes, RequestOutput, Unit] =
def parsingRendering(settings: ServerSettings, log: LoggingAdapter): BidiFlow[ResponseRenderingContext, ResponseRenderingOutput, SessionBytes, RequestOutput, NotUsed] =
BidiFlow.fromFlows(rendering(settings, log), parsing(settings, log))
def controller(settings: ServerSettings, log: LoggingAdapter): BidiFlow[HttpResponse, ResponseRenderingContext, RequestOutput, RequestOutput, Unit] =
def controller(settings: ServerSettings, log: LoggingAdapter): BidiFlow[HttpResponse, ResponseRenderingContext, RequestOutput, RequestOutput, NotUsed] =
BidiFlow.fromGraph(new ControllerStage(settings, log)).reversed
def requestPreparation(settings: ServerSettings): BidiFlow[HttpResponse, HttpResponse, RequestOutput, HttpRequest, Unit] =
def requestPreparation(settings: ServerSettings): BidiFlow[HttpResponse, HttpResponse, RequestOutput, HttpRequest, NotUsed] =
BidiFlow.fromFlows(Flow[HttpResponse], new PrepareRequests(settings))
def requestTimeoutSupport(timeout: Duration): BidiFlow[HttpResponse, HttpResponse, HttpRequest, HttpRequest, Unit] =
def requestTimeoutSupport(timeout: Duration): BidiFlow[HttpResponse, HttpResponse, HttpRequest, HttpRequest, NotUsed] =
timeout match {
case x: FiniteDuration BidiFlow.fromGraph(new RequestTimeoutSupport(x)).reversed
case _ BidiFlow.identity
@ -138,7 +139,7 @@ private[http] object HttpServerBluePrint {
}
}
def parsing(settings: ServerSettings, log: LoggingAdapter): Flow[SessionBytes, RequestOutput, Unit] = {
def parsing(settings: ServerSettings, log: LoggingAdapter): Flow[SessionBytes, RequestOutput, NotUsed] = {
import settings._
// the initial header parser we initially use for every connection,
@ -168,7 +169,7 @@ private[http] object HttpServerBluePrint {
.map(establishAbsoluteUri)
}
def rendering(settings: ServerSettings, log: LoggingAdapter): Flow[ResponseRenderingContext, ResponseRenderingOutput, Unit] = {
def rendering(settings: ServerSettings, log: LoggingAdapter): Flow[ResponseRenderingContext, ResponseRenderingOutput, NotUsed] = {
import settings._
val responseRendererFactory = new HttpResponseRendererFactory(serverHeader, responseHeaderSizeHint, log)
@ -439,7 +440,7 @@ private[http] object HttpServerBluePrint {
def with100ContinueTrigger[T <: ParserOutput](createEntity: EntityCreator[T, RequestEntity]) =
StreamedEntityCreator {
createEntity.compose[Source[T, Unit]] {
createEntity.compose[Source[T, NotUsed]] {
_.via(Flow[T].transform(() new PushPullStage[T, T] {
private var oneHundredContinueSent = false
def onPush(elem: T, ctx: Context[T]) = ctx.push(elem)
@ -461,7 +462,7 @@ private[http] object HttpServerBluePrint {
* - produces exactly one response per request
* - has not more than `pipeliningLimit` responses outstanding
*/
def userHandlerGuard(pipeliningLimit: Int): BidiFlow[HttpResponse, HttpResponse, HttpRequest, HttpRequest, Unit] =
def userHandlerGuard(pipeliningLimit: Int): BidiFlow[HttpResponse, HttpResponse, HttpRequest, HttpRequest, NotUsed] =
One2OneBidiFlow[HttpRequest, HttpResponse](pipeliningLimit).reversed
private class ProtocolSwitchStage(settings: ServerSettings, log: LoggingAdapter)

View file

@ -4,6 +4,7 @@
package akka.http.impl.engine.ws
import akka.NotUsed
import akka.stream.scaladsl.Flow
import akka.stream.stage.{ SyncDirective, Context, StatefulStage }
import akka.util.ByteString
@ -19,7 +20,7 @@ import scala.util.control.NonFatal
*/
private[http] object FrameHandler {
def create(server: Boolean): Flow[FrameEventOrError, Output, Unit] =
def create(server: Boolean): Flow[FrameEventOrError, Output, NotUsed] =
Flow[FrameEventOrError].transform(() new HandlerStage(server))
private class HandlerStage(server: Boolean) extends StatefulStage[FrameEventOrError, Output] {

View file

@ -4,6 +4,7 @@
package akka.http.impl.engine.ws
import akka.NotUsed
import akka.event.LoggingAdapter
import akka.stream.scaladsl.Flow
import scala.concurrent.duration.FiniteDuration
@ -145,6 +146,6 @@ private[http] class FrameOutHandler(serverSide: Boolean, _closeTimeout: FiniteDu
private[http] object FrameOutHandler {
type Input = AnyRef
def create(serverSide: Boolean, closeTimeout: FiniteDuration, log: LoggingAdapter): Flow[Input, FrameStart, Unit] =
def create(serverSide: Boolean, closeTimeout: FiniteDuration, log: LoggingAdapter): Flow[Input, FrameStart, NotUsed] =
Flow[Input].transform(() new FrameOutHandler(serverSide, closeTimeout, log))
}

View file

@ -6,6 +6,7 @@ package akka.http.impl.engine.ws
import java.util.Random
import akka.NotUsed
import akka.stream.scaladsl.{ Keep, BidiFlow, Flow }
import akka.stream.stage.{ SyncDirective, Context, StatefulStage }
@ -15,10 +16,10 @@ import akka.stream.stage.{ SyncDirective, Context, StatefulStage }
* INTERNAL API
*/
private[http] object Masking {
def apply(serverSide: Boolean, maskRandom: () Random): BidiFlow[ /* net in */ FrameEvent, /* app out */ FrameEventOrError, /* app in */ FrameEvent, /* net out */ FrameEvent, Unit] =
def apply(serverSide: Boolean, maskRandom: () Random): BidiFlow[ /* net in */ FrameEvent, /* app out */ FrameEventOrError, /* app in */ FrameEvent, /* net out */ FrameEvent, NotUsed] =
BidiFlow.fromFlowsMat(unmaskIf(serverSide), maskIf(!serverSide, maskRandom))(Keep.none)
def maskIf(condition: Boolean, maskRandom: () Random): Flow[FrameEvent, FrameEvent, Unit] =
def maskIf(condition: Boolean, maskRandom: () Random): Flow[FrameEvent, FrameEvent, NotUsed] =
if (condition)
Flow[FrameEvent]
.transform(() new Masking(maskRandom())) // new random per materialization
@ -27,7 +28,7 @@ private[http] object Masking {
case FrameError(ex) throw ex
}
else Flow[FrameEvent]
def unmaskIf(condition: Boolean): Flow[FrameEvent, FrameEventOrError, Unit] =
def unmaskIf(condition: Boolean): Flow[FrameEvent, FrameEventOrError, NotUsed] =
if (condition) Flow[FrameEvent].transform(() new Unmasking())
else Flow[FrameEvent]

View file

@ -4,6 +4,7 @@
package akka.http.impl.engine.ws
import akka.NotUsed
import akka.util.ByteString
import akka.stream.scaladsl.{ Source, Flow }
@ -16,12 +17,12 @@ import akka.http.scaladsl.model.ws._
* INTERNAL API
*/
private[http] object MessageToFrameRenderer {
def create(serverSide: Boolean): Flow[Message, FrameStart, Unit] = {
def create(serverSide: Boolean): Flow[Message, FrameStart, NotUsed] = {
def strictFrames(opcode: Opcode, data: ByteString): Source[FrameStart, _] =
// FIXME: fragment?
Source.single(FrameEvent.fullFrame(opcode, None, data, fin = true))
def streamedFrames[M](opcode: Opcode, data: Source[ByteString, M]): Source[FrameStart, Unit] =
def streamedFrames[M](opcode: Opcode, data: Source[ByteString, M]): Source[FrameStart, NotUsed] =
Source.single(FrameEvent.empty(opcode, fin = false)) ++
data.map(FrameEvent.fullFrame(Opcode.Continuation, None, _, fin = false)) ++
Source.single(FrameEvent.emptyLastContinuationFrame)

View file

@ -5,6 +5,7 @@
package akka.http.impl.engine.ws
import java.util.Random
import akka.NotUsed
import akka.event.LoggingAdapter
import akka.util.ByteString
import scala.concurrent.duration._
@ -28,20 +29,20 @@ private[http] object Websocket {
def stack(serverSide: Boolean,
maskingRandomFactory: () Random,
closeTimeout: FiniteDuration = 3.seconds,
log: LoggingAdapter): BidiFlow[FrameEvent, Message, Message, FrameEvent, Unit] =
log: LoggingAdapter): BidiFlow[FrameEvent, Message, Message, FrameEvent, NotUsed] =
masking(serverSide, maskingRandomFactory) atop
frameHandling(serverSide, closeTimeout, log) atop
messageAPI(serverSide, closeTimeout)
/** The lowest layer that implements the binary protocol */
def framing: BidiFlow[ByteString, FrameEvent, FrameEvent, ByteString, Unit] =
def framing: BidiFlow[ByteString, FrameEvent, FrameEvent, ByteString, NotUsed] =
BidiFlow.fromFlows(
Flow[ByteString].via(FrameEventParser),
Flow[FrameEvent].transform(() new FrameEventRenderer))
.named("ws-framing")
/** The layer that handles masking using the rules defined in the specification */
def masking(serverSide: Boolean, maskingRandomFactory: () Random): BidiFlow[FrameEvent, FrameEventOrError, FrameEvent, FrameEvent, Unit] =
def masking(serverSide: Boolean, maskingRandomFactory: () Random): BidiFlow[FrameEvent, FrameEventOrError, FrameEvent, FrameEvent, NotUsed] =
Masking(serverSide, maskingRandomFactory)
.named("ws-masking")
@ -51,7 +52,7 @@ private[http] object Websocket {
*/
def frameHandling(serverSide: Boolean = true,
closeTimeout: FiniteDuration,
log: LoggingAdapter): BidiFlow[FrameEventOrError, FrameHandler.Output, FrameOutHandler.Input, FrameStart, Unit] =
log: LoggingAdapter): BidiFlow[FrameEventOrError, FrameHandler.Output, FrameOutHandler.Input, FrameStart, NotUsed] =
BidiFlow.fromFlows(
FrameHandler.create(server = serverSide),
FrameOutHandler.create(serverSide, closeTimeout, log))
@ -61,7 +62,7 @@ private[http] object Websocket {
* The layer that provides the high-level user facing API on top of frame handling.
*/
def messageAPI(serverSide: Boolean,
closeTimeout: FiniteDuration): BidiFlow[FrameHandler.Output, Message, Message, FrameOutHandler.Input, Unit] = {
closeTimeout: FiniteDuration): BidiFlow[FrameHandler.Output, Message, Message, FrameOutHandler.Input, NotUsed] = {
/* Completes this branch of the flow if no more messages are expected and converts close codes into errors */
class PrepareForUserHandler extends PushStage[MessagePart, MessagePart] {
var inMessage = false
@ -81,7 +82,7 @@ private[http] object Websocket {
}
/* Collects user-level API messages from MessageDataParts */
val collectMessage: Flow[MessageDataPart, Message, Unit] =
val collectMessage: Flow[MessageDataPart, Message, NotUsed] =
Flow[MessageDataPart]
.prefixAndTail(1)
.mapConcat {
@ -110,7 +111,7 @@ private[http] object Websocket {
}) :: Nil
}
def prepareMessages: Flow[MessagePart, Message, Unit] =
def prepareMessages: Flow[MessagePart, Message, NotUsed] =
Flow[MessagePart]
.transform(() new PrepareForUserHandler)
.splitWhen(_.isMessageEnd) // FIXME using splitAfter from #16885 would simplify protocol a lot
@ -121,7 +122,7 @@ private[http] object Websocket {
.concatSubstreams
.named("ws-prepare-messages")
def renderMessages: Flow[Message, FrameStart, Unit] =
def renderMessages: Flow[Message, FrameStart, NotUsed] =
MessageToFrameRenderer.create(serverSide)
.named("ws-render-messages")

View file

@ -4,6 +4,7 @@
package akka.http.impl.engine.ws
import akka.NotUsed
import akka.http.scaladsl.model.ws._
import scala.concurrent.{ Future, Promise }
@ -129,7 +130,7 @@ object WebsocketClientBlueprint {
}) mapMaterializedValue (_ result.future)
}
def simpleTls: BidiFlow[SslTlsInbound, ByteString, ByteString, SendBytes, Unit] =
def simpleTls: BidiFlow[SslTlsInbound, ByteString, ByteString, SendBytes, NotUsed] =
BidiFlow.fromFlowsMat(
Flow[SslTlsInbound].collect { case SessionBytes(_, bytes) bytes },
Flow[ByteString].map(SendBytes))(Keep.none)

View file

@ -12,7 +12,7 @@ import akka.stream.{ Graph, FlowShape, javadsl, scaladsl }
import scala.collection.immutable
import scala.reflect.ClassTag
import akka.japi
import akka.{ NotUsed, japi }
import akka.http.impl.model.{ JavaQuery, JavaUri }
import akka.http.javadsl.{ model jm }
import akka.http.scaladsl.{ model sm }
@ -129,11 +129,11 @@ private[http] object JavaMapping {
}
}
def scalaToJavaAdapterFlow[J, S](implicit mapping: JavaMapping[J, S]): scaladsl.Flow[S, J, Unit] =
def scalaToJavaAdapterFlow[J, S](implicit mapping: JavaMapping[J, S]): scaladsl.Flow[S, J, NotUsed] =
scaladsl.Flow[S].map(mapping.toJava)
def javaToScalaAdapterFlow[J, S](implicit mapping: JavaMapping[J, S]): scaladsl.Flow[J, S, Unit] =
def javaToScalaAdapterFlow[J, S](implicit mapping: JavaMapping[J, S]): scaladsl.Flow[J, S, NotUsed] =
scaladsl.Flow[J].map(mapping.toScala)
def adapterBidiFlow[JIn, SIn, SOut, JOut](implicit inMapping: JavaMapping[JIn, SIn], outMapping: JavaMapping[JOut, SOut]): scaladsl.BidiFlow[JIn, SIn, SOut, JOut, Unit] =
def adapterBidiFlow[JIn, SIn, SOut, JOut](implicit inMapping: JavaMapping[JIn, SIn], outMapping: JavaMapping[JOut, SOut]): scaladsl.BidiFlow[JIn, SIn, SOut, JOut, NotUsed] =
scaladsl.BidiFlow.fromFlowsMat(javaToScalaAdapterFlow(inMapping), scalaToJavaAdapterFlow(outMapping))(scaladsl.Keep.none)
implicit def pairMapping[J1, J2, S1, S2](implicit _1Mapping: JavaMapping[J1, S1], _2Mapping: JavaMapping[J2, S2]): JavaMapping[Pair[J1, J2], (S1, S2)] =

View file

@ -6,6 +6,7 @@ package akka.http.impl.util
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicReference }
import akka.NotUsed
import akka.http.scaladsl.model.RequestEntity
import akka.stream._
import akka.stream.impl.StreamLayout.Module
@ -51,7 +52,7 @@ private[http] object StreamUtils {
def failedPublisher[T](ex: Throwable): Publisher[T] =
impl.ErrorPublisher(ex, "failed").asInstanceOf[Publisher[T]]
def mapErrorTransformer(f: Throwable Throwable): Flow[ByteString, ByteString, Unit] = {
def mapErrorTransformer(f: Throwable Throwable): Flow[ByteString, ByteString, NotUsed] = {
val transformer = new PushStage[ByteString, ByteString] {
override def onPush(element: ByteString, ctx: Context[ByteString]): SyncDirective =
ctx.push(element)
@ -79,7 +80,7 @@ private[http] object StreamUtils {
source.transform(() transformer) -> promise.future
}
def sliceBytesTransformer(start: Long, length: Long): Flow[ByteString, ByteString, Unit] = {
def sliceBytesTransformer(start: Long, length: Long): Flow[ByteString, ByteString, NotUsed] = {
val transformer = new StatefulStage[ByteString, ByteString] {
def skipping = new State {
@ -289,7 +290,7 @@ private[http] object StreamUtils {
* Similar to Source.maybe but doesn't rely on materialization. Can only be used once.
*/
trait OneTimeValve {
def source[T]: Source[T, Unit]
def source[T]: Source[T, NotUsed]
def open(): Unit
}
object OneTimeValve {
@ -297,7 +298,7 @@ private[http] object StreamUtils {
val promise = Promise[Unit]()
val _source = Source.fromFuture(promise.future).drop(1) // we are only interested in the completion event
def source[T]: Source[T, Unit] = _source.asInstanceOf[Source[T, Unit]] // safe, because source won't generate any elements
def source[T]: Source[T, NotUsed] = _source.asInstanceOf[Source[T, NotUsed]] // safe, because source won't generate any elements
def open(): Unit = promise.success(())
}
}

View file

@ -4,6 +4,8 @@
package akka.http.impl
import akka.NotUsed
import language.implicitConversions
import language.higherKinds
import java.nio.charset.Charset
@ -41,7 +43,7 @@ package object util {
private[http] implicit def enhanceByteStringsMat[Mat](byteStrings: Source[ByteString, Mat]): EnhancedByteStringSource[Mat] =
new EnhancedByteStringSource(byteStrings)
private[http] def printEvent[T](marker: String): Flow[T, T, Unit] =
private[http] def printEvent[T](marker: String): Flow[T, T, NotUsed] =
Flow[T].transform(() new PushPullStage[T, T] {
override def onPush(element: T, ctx: Context[T]): SyncDirective = {
println(s"$marker: $element")

View file

@ -8,7 +8,7 @@ import java.net.InetSocketAddress
import java.util.Optional
import akka.http.impl.util.JavaMapping
import akka.http.javadsl.model.ws._
import akka.stream
import akka.{ NotUsed, stream }
import akka.stream.io.{ SslTlsInbound, SslTlsOutbound }
import scala.language.implicitConversions
@ -46,7 +46,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
* Constructs a server layer stage using the configured default [[ServerSettings]]. The returned [[BidiFlow]] isn't
* reusable and can only be materialized once.
*/
def serverLayer(materializer: Materializer): BidiFlow[HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest, Unit] =
def serverLayer(materializer: Materializer): BidiFlow[HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest, NotUsed] =
adaptServerLayer(delegate.serverLayer()(materializer))
/**
@ -54,7 +54,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
* can only be materialized once.
*/
def serverLayer(settings: ServerSettings,
materializer: Materializer): BidiFlow[HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest, Unit] =
materializer: Materializer): BidiFlow[HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest, NotUsed] =
adaptServerLayer(delegate.serverLayer(settings)(materializer))
/**
@ -64,7 +64,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
*/
def serverLayer(settings: ServerSettings,
remoteAddress: Optional[InetSocketAddress],
materializer: Materializer): BidiFlow[HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest, Unit] =
materializer: Materializer): BidiFlow[HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest, NotUsed] =
adaptServerLayer(delegate.serverLayer(settings, remoteAddress.asScala)(materializer))
/**
@ -75,7 +75,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
def serverLayer(settings: ServerSettings,
remoteAddress: Optional[InetSocketAddress],
log: LoggingAdapter,
materializer: Materializer): BidiFlow[HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest, Unit] =
materializer: Materializer): BidiFlow[HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest, NotUsed] =
adaptServerLayer(delegate.serverLayer(settings, remoteAddress.asScala, log)(materializer))
/**
@ -205,14 +205,14 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
/**
* Constructs a client layer stage using the configured default [[ClientConnectionSettings]].
*/
def clientLayer(hostHeader: headers.Host): BidiFlow[HttpRequest, SslTlsOutbound, SslTlsInbound, HttpResponse, Unit] =
def clientLayer(hostHeader: headers.Host): BidiFlow[HttpRequest, SslTlsOutbound, SslTlsInbound, HttpResponse, NotUsed] =
adaptClientLayer(delegate.clientLayer(JavaMapping.toScala(hostHeader)))
/**
* Constructs a client layer stage using the given [[ClientConnectionSettings]].
*/
def clientLayer(hostHeader: headers.Host,
settings: ClientConnectionSettings): BidiFlow[HttpRequest, SslTlsOutbound, SslTlsInbound, HttpResponse, Unit] =
settings: ClientConnectionSettings): BidiFlow[HttpRequest, SslTlsOutbound, SslTlsInbound, HttpResponse, NotUsed] =
adaptClientLayer(delegate.clientLayer(JavaMapping.toScala(hostHeader), settings))
/**
@ -220,7 +220,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
*/
def clientLayer(hostHeader: headers.Host,
settings: ClientConnectionSettings,
log: LoggingAdapter): BidiFlow[HttpRequest, SslTlsOutbound, SslTlsInbound, HttpResponse, Unit] =
log: LoggingAdapter): BidiFlow[HttpRequest, SslTlsOutbound, SslTlsInbound, HttpResponse, NotUsed] =
adaptClientLayer(delegate.clientLayer(JavaMapping.toScala(hostHeader), settings, log))
/**
@ -454,7 +454,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
* In order to allow for easy response-to-request association the flow takes in a custom, opaque context
* object of type `T` from the application which is emitted together with the corresponding response.
*/
def superPool[T](materializer: Materializer): Flow[Pair[HttpRequest, T], Pair[Try[HttpResponse], T], Unit] =
def superPool[T](materializer: Materializer): Flow[Pair[HttpRequest, T], Pair[Try[HttpResponse], T], NotUsed] =
adaptTupleFlow(delegate.superPool[T]()(materializer))
/**
@ -474,7 +474,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
*/
def superPool[T](settings: ConnectionPoolSettings,
httpsContext: Optional[HttpsContext],
log: LoggingAdapter, materializer: Materializer): Flow[Pair[HttpRequest, T], Pair[Try[HttpResponse], T], Unit] =
log: LoggingAdapter, materializer: Materializer): Flow[Pair[HttpRequest, T], Pair[Try[HttpResponse], T], NotUsed] =
adaptTupleFlow(delegate.superPool[T](settings, httpsContext, log)(materializer))
/**
@ -616,12 +616,12 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
JavaMapping.toJava(scalaFlow)(JavaMapping.flowMapping[Pair[HttpRequest, T], (scaladsl.model.HttpRequest, T), Pair[Try[HttpResponse], T], (Try[scaladsl.model.HttpResponse], T), Mat])
}
private def adaptServerLayer(serverLayer: scaladsl.Http.ServerLayer): BidiFlow[HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest, Unit] =
private def adaptServerLayer(serverLayer: scaladsl.Http.ServerLayer): BidiFlow[HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest, NotUsed] =
new BidiFlow(
JavaMapping.adapterBidiFlow[HttpResponse, sm.HttpResponse, sm.HttpRequest, HttpRequest]
.atop(serverLayer))
private def adaptClientLayer(clientLayer: scaladsl.Http.ClientLayer): BidiFlow[HttpRequest, SslTlsOutbound, SslTlsInbound, HttpResponse, Unit] =
private def adaptClientLayer(clientLayer: scaladsl.Http.ClientLayer): BidiFlow[HttpRequest, SslTlsOutbound, SslTlsInbound, HttpResponse, NotUsed] =
new BidiFlow(
JavaMapping.adapterBidiFlow[HttpRequest, sm.HttpRequest, sm.HttpResponse, HttpResponse]
.atop(clientLayer))

View file

@ -5,6 +5,7 @@
package akka.http.javadsl
import java.net.InetSocketAddress
import akka.NotUsed
import akka.japi.function.Function;
import scala.concurrent.Future
import akka.stream.Materializer
@ -31,7 +32,7 @@ class IncomingConnection private[http] (delegate: akka.http.scaladsl.Http.Incomi
*
* Use `Flow.join` or one of the handleXXX methods to consume handle requests on this connection.
*/
def flow: Flow[HttpResponse, HttpRequest, Unit] = Flow.fromGraph(delegate.flow).asInstanceOf[Flow[HttpResponse, HttpRequest, Unit]]
def flow: Flow[HttpResponse, HttpRequest, NotUsed] = Flow.fromGraph(delegate.flow).asInstanceOf[Flow[HttpResponse, HttpRequest, NotUsed]]
/**
* Handles the connection with the given flow, which is materialized exactly once

View file

@ -21,6 +21,7 @@ import akka.http.scaladsl.model._
import akka.http.scaladsl.model.headers.Host
import akka.http.scaladsl.model.ws.{ WebsocketUpgradeResponse, WebsocketRequest, Message }
import akka.http.scaladsl.util.FastFuture
import akka.NotUsed
import akka.stream.Materializer
import akka.stream.io._
import akka.stream.scaladsl._
@ -400,7 +401,7 @@ class HttpExt(private val config: Config)(implicit val system: ActorSystem) exte
*/
def superPool[T](settings: ConnectionPoolSettings = ConnectionPoolSettings(system),
httpsContext: Option[HttpsContext] = None,
log: LoggingAdapter = system.log)(implicit fm: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), Unit] =
log: LoggingAdapter = system.log)(implicit fm: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), NotUsed] =
clientFlow[T](settings) { request request -> cachedGateway(request, settings, httpsContext, log) }
/**
@ -560,7 +561,7 @@ class HttpExt(private val config: Config)(implicit val system: ActorSystem) exte
.mapMaterializedValue(_ HostConnectionPool(hcps)(gatewayFuture))
private def clientFlow[T](settings: ConnectionPoolSettings)(f: HttpRequest (HttpRequest, Future[PoolGateway]))(
implicit system: ActorSystem, fm: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), Unit] = {
implicit system: ActorSystem, fm: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), NotUsed] = {
// a connection pool can never have more than pipeliningLimit * maxConnections requests in flight at any point
val parallelism = settings.pipeliningLimit * settings.maxConnections
Flow[(HttpRequest, T)].mapAsyncUnordered(parallelism) {
@ -601,7 +602,7 @@ object Http extends ExtensionId[HttpExt] with ExtensionIdProvider {
* +------+
* }}}
*/
type ServerLayer = BidiFlow[HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest, Unit]
type ServerLayer = BidiFlow[HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest, NotUsed]
//#
//#client-layer
@ -617,7 +618,7 @@ object Http extends ExtensionId[HttpExt] with ExtensionIdProvider {
* +------+
* }}}
*/
type ClientLayer = BidiFlow[HttpRequest, SslTlsOutbound, SslTlsInbound, HttpResponse, Unit]
type ClientLayer = BidiFlow[HttpRequest, SslTlsOutbound, SslTlsInbound, HttpResponse, NotUsed]
//#
/**
@ -657,7 +658,7 @@ object Http extends ExtensionId[HttpExt] with ExtensionIdProvider {
final case class IncomingConnection(
localAddress: InetSocketAddress,
remoteAddress: InetSocketAddress,
flow: Flow[HttpResponse, HttpRequest, Unit]) {
flow: Flow[HttpResponse, HttpRequest, NotUsed]) {
/**
* Handles the connection with the given flow, which is materialized exactly once

View file

@ -17,7 +17,7 @@ import akka.util.ByteString
import akka.stream.scaladsl._
import akka.stream.stage._
import akka.stream._
import akka.{ japi, stream }
import akka.{ NotUsed, japi, stream }
import akka.http.scaladsl.model.ContentType.{ NonBinary, Binary }
import akka.http.scaladsl.util.FastFuture
import akka.http.javadsl.{ model jm }
@ -208,7 +208,7 @@ object HttpEntity {
override def isKnownEmpty: Boolean = data.isEmpty
override def dataBytes: Source[ByteString, Unit] = Source(data :: Nil)
override def dataBytes: Source[ByteString, NotUsed] = Source(data :: Nil)
override def toStrict(timeout: FiniteDuration)(implicit fm: Materializer) =
FastFuture.successful(this)

View file

@ -5,10 +5,10 @@
package akka.http.scaladsl.model.ws
import java.lang.Iterable
import akka.http.impl.util.JavaMapping
import scala.collection.immutable
import akka.NotUsed
import akka.stream._
import akka.http.impl.util.JavaMapping
import akka.http.javadsl.{ model jm }
import akka.http.scaladsl.model.HttpResponse
@ -85,6 +85,6 @@ trait UpgradeToWebsocket extends jm.ws.UpgradeToWebsocket {
subprotocol: String): HttpResponse =
handleMessages(createScalaFlow(inSink, outSource), subprotocol = Some(subprotocol))
private[this] def createScalaFlow(inSink: Graph[SinkShape[jm.ws.Message], _ <: Any], outSource: Graph[SourceShape[jm.ws.Message], _ <: Any]): Graph[FlowShape[Message, Message], Unit] =
JavaMapping.toScala(scaladsl.Flow.fromSinkAndSourceMat(inSink, outSource)(scaladsl.Keep.none): Graph[FlowShape[jm.ws.Message, jm.ws.Message], Unit])
private[this] def createScalaFlow(inSink: Graph[SinkShape[jm.ws.Message], _ <: Any], outSource: Graph[SourceShape[jm.ws.Message], _ <: Any]): Graph[FlowShape[Message, Message], NotUsed] =
JavaMapping.toScala(scaladsl.Flow.fromSinkAndSourceMat(inSink, outSource)(scaladsl.Keep.none): Graph[FlowShape[jm.ws.Message, jm.ws.Message], NotUsed])
}

View file

@ -4,6 +4,7 @@
package akka.http.javadsl;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.dispatch.Futures;
import akka.http.javadsl.model.ws.Message;
@ -19,7 +20,6 @@ import akka.stream.javadsl.Source;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
import java.util.Arrays;
import java.util.List;
@ -49,7 +49,7 @@ public class WSEchoTestClientApp {
system.dispatcher(),
ignoredMessage);
Source<Message, BoxedUnit> echoSource =
Source<Message, NotUsed> echoSource =
Source.from(Arrays.<Message>asList(
TextMessage.create("abc"),
TextMessage.create("def"),
@ -60,10 +60,10 @@ public class WSEchoTestClientApp {
Flow.of(Message.class)
.map(messageStringifier)
.grouped(1000)
.toMat(Sink.<List<String>>head(), Keep.<BoxedUnit, Future<List<String>>>right());
.toMat(Sink.<List<String>>head(), Keep.<NotUsed, Future<List<String>>>right());
Flow<Message, Message, Future<List<String>>> echoClient =
Flow.fromSinkAndSourceMat(echoSink, echoSource, Keep.<Future<List<String>>, BoxedUnit>left());
Flow.fromSinkAndSourceMat(echoSink, echoSource, Keep.<Future<List<String>>, NotUsed>left());
Future<List<String>> result =
Http.get(system).singleWebsocketRequest(

View file

@ -4,6 +4,7 @@
package akka.http.javadsl.model;
import akka.NotUsed;
import akka.actor.ActorSystem;
import akka.http.javadsl.Http;
import akka.http.javadsl.ServerBinding;
@ -19,7 +20,6 @@ import akka.stream.javadsl.Source;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import scala.runtime.BoxedUnit;
import java.io.BufferedReader;
import java.io.InputStreamReader;
@ -55,11 +55,11 @@ public class JavaTestServer {
}
}
public static Flow<Message, Message, BoxedUnit> echoMessages() {
public static Flow<Message, Message, NotUsed> echoMessages() {
return Flow.create(); // the identity operation
}
public static Flow<Message, Message, BoxedUnit> greeter() {
public static Flow<Message, Message, NotUsed> greeter() {
return
Flow.<Message>create()
.collect(new JavaPartialFunction<Message, Message>() {

View file

@ -4,6 +4,7 @@
package akka.http.impl.engine.client
import akka.NotUsed
import org.scalatest.concurrent.PatienceConfiguration.Timeout
import org.scalatest.concurrent.ScalaFutures
import akka.stream.ActorMaterializer
@ -90,7 +91,7 @@ class TlsEndpointVerificationSpec extends AkkaSpec("""
def pipeline(clientContext: HttpsContext, hostname: String): HttpRequest Future[HttpResponse] = req
Source.single(req).via(pipelineFlow(clientContext, hostname)).runWith(Sink.head)
def pipelineFlow(clientContext: HttpsContext, hostname: String): Flow[HttpRequest, HttpResponse, Unit] = {
def pipelineFlow(clientContext: HttpsContext, hostname: String): Flow[HttpRequest, HttpResponse, NotUsed] = {
val handler: HttpRequest HttpResponse = { req
// verify Tls-Session-Info header information
val name = req.header[`Tls-Session-Info`].flatMap(_.localPrincipal).map(_.getName)

View file

@ -4,6 +4,8 @@
package akka.http.impl.engine.parsing
import akka.NotUsed
import scala.concurrent.Future
import scala.concurrent.duration._
@ -521,5 +523,5 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
def prep(response: String) = response.stripMarginWithNewline("\r\n")
}
def source[T](elems: T*): Source[T, Unit] = Source(elems.toList)
def source[T](elems: T*): Source[T, NotUsed] = Source(elems.toList)
}

View file

@ -4,6 +4,7 @@
package akka.http.impl.engine.parsing
import akka.NotUsed
import akka.http.ParserSettings
import akka.http.scaladsl.util.FastFuture
import akka.stream.io.{ SslTlsPlacebo, SessionBytes }
@ -289,7 +290,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
}.map(strictEqualify)
}
def rawParse(requestMethod: HttpMethod, input: String*): Source[Either[ResponseOutput, HttpResponse], Unit] =
def rawParse(requestMethod: HttpMethod, input: String*): Source[Either[ResponseOutput, HttpResponse], NotUsed] =
Source(input.toList)
.map(bytes SessionBytes(SslTlsPlacebo.dummySession, ByteString(bytes)))
.transform(() newParserStage(requestMethod)).named("parser")
@ -328,6 +329,6 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
def prep(response: String) = response.stripMarginWithNewline("\r\n")
def source[T](elems: T*): Source[T, Unit] = Source(elems.toList)
def source[T](elems: T*): Source[T, NotUsed] = Source(elems.toList)
}
}

View file

@ -4,6 +4,7 @@
package akka.http.impl.engine.ws
import akka.NotUsed
import akka.actor.ActorSystem
import akka.stream.scaladsl.Sink
import akka.stream.testkit.TestSubscriber
@ -13,7 +14,7 @@ import scala.annotation.tailrec
import scala.concurrent.duration.FiniteDuration
trait ByteStringSinkProbe {
def sink: Sink[ByteString, Unit]
def sink: Sink[ByteString, NotUsed]
def expectBytes(length: Int): ByteString
def expectBytes(expected: ByteString): Unit
@ -35,7 +36,7 @@ object ByteStringSinkProbe {
def apply()(implicit system: ActorSystem): ByteStringSinkProbe =
new ByteStringSinkProbe {
val probe = TestSubscriber.probe[ByteString]()
val sink: Sink[ByteString, Unit] = Sink.fromSubscriber(probe)
val sink: Sink[ByteString, NotUsed] = Sink.fromSubscriber(probe)
def expectNoBytes(): Unit = {
probe.ensureSubscription()

View file

@ -4,6 +4,8 @@
package akka.http.impl.engine.ws
import akka.NotUsed
import scala.concurrent.duration._
import akka.actor.ActorSystem
@ -24,10 +26,10 @@ object EchoTestClientApp extends App {
import system.dispatcher
implicit val materializer = ActorMaterializer()
def delayedCompletion(delay: FiniteDuration): Source[Nothing, Unit] =
def delayedCompletion(delay: FiniteDuration): Source[Nothing, NotUsed] =
Source.single(1)
.mapAsync(1)(_ akka.pattern.after(delay, system.scheduler)(Future(1)))
.drop(1).asInstanceOf[Source[Nothing, Unit]]
.drop(1).asInstanceOf[Source[Nothing, NotUsed]]
def messages: List[Message] =
List(
@ -36,7 +38,7 @@ object EchoTestClientApp extends App {
TextMessage("Test 2"),
BinaryMessage(ByteString("def")))
def source: Source[Message, Unit] =
def source: Source[Message, NotUsed] =
Source(messages) ++ delayedCompletion(1.second) // otherwise, we may start closing too soon
def sink: Sink[Message, Future[Seq[String]]] =

View file

@ -4,6 +4,8 @@
package akka.http.impl.engine.ws
import akka.NotUsed
import scala.concurrent.duration._
import scala.util.Random
import org.scalatest.{ Matchers, FreeSpec }
@ -854,7 +856,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
val messageIn = TestSubscriber.probe[Message]
val messageOut = TestPublisher.probe[Message]()
val messageHandler: Flow[Message, Message, Unit] =
val messageHandler: Flow[Message, Message, NotUsed] =
Flow.fromSinkAndSource(
Flow[Message].buffer(1, OverflowStrategy.backpressure).to(Sink.fromSubscriber(messageIn)), // alternatively need to request(1) before expectComplete
Source.fromPublisher(messageOut))
@ -971,7 +973,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
}
val trace = false // set to `true` for debugging purposes
def printEvent[T](marker: String): Flow[T, T, Unit] =
def printEvent[T](marker: String): Flow[T, T, NotUsed] =
if (trace) akka.http.impl.util.printEvent(marker)
else Flow[T]
}

View file

@ -4,6 +4,8 @@
package akka.http.impl.engine.ws
import akka.NotUsed
import scala.concurrent.Await
import scala.concurrent.duration._
@ -47,6 +49,6 @@ object WSServerAutobahnTest extends App {
system.shutdown()
}
def echoWebsocketService: Flow[Message, Message, Unit] =
def echoWebsocketService: Flow[Message, Message, NotUsed] =
Flow[Message] // just let message flow directly to the output
}

View file

@ -6,6 +6,7 @@ package akka.http.impl.engine.ws
import java.util.Random
import akka.NotUsed
import akka.http.scaladsl.model.ws.{ InvalidUpgradeResponse, WebsocketUpgradeResponse }
import akka.stream.ClosedShape
@ -119,7 +120,7 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
}
"don't send out frames before handshake was finished successfully" in new TestSetup {
def clientImplementation: Flow[Message, Message, Unit] =
def clientImplementation: Flow[Message, Message, NotUsed] =
Flow.fromSinkAndSourceMat(Sink.ignore, Source.single(TextMessage("fast message")))(Keep.none)
expectWireData(UpgradeRequestBytes)
@ -291,7 +292,7 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
abstract class TestSetup extends WSTestSetupBase {
protected def noMsgTimeout: FiniteDuration = 100.millis
protected def clientImplementation: Flow[Message, Message, Unit]
protected def clientImplementation: Flow[Message, Message, NotUsed]
protected def requestedSubProtocol: Option[String] = None
val random = new Random(0)
@ -362,14 +363,14 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
}
trait ClientEchoes extends TestSetup {
override def clientImplementation: Flow[Message, Message, Unit] = echoServer
def echoServer: Flow[Message, Message, Unit] = Flow[Message]
override def clientImplementation: Flow[Message, Message, NotUsed] = echoServer
def echoServer: Flow[Message, Message, NotUsed] = Flow[Message]
}
trait ClientProbes extends TestSetup {
lazy val messagesOut = TestPublisher.probe[Message]()
lazy val messagesIn = TestSubscriber.probe[Message]()
override def clientImplementation: Flow[Message, Message, Unit] =
override def clientImplementation: Flow[Message, Message, NotUsed] =
Flow.fromSinkAndSourceMat(Sink.fromSubscriber(messagesIn), Source.fromPublisher(messagesOut))(Keep.none)
}
}

View file

@ -4,6 +4,8 @@
package akka.http.scaladsl
import akka.NotUsed
import scala.concurrent.duration._
import scala.concurrent.Await
import akka.actor.ActorSystem
@ -64,10 +66,10 @@ object TestServer extends App {
| </body>
|</html>""".stripMargin))
def echoWebsocketService: Flow[Message, Message, Unit] =
def echoWebsocketService: Flow[Message, Message, NotUsed] =
Flow[Message] // just let message flow directly to the output
def greeterWebsocketService: Flow[Message, Message, Unit] =
def greeterWebsocketService: Flow[Message, Message, NotUsed] =
Flow[Message]
.collect {
case TextMessage.Strict(name) TextMessage(s"Hello '$name'")

View file

@ -5,6 +5,7 @@
package akka.http.scaladsl.model
import java.util.concurrent.TimeoutException
import akka.NotUsed
import com.typesafe.config.{ ConfigFactory, Config }
import scala.concurrent.{ Promise, Await }
import scala.concurrent.duration._
@ -176,7 +177,7 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
strict.toString + " == " + expectedRendering)
}
def duplicateBytesTransformer(): Flow[ByteString, ByteString, Unit] =
def duplicateBytesTransformer(): Flow[ByteString, ByteString, NotUsed] =
Flow[ByteString].transform(() StreamUtils.byteStringTransformer(doubleChars, () trailer))
def trailer: ByteString = ByteString("--dup")

Some files were not shown because too many files have changed in this diff Show more