Rename FlowMaterializer to Materializer

This commit is contained in:
Endre Sándor Varga 2015-06-23 18:28:53 +02:00
parent dc7269e620
commit 7879a5521b
245 changed files with 860 additions and 860 deletions

View file

@ -196,7 +196,7 @@ well-known sinks, such as ``runForeach(el -> )`` (being an alias to ``runWith(Si
Materialization is currently performed synchronously on the materializing thread.
The actual stream processing is handled by actors started up during the streams materialization,
which will be running on the thread pools they have been configured to run on - which defaults to the dispatcher set in
:class:`MaterializationSettings` while constructing the :class:`ActorFlowMaterializer`.
:class:`MaterializationSettings` while constructing the :class:`ActorMaterializer`.
.. note::
Reusing *instances* of linear computation stages (Source, Sink, Flow) inside FlowGraphs is legal,

View file

@ -299,7 +299,7 @@ is ``completed`` before ``g``, but still emitted afterwards.
The numbers in parenthesis illustrates how many calls that are in progress at
the same time. Here the downstream demand and thereby the number of concurrent
calls are limited by the buffer size (4) of the :class:`ActorFlowMaterializerSettings`.
calls are limited by the buffer size (4) of the :class:`ActorMaterializerSettings`.
Here is how we can use the same service with ``mapAsyncUnordered``:
@ -355,7 +355,7 @@ Note that ``after`` lines are not in the same order as the ``before`` lines. For
The numbers in parenthesis illustrates how many calls that are in progress at
the same time. Here the downstream demand and thereby the number of concurrent
calls are limited by the buffer size (4) of the :class:`ActorFlowMaterializerSettings`.
calls are limited by the buffer size (4) of the :class:`ActorMaterializerSettings`.
.. _reactive-streams-integration-java:

View file

@ -19,12 +19,12 @@ Here's the data model we'll be working with throughout the quickstart examples:
Transforming and consuming simple streams
-----------------------------------------
In order to prepare our environment by creating an :class:`ActorSystem` and :class:`ActorFlowMaterializer`,
In order to prepare our environment by creating an :class:`ActorSystem` and :class:`ActorMaterializer`,
which will be responsible for materializing and running the streams we are about to create:
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/TwitterStreamQuickstartDocTest.java#materializer-setup
The :class:`ActorFlowMaterializer` can optionally take :class:`ActorFlowMaterializerSettings` which can be used to define
The :class:`ActorMaterializer` can optionally take :class:`ActorMaterializerSettings` which can be used to define
materialization properties, such as default buffer sizes (see also :ref:`stream-buffers-java`), the dispatcher to
be used by the pipeline etc. These can be overridden ``withAttributes`` on :class:`Flow`, :class:`Source`, :class:`Sink` and :class:`Graph`.
@ -60,7 +60,7 @@ or by using the shorthand version (which are defined only for the most popular s
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/TwitterStreamQuickstartDocTest.java#authors-foreach-println
Materializing and running a stream always requires a :class:`FlowMaterializer` to be passed in explicitly,
Materializing and running a stream always requires a :class:`Materializer` to be passed in explicitly,
like this: ``.run(mat)``.
Flattening sequences in streams
@ -146,7 +146,7 @@ First, we prepare the :class:`FoldSink` which will be used to sum all ``Integer`
Next we connect the ``tweets`` stream though a ``map`` step which converts each tweet into the number ``1``,
finally we connect the flow ``to`` the previously prepared Sink. Notice that this step does *not* yet materialize the
processing pipeline, it merely prepares the description of the Flow, which is now connected to a Sink, and therefore can
be ``run()``, as indicated by its type: :class:`RunnableFlow`. Next we call ``run()`` which uses the implicit :class:`ActorFlowMaterializer`
be ``run()``, as indicated by its type: :class:`RunnableFlow`. Next we call ``run()`` which uses the implicit :class:`ActorMaterializer`
to materialize and run the flow. The value returned by calling ``run()`` on a ``RunnableFlow`` or ``FlowGraph`` is ``MaterializedMap``,
which can be used to retrieve materialized values from the running stream.

View file

@ -62,7 +62,7 @@ to a level that throughput requirements of the application require. Default buff
akka.stream.materializer.max-input-buffer-size = 16
Alternatively they can be set by passing a :class:`ActorFlowMaterializerSettings` to the materializer:
Alternatively they can be set by passing a :class:`ActorMaterializerSettings` to the materializer:
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/StreamBuffersRateDocTest.java#materializer-buffer

View file

@ -13,13 +13,13 @@ class HttpClientExampleSpec extends WordSpec with Matchers {
pending // compile-time only test
//#outgoing-connection-example
import scala.concurrent.Future
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl._
import akka.http.scaladsl.model._
import akka.http.scaladsl.Http
implicit val system = ActorSystem()
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
val connectionFlow: Flow[HttpRequest, HttpResponse, Future[Http.OutgoingConnection]] =
Http().outgoingConnection("akka.io")
@ -35,13 +35,13 @@ class HttpClientExampleSpec extends WordSpec with Matchers {
//#host-level-example
import scala.concurrent.Future
import scala.util.Try
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl._
import akka.http.scaladsl.model._
import akka.http.scaladsl.Http
implicit val system = ActorSystem()
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
// construct a pool client flow with context type `Int`
val poolClientFlow = Http().cachedHostConnectionPool[Int]("akka.io")
@ -56,12 +56,12 @@ class HttpClientExampleSpec extends WordSpec with Matchers {
pending // compile-time only test
//#single-request-example
import scala.concurrent.Future
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.model._
import akka.http.scaladsl.Http
implicit val system = ActorSystem()
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
val responseFuture: Future[HttpResponse] =
Http().singleRequest(HttpRequest(uri = "http://akka.io"))

View file

@ -11,12 +11,12 @@ import akka.actor.ActorSystem
class HttpServerExampleSpec extends WordSpec with Matchers {
"binding-example" in {
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl._
import akka.http.scaladsl.Http
implicit val system = ActorSystem()
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
val serverSource: Source[Http.IncomingConnection, Future[Http.ServerBinding]] =
Http().bind(interface = "localhost", port = 8080)
@ -28,14 +28,14 @@ class HttpServerExampleSpec extends WordSpec with Matchers {
}
"full-server-example" in {
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.Sink
import akka.http.scaladsl.Http
import akka.http.scaladsl.model.HttpMethods._
import akka.http.scaladsl.model._
implicit val system = ActorSystem()
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
val serverSource = Http().bind(interface = "localhost", port = 8080)
@ -65,13 +65,13 @@ class HttpServerExampleSpec extends WordSpec with Matchers {
}
"low-level-server-example" in {
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.Http
import akka.http.scaladsl.model.HttpMethods._
import akka.http.scaladsl.model._
implicit val system = ActorSystem()
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
val requestHandler: HttpRequest => HttpResponse = {
case HttpRequest(GET, Uri.Path("/"), _, _, _) =>
@ -94,13 +94,13 @@ class HttpServerExampleSpec extends WordSpec with Matchers {
// format: OFF
"high-level-server-example" in {
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.Http
import akka.http.scaladsl.server.Directives._
import akka.http.scaladsl.marshallers.xml.ScalaXmlSupport._
implicit val system = ActorSystem()
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
val route =
get {
@ -123,14 +123,14 @@ class HttpServerExampleSpec extends WordSpec with Matchers {
}
"minimal-routing-example" in {
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.Http
import akka.http.scaladsl.server.Directives._
import akka.http.scaladsl.marshallers.xml.ScalaXmlSupport._
object Main extends App {
implicit val system = ActorSystem("my-system")
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
val route =
path("hello") {

View file

@ -10,7 +10,7 @@ object MyExplicitExceptionHandler {
//#explicit-handler-example
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.Http
import akka.http.scaladsl.model._
import akka.http.scaladsl.server._
@ -27,7 +27,7 @@ object MyExplicitExceptionHandler {
object MyApp extends App {
implicit val system = ActorSystem()
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
val route: Route =
handleExceptions(myExceptionHandler) {
@ -44,7 +44,7 @@ object MyImplicitExceptionHandler {
//#implicit-handler-example
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.Http
import akka.http.scaladsl.model._
import akka.http.scaladsl.server._
@ -62,7 +62,7 @@ object MyImplicitExceptionHandler {
object MyApp extends App {
implicit val system = ActorSystem()
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
val route: Route =
// ... some route structure

View file

@ -10,7 +10,7 @@ object MyRejectionHandler {
//#custom-handler-example
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.Http
import akka.http.scaladsl.model._
import akka.http.scaladsl.server._
@ -34,7 +34,7 @@ object MyRejectionHandler {
object MyApp extends App {
implicit val system = ActorSystem()
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
val route: Route =
// ... some route structure

View file

@ -11,7 +11,7 @@ class WebsocketExampleSpec extends WordSpec with Matchers {
pending // compile-time only test
//#websocket-example-using-core
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.{ Source, Flow }
import akka.http.scaladsl.Http
import akka.http.scaladsl.model.ws.UpgradeToWebsocket
@ -20,7 +20,7 @@ class WebsocketExampleSpec extends WordSpec with Matchers {
import akka.http.scaladsl.model.HttpMethods._
implicit val system = ActorSystem()
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
// The Greeter WebSocket Service expects a "name" per message and
// returns a greeting message for that name
@ -57,14 +57,14 @@ class WebsocketExampleSpec extends WordSpec with Matchers {
pending // compile-time only test
//#websocket-example-using-routing
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.{ Source, Flow }
import akka.http.scaladsl.Http
import akka.http.scaladsl.model.ws.{ TextMessage, Message }
import akka.http.scaladsl.server.Directives
implicit val system = ActorSystem()
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
import Directives._

View file

@ -5,7 +5,7 @@ package docs.stream
import scala.annotation.tailrec
import akka.actor.Props
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.actor.ActorPublisher
import akka.stream.scaladsl.{ Flow, Sink, Source }
import akka.stream.testkit.AkkaSpec
@ -69,7 +69,7 @@ object ActorPublisherDocSpec {
class ActorPublisherDocSpec extends AkkaSpec {
import ActorPublisherDocSpec._
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
"illustrate usage of ActorPublisher" in {
def println(s: String): Unit =

View file

@ -9,7 +9,7 @@ import akka.actor.Props
import akka.routing.ActorRefRoutee
import akka.routing.RoundRobinRoutingLogic
import akka.routing.Router
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.actor.ActorSubscriber
import akka.stream.actor.ActorSubscriberMessage
import akka.stream.actor.MaxInFlightRequestStrategy
@ -72,7 +72,7 @@ object ActorSubscriberDocSpec {
class ActorSubscriberDocSpec extends AkkaSpec {
import ActorSubscriberDocSpec._
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
"illustrate usage of ActorSubscriber" in {
val replyTo = testActor

View file

@ -130,7 +130,7 @@ object BidiFlowDocSpec {
class BidiFlowDocSpec extends AkkaSpec with ConversionCheckedTripleEquals {
import BidiFlowDocSpec._
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
"A BidiFlow" must {

View file

@ -29,7 +29,7 @@ class FlexiDocSpec extends AkkaSpec {
import FlexiDocSpec._
implicit val ec = system.dispatcher
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
"implement zip using readall" in {
//#fleximerge-zip-readall

View file

@ -14,10 +14,10 @@ class FlowDocSpec extends AkkaSpec {
implicit val ec = system.dispatcher
//#imports
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
//#imports
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
"source is immutable" in {
//#source-immutable

View file

@ -4,8 +4,8 @@
package docs.stream
import scala.concurrent.Await
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.ActorMaterializer
import akka.stream.ActorMaterializerSettings
import akka.stream.Supervision
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
@ -17,7 +17,7 @@ class FlowErrorDocSpec extends AkkaSpec {
"demonstrate fail stream" in {
//#stop
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
val source = Source(0 to 5).map(100 / _)
val result = source.runWith(Sink.fold(0)(_ + _))
// division by zero will fail the stream and the
@ -35,8 +35,8 @@ class FlowErrorDocSpec extends AkkaSpec {
case _: ArithmeticException => Supervision.Resume
case _ => Supervision.Stop
}
implicit val mat = ActorFlowMaterializer(
ActorFlowMaterializerSettings(system).withSupervisionStrategy(decider))
implicit val mat = ActorMaterializer(
ActorMaterializerSettings(system).withSupervisionStrategy(decider))
val source = Source(0 to 5).map(100 / _)
val result = source.runWith(Sink.fold(0)(_ + _))
// the element causing division by zero will be dropped
@ -48,7 +48,7 @@ class FlowErrorDocSpec extends AkkaSpec {
"demonstrate resume section" in {
//#resume-section
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
val decider: Supervision.Decider = {
case _: ArithmeticException => Supervision.Resume
case _ => Supervision.Stop
@ -68,7 +68,7 @@ class FlowErrorDocSpec extends AkkaSpec {
"demonstrate restart section" in {
//#restart-section
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
val decider: Supervision.Decider = {
case _: IllegalArgumentException => Supervision.Restart
case _ => Supervision.Stop

View file

@ -16,7 +16,7 @@ class FlowGraphDocSpec extends AkkaSpec {
implicit val ec = system.dispatcher
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
"build simple graph" in {
//format: OFF

View file

@ -1,6 +1,6 @@
package docs.stream
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.{ RunnableFlow, Sink, Source, Flow, Keep }
import akka.stream.stage.PushPullStage
import akka.stream.testkit.AkkaSpec
@ -15,7 +15,7 @@ class FlowStagesSpec extends AkkaSpec with ScalaFutures {
import akka.stream.stage._
//#import-stage
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
"stages demo" must {

View file

@ -1,12 +1,12 @@
package docs.stream
import akka.stream.{ OverflowStrategy, ActorFlowMaterializer }
import akka.stream.{ OverflowStrategy, ActorMaterializer }
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
class GraphCyclesSpec extends AkkaSpec {
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
"Cycle demonstration" must {
val source = Source(() => Iterator.from(0))

View file

@ -6,7 +6,7 @@ package docs.stream
import scala.concurrent.duration._
import akka.stream.testkit.AkkaSpec
import akka.stream.scaladsl._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import scala.concurrent.Future
import akka.testkit.TestProbe
import akka.actor.ActorRef
@ -18,7 +18,7 @@ import akka.util.Timeout
import akka.stream.Attributes
import akka.stream.ActorAttributes
import scala.concurrent.ExecutionContext
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.ActorMaterializerSettings
import java.util.concurrent.atomic.AtomicInteger
import akka.stream.Supervision
import akka.stream.scaladsl.Flow
@ -124,7 +124,7 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
import TwitterStreamQuickstartDocSpec._
import IntegrationDocSpec._
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
"calling external service with mapAsync" in {
val probe = TestProbe()
@ -322,8 +322,8 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
implicit val blockingExecutionContext = system.dispatchers.lookup("blocking-dispatcher")
val service = new SometimesSlowService
implicit val mat = ActorFlowMaterializer(
ActorFlowMaterializerSettings(system).withInputBuffer(initialSize = 4, maxSize = 4))
implicit val mat = ActorMaterializer(
ActorMaterializerSettings(system).withInputBuffer(initialSize = 4, maxSize = 4))
Source(List("a", "B", "C", "D", "e", "F", "g", "H", "i", "J"))
.map(elem => { println(s"before: $elem"); elem })
@ -354,8 +354,8 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
implicit val blockingExecutionContext = system.dispatchers.lookup("blocking-dispatcher")
val service = new SometimesSlowService
implicit val mat = ActorFlowMaterializer(
ActorFlowMaterializerSettings(system).withInputBuffer(initialSize = 4, maxSize = 4))
implicit val mat = ActorMaterializer(
ActorMaterializerSettings(system).withInputBuffer(initialSize = 4, maxSize = 4))
Source(List("a", "B", "C", "D", "e", "F", "g", "H", "i", "J"))
.map(elem => { println(s"before: $elem"); elem })

View file

@ -3,7 +3,7 @@
*/
package docs.stream
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.Flow
import akka.stream.testkit._
import akka.stream.scaladsl.Sink
@ -12,7 +12,7 @@ import akka.stream.scaladsl.Source
class ReactiveStreamsDocSpec extends AkkaSpec {
import TwitterStreamQuickstartDocSpec._
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
//#imports
import org.reactivestreams.Publisher

View file

@ -1,12 +1,12 @@
package docs.stream
import akka.stream.{ OverflowStrategy, ActorFlowMaterializerSettings, ActorFlowMaterializer }
import akka.stream.{ OverflowStrategy, ActorMaterializerSettings, ActorMaterializer }
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.stream.Attributes
class StreamBuffersRateSpec extends AkkaSpec {
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
"Demonstrate pipelining" in {
def println(s: Any) = ()
@ -21,8 +21,8 @@ class StreamBuffersRateSpec extends AkkaSpec {
"Demonstrate buffer sizes" in {
//#materializer-buffer
val materializer = ActorFlowMaterializer(
ActorFlowMaterializerSettings(system)
val materializer = ActorMaterializer(
ActorMaterializerSettings(system)
.withInputBuffer(
initialSize = 64,
maxSize = 64))

View file

@ -16,7 +16,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
implicit val ec = system.dispatcher
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
"build with open ports" in {
//#simple-partial-flow-graph

View file

@ -16,7 +16,7 @@ import akka.pattern
class StreamTestKitDocSpec extends AkkaSpec {
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
"strict collection" in {
//#strict-collection

View file

@ -6,7 +6,7 @@ package docs.stream
//#imports
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.OverflowStrategy
import akka.stream.scaladsl._
@ -62,11 +62,11 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
trait Example1 {
//#materializer-setup
implicit val system = ActorSystem("reactive-tweets")
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
//#materializer-setup
}
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
"filter and map" in {
//#authors-filter-map

View file

@ -1,6 +1,6 @@
package docs.stream.cookbook
import akka.stream.{ ActorFlowMaterializerSettings, ActorFlowMaterializer }
import akka.stream.{ ActorMaterializerSettings, ActorMaterializer }
import akka.stream.scaladsl._
import akka.stream.testkit._
@ -10,7 +10,7 @@ import scala.concurrent.duration._
class RecipeCollectingMetrics extends RecipeSpec {
import HoldOps._
implicit val m2 = ActorFlowMaterializer(ActorFlowMaterializerSettings(system).withInputBuffer(1, 1))
implicit val m2 = ActorMaterializer(ActorMaterializerSettings(system).withInputBuffer(1, 1))
"Recipe for periodically collecting metrics" must {

View file

@ -1,11 +1,11 @@
package docs.stream.cookbook
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.testkit.AkkaSpec
trait RecipeSpec extends AkkaSpec {
implicit val m = ActorFlowMaterializer()
implicit val m = ActorMaterializer()
type Message = String
type Trigger = Unit
type Job = String

View file

@ -19,7 +19,7 @@ import scala.concurrent.Future
class StreamFileDocSpec extends AkkaSpec(UnboundedMailboxConfig) {
implicit val ec = system.dispatcher
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
// silence sysout
def println(s: String) = ()

View file

@ -23,7 +23,7 @@ import scala.concurrent.Future
class StreamTcpDocSpec extends AkkaSpec {
implicit val ec = system.dispatcher
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
// silence sysout
def println(s: String) = ()

View file

@ -42,7 +42,7 @@ Directive Description
:ref:`-extractCredentials-` Extracts the potentially present ``HttpCredentials`` provided with the
request's ``Authorization`` header
:ref:`-extractExecutionContext-` Extracts the ``ExecutionContext`` from the ``RequestContext``
:ref:`-extractFlowMaterializer-` Extracts the ``FlowMaterializer`` from the ``RequestContext``
:ref:`-extractMaterializer-` Extracts the ``Materializer`` from the ``RequestContext``
:ref:`-extractHost-` Extracts the hostname part of the Host request header value
:ref:`-extractLog-` Extracts the ``LoggingAdapter`` from the ``RequestContext``
:ref:`-extractMethod-` Extracts the request method
@ -191,7 +191,7 @@ Directive Description
:ref:`-tprovide-` Injects a given tuple of values into a directive
:ref:`-validate-` Checks a given condition before running its inner route
:ref:`-withExecutionContext-` Runs its inner route with the given alternative ``ExecutionContext``
:ref:`-withFlowMaterializer-` Runs its inner route with the given alternative ``FlowMaterializer``
:ref:`-withMaterializer-` Runs its inner route with the given alternative ``Materializer``
:ref:`-withLog-` Runs its inner route with the given alternative ``LoggingAdapter``
:ref:`-withRangeSupport-` Adds ``Accept-Ranges: bytes`` to responses to GET requests, produces partial
responses if the initial request contained a valid ``Range`` header

View file

@ -1,6 +1,6 @@
.. _-extractFlowMaterializer-:
.. _-extractMaterializer-:
extractFlowMaterializer
extractMaterializer
=======================
...
@ -9,7 +9,7 @@ Signature
---------
.. includecode2:: /../../akka-http/src/main/scala/akka/http/scaladsl/server/directives/BasicDirectives.scala
:snippet: extractFlowMaterializer
:snippet: extractMaterializer
Description
-----------
@ -20,4 +20,4 @@ Example
-------
... includecode2:: ../../../../code/docs/http/scaladsl/server/directives/BasicDirectivesExamplesSpec.scala
:snippet: 0extractFlowMaterializer
:snippet: 0extractMaterializer

View file

@ -18,7 +18,7 @@ a single value or a tuple of values.
* :ref:`-extract-`
* :ref:`-extractExecutionContext-`
* :ref:`-extractFlowMaterializer-`
* :ref:`-extractMaterializer-`
* :ref:`-extractLog-`
* :ref:`-extractRequest-`
* :ref:`-extractRequestContext-`
@ -40,7 +40,7 @@ Transforming the Request(Context)
* :ref:`-mapSettings-`
* :ref:`-mapUnmatchedPath-`
* :ref:`-withExecutionContext-`
* :ref:`-withFlowMaterializer-`
* :ref:`-withMaterializer-`
* :ref:`-withLog-`
* :ref:`-withSettings-`
@ -94,7 +94,7 @@ Alphabetically
cancelRejections
extract
extractExecutionContext
extractFlowMaterializer
extractMaterializer
extractLog
extractRequest
extractRequestContext
@ -122,6 +122,6 @@ Alphabetically
textract
tprovide
withExecutionContext
withFlowMaterializer
withMaterializer
withLog
withSettings

View file

@ -1,6 +1,6 @@
.. _-withFlowMaterializer-:
.. _-withMaterializer-:
withFlowMaterializer
withMaterializer
====================
...
@ -9,7 +9,7 @@ Signature
---------
.. includecode2:: /../../akka-http/src/main/scala/akka/http/scaladsl/server/directives/BasicDirectives.scala
:snippet: withFlowMaterializer
:snippet: withMaterializer
Description
-----------
@ -20,4 +20,4 @@ Example
-------
... includecode2:: ../../../../code/docs/http/scaladsl/server/directives/BasicDirectivesExamplesSpec.scala
:snippet: 0withFlowMaterializer
:snippet: 0withMaterializer

View file

@ -37,7 +37,7 @@ RequestContext
--------------
The request context wraps an ``HttpRequest`` instance to enrich it with additional information that are typically
required by the routing logic, like an ``ExecutionContext``, ``FlowMaterializer``, ``LoggingAdapter`` and the configured
required by the routing logic, like an ``ExecutionContext``, ``Materializer``, ``LoggingAdapter`` and the configured
``RoutingSettings``. It also contains the ``unmatchedPath``, a value that describes how much of the request URI has not
yet been matched by a :ref:`Path Directive <PathDirectives>`.

View file

@ -200,7 +200,7 @@ well-known sinks, such as ``runForeach(el => )`` (being an alias to ``runWith(Si
Materialization is currently performed synchronously on the materializing thread.
The actual stream processing is handled by actors started up during the streams materialization,
which will be running on the thread pools they have been configured to run on - which defaults to the dispatcher set in
:class:`MaterializationSettings` while constructing the :class:`ActorFlowMaterializer`.
:class:`MaterializationSettings` while constructing the :class:`ActorMaterializer`.
.. note::
Reusing *instances* of linear computation stages (Source, Sink, Flow) inside FlowGraphs is legal,

View file

@ -294,7 +294,7 @@ is ``completed`` before ``g``, but still emitted afterwards.
The numbers in parenthesis illustrates how many calls that are in progress at
the same time. Here the downstream demand and thereby the number of concurrent
calls are limited by the buffer size (4) of the :class:`ActorFlowMaterializerSettings`.
calls are limited by the buffer size (4) of the :class:`ActorMaterializerSettings`.
Here is how we can use the same service with ``mapAsyncUnordered``:
@ -350,7 +350,7 @@ Note that ``after`` lines are not in the same order as the ``before`` lines. For
The numbers in parenthesis illustrates how many calls that are in progress at
the same time. Here the downstream demand and thereby the number of concurrent
calls are limited by the buffer size (4) of the :class:`ActorFlowMaterializerSettings`.
calls are limited by the buffer size (4) of the :class:`ActorMaterializerSettings`.
.. _reactive-streams-integration-scala:

View file

@ -19,12 +19,12 @@ Here's the data model we'll be working with throughout the quickstart examples:
Transforming and consuming simple streams
-----------------------------------------
In order to prepare our environment by creating an :class:`ActorSystem` and :class:`ActorFlowMaterializer`,
In order to prepare our environment by creating an :class:`ActorSystem` and :class:`ActorMaterializer`,
which will be responsible for materializing and running the streams we are about to create:
.. includecode:: code/docs/stream/TwitterStreamQuickstartDocSpec.scala#materializer-setup
The :class:`ActorFlowMaterializer` can optionally take :class:`ActorFlowMaterializerSettings` which can be used to define
The :class:`ActorMaterializer` can optionally take :class:`ActorMaterializerSettings` which can be used to define
materialization properties, such as default buffer sizes (see also :ref:`stream-buffers-scala`), the dispatcher to
be used by the pipeline etc. These can be overridden ``withAttributes`` on :class:`Flow`, :class:`Source`, :class:`Sink` and :class:`Graph`.
@ -55,7 +55,7 @@ or by using the shorthand version (which are defined only for the most popular s
.. includecode:: code/docs/stream/TwitterStreamQuickstartDocSpec.scala#authors-foreach-println
Materializing and running a stream always requires a :class:`FlowMaterializer` to be in implicit scope (or passed in explicitly,
Materializing and running a stream always requires a :class:`Materializer` to be in implicit scope (or passed in explicitly,
like this: ``.run(materializer)``).
Flattening sequences in streams
@ -151,7 +151,7 @@ and because of using ``Keep.right``, the resulting :class:`RunnableFlow` has als
This step does *not* yet materialize the
processing pipeline, it merely prepares the description of the Flow, which is now connected to a Sink, and therefore can
be ``run()``, as indicated by its type: :class:`RunnableFlow[Future[Int]]`. Next we call ``run()`` which uses the implicit :class:`ActorFlowMaterializer`
be ``run()``, as indicated by its type: :class:`RunnableFlow[Future[Int]]`. Next we call ``run()`` which uses the implicit :class:`ActorMaterializer`
to materialize and run the flow. The value returned by calling ``run()`` on a ``RunnableFlow[T]`` is of type ``T``.
In our case this type is ``Future[Int]`` which, when completed, will contain the total length of our tweets stream.
In case of the stream failing, this future would complete with a Failure.

View file

@ -62,7 +62,7 @@ to a level that throughput requirements of the application require. Default buff
akka.stream.materializer.max-input-buffer-size = 16
Alternatively they can be set by passing a :class:`ActorFlowMaterializerSettings` to the materializer:
Alternatively they can be set by passing a :class:`ActorMaterializerSettings` to the materializer:
.. includecode:: code/docs/stream/StreamBuffersRateSpec.scala#materializer-buffer

View file

@ -102,7 +102,7 @@ It is commonly necessary to parameterize a flow so that it can be materialized f
It might be tempting to allow different pieces of a flow topology to access the materialization results of other pieces in order to customize their behavior, but that would violate composability and reusability as argued above. Therefore the arguments and results of materialization need to be segregated:
* The FlowMaterializer is configured with a (type-safe) mapping from keys to values, which is exposed to the processing stages during their materialization.
* The Materializer is configured with a (type-safe) mapping from keys to values, which is exposed to the processing stages during their materialization.
* The values in this mapping may act as channels, for example by using a Promise/Future pair to communicate a value; another possibility for such information-passing is of course to explicitly model it as a stream of configuration data elements within the graph itself.
* The materialized values obtained from the processing stages are combined as prescribed by the user, but can of course be dependent on the values in the argument mapping.

View file

@ -11,7 +11,7 @@ import scala.concurrent.{ Promise, Future }
import scala.util.Try
import akka.event.LoggingAdapter
import akka.actor._
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.scaladsl._
import akka.http.scaladsl.model._
import akka.http.scaladsl.Http
@ -69,7 +69,7 @@ private object PoolFlow {
*/
def apply(connectionFlow: Flow[HttpRequest, HttpResponse, Future[Http.OutgoingConnection]],
remoteAddress: InetSocketAddress, settings: ConnectionPoolSettings, log: LoggingAdapter)(
implicit system: ActorSystem, fm: FlowMaterializer): Flow[RequestContext, ResponseContext, Unit] =
implicit system: ActorSystem, fm: Materializer): Flow[RequestContext, ResponseContext, Unit] =
Flow() { implicit b
import settings._
import FlowGraph.Implicits._

View file

@ -7,7 +7,7 @@ import akka.http.HostConnectionPoolSetup
import akka.actor.{ Deploy, Props, ActorSystem, ActorRef }
import akka.http.scaladsl.Http
import akka.http.scaladsl.model.{ HttpResponse, HttpRequest }
import akka.stream.FlowMaterializer
import akka.stream.Materializer
private object PoolGateway {
@ -34,7 +34,7 @@ private object PoolGateway {
*/
private[http] class PoolGateway(hcps: HostConnectionPoolSetup,
_shutdownStartedPromise: Promise[Unit])( // constructor arg only
implicit system: ActorSystem, fm: FlowMaterializer) {
implicit system: ActorSystem, fm: Materializer) {
import PoolGateway._
import fm.executionContext

View file

@ -9,7 +9,7 @@ import scala.annotation.tailrec
import scala.concurrent.Promise
import scala.concurrent.duration.FiniteDuration
import akka.actor._
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.actor.{ ActorPublisher, ActorSubscriber, ZeroRequestStrategy }
import akka.stream.actor.ActorPublisherMessage._
import akka.stream.actor.ActorSubscriberMessage._
@ -45,7 +45,7 @@ private object PoolInterfaceActor {
*/
private class PoolInterfaceActor(hcps: HostConnectionPoolSetup,
shutdownCompletedPromise: Promise[Unit],
gateway: PoolGateway)(implicit fm: FlowMaterializer)
gateway: PoolGateway)(implicit fm: Materializer)
extends ActorSubscriber with ActorPublisher[RequestContext] with ActorLogging {
import PoolInterfaceActor._

View file

@ -53,7 +53,7 @@ private object PoolSlot {
*/
def apply(slotIx: Int, connectionFlow: Flow[HttpRequest, HttpResponse, Any],
remoteAddress: InetSocketAddress, // TODO: remove after #16168 is cleared
settings: ConnectionPoolSettings)(implicit system: ActorSystem, fm: FlowMaterializer): Graph[Ports, Any] =
settings: ConnectionPoolSettings)(implicit system: ActorSystem, fm: Materializer): Graph[Ports, Any] =
FlowGraph.partial() { implicit b
import FlowGraph.Implicits._
@ -84,7 +84,7 @@ private object PoolSlot {
* shutting down completely).
*/
private class SlotProcessor(slotIx: Int, connectionFlow: Flow[HttpRequest, HttpResponse, Any],
settings: ConnectionPoolSettings)(implicit fm: FlowMaterializer)
settings: ConnectionPoolSettings)(implicit fm: Materializer)
extends ActorSubscriber with ActorPublisher[List[ProcessorOut]] with ActorLogging {
var exposedPublisher: akka.stream.impl.ActorPublisher[Any] = _

View file

@ -34,7 +34,7 @@ private[http] object HttpServerBluePrint {
type ServerShape = BidiShape[HttpResponse, SslTlsOutbound, SslTlsInbound, HttpRequest]
def apply(settings: ServerSettings, remoteAddress: Option[InetSocketAddress], log: LoggingAdapter)(implicit mat: FlowMaterializer): Graph[ServerShape, Unit] = {
def apply(settings: ServerSettings, remoteAddress: Option[InetSocketAddress], log: LoggingAdapter)(implicit mat: Materializer): Graph[ServerShape, Unit] = {
import settings._
// the initial header parser we initially use for every connection,
@ -291,7 +291,7 @@ private[http] object HttpServerBluePrint {
trait WebsocketSetup {
def websocketFlow: Flow[ByteString, ByteString, Any]
def installHandler(handlerFlow: Flow[FrameEvent, FrameEvent, Any])(implicit mat: FlowMaterializer): Unit
def installHandler(handlerFlow: Flow[FrameEvent, FrameEvent, Any])(implicit mat: Materializer): Unit
}
def websocketSetup: WebsocketSetup = {
val sinkCell = new StreamUtils.OneTimeWriteCell[Publisher[FrameEvent]]
@ -309,7 +309,7 @@ private[http] object HttpServerBluePrint {
new WebsocketSetup {
def websocketFlow: Flow[ByteString, ByteString, Any] = flow
def installHandler(handlerFlow: Flow[FrameEvent, FrameEvent, Any])(implicit mat: FlowMaterializer): Unit =
def installHandler(handlerFlow: Flow[FrameEvent, FrameEvent, Any])(implicit mat: Materializer): Unit =
Source(sinkCell.value)
.via(handlerFlow)
.to(Sink(sourceCell.value))

View file

@ -8,7 +8,7 @@ import akka.http.scaladsl.model.headers._
import akka.http.scaladsl.model.ws.{ Message, UpgradeToWebsocket }
import akka.http.scaladsl.model.{ StatusCodes, HttpResponse, HttpProtocol, HttpHeader }
import akka.parboiled2.util.Base64
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.scaladsl.Flow
import scala.collection.immutable.Seq

View file

@ -6,7 +6,7 @@ package akka.http.impl.engine.ws
import akka.http.scaladsl.model.HttpResponse
import akka.http.scaladsl.model.ws.{ Message, UpgradeToWebsocket }
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.scaladsl.Flow
/**

View file

@ -5,7 +5,7 @@
package akka.http.impl.engine.ws
import akka.http.scaladsl.model.headers.CustomHeader
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.scaladsl.Flow
private[http] case class UpgradeToWebsocketResponseHeader(handlerFlow: Flow[FrameEvent, FrameEvent, Any])

View file

@ -154,7 +154,7 @@ private[http] object StreamUtils {
* Applies a sequence of transformers on one source and returns a sequence of sources with the result. The input source
* will only be traversed once.
*/
def transformMultiple(input: Source[ByteString, Any], transformers: immutable.Seq[Flow[ByteString, ByteString, Any]])(implicit materializer: FlowMaterializer): immutable.Seq[Source[ByteString, Any]] =
def transformMultiple(input: Source[ByteString, Any], transformers: immutable.Seq[Flow[ByteString, ByteString, Any]])(implicit materializer: Materializer): immutable.Seq[Source[ByteString, Any]] =
transformers match {
case Nil Nil
case Seq(one) Vector(input.via(one))
@ -354,8 +354,8 @@ private[http] object StreamUtils {
* INTERNAL API
*/
private[http] class EnhancedByteStringSource[Mat](val byteStringStream: Source[ByteString, Mat]) extends AnyVal {
def join(implicit materializer: FlowMaterializer): Future[ByteString] =
def join(implicit materializer: Materializer): Future[ByteString] =
byteStringStream.runFold(ByteString.empty)(_ ++ _)
def utf8String(implicit materializer: FlowMaterializer, ec: ExecutionContext): Future[String] =
def utf8String(implicit materializer: Materializer, ec: ExecutionContext): Future[String] =
join.map(_.utf8String)
}

View file

@ -15,7 +15,7 @@ import akka.japi.{ Option, Function }
import akka.actor.{ ExtendedActorSystem, ActorSystem, ExtensionIdProvider, ExtensionId }
import akka.event.LoggingAdapter
import akka.io.Inet
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.javadsl.{ Flow, Source }
import akka.http.impl.util.JavaMapping.Implicits._
import akka.http.scaladsl.{ model sm }
@ -46,7 +46,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
* fail, unless the first materialization has already been unbound. Unbinding can be triggered via the materialized
* [[ServerBinding]].
*/
def bind(interface: String, port: Int, materializer: FlowMaterializer): Source[IncomingConnection, Future[ServerBinding]] =
def bind(interface: String, port: Int, materializer: Materializer): Source[IncomingConnection, Future[ServerBinding]] =
Source.adapt(delegate.bind(interface, port)(materializer)
.map(new IncomingConnection(_))
.mapMaterializedValue(_.map(new ServerBinding(_))(ec)))
@ -65,7 +65,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
settings: ServerSettings,
httpsContext: Option[HttpsContext],
log: LoggingAdapter,
materializer: FlowMaterializer): Source[IncomingConnection, Future[ServerBinding]] =
materializer: Materializer): Source[IncomingConnection, Future[ServerBinding]] =
Source.adapt(delegate.bind(interface, port, settings, httpsContext, log)(materializer)
.map(new IncomingConnection(_))
.mapMaterializedValue(_.map(new ServerBinding(_))(ec)))
@ -80,7 +80,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
*/
def bindAndHandle(handler: Flow[HttpRequest, HttpResponse, _],
interface: String, port: Int,
materializer: FlowMaterializer): Future[ServerBinding] =
materializer: Materializer): Future[ServerBinding] =
delegate.bindAndHandle(handler.asInstanceOf[Flow[sm.HttpRequest, sm.HttpResponse, _]].asScala,
interface, port)(materializer)
.map(new ServerBinding(_))(ec)
@ -98,7 +98,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
settings: ServerSettings,
httpsContext: Option[HttpsContext],
log: LoggingAdapter,
materializer: FlowMaterializer): Future[ServerBinding] =
materializer: Materializer): Future[ServerBinding] =
delegate.bindAndHandle(handler.asInstanceOf[Flow[sm.HttpRequest, sm.HttpResponse, _]].asScala,
interface, port, settings, httpsContext, log)(materializer)
.map(new ServerBinding(_))(ec)
@ -113,7 +113,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
*/
def bindAndHandleSync(handler: Function[HttpRequest, HttpResponse],
interface: String, port: Int,
materializer: FlowMaterializer): Future[ServerBinding] =
materializer: Materializer): Future[ServerBinding] =
delegate.bindAndHandleSync(handler.apply(_).asScala, interface, port)(materializer)
.map(new ServerBinding(_))(ec)
@ -130,7 +130,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
settings: ServerSettings,
httpsContext: Option[HttpsContext],
log: LoggingAdapter,
materializer: FlowMaterializer): Future[ServerBinding] =
materializer: Materializer): Future[ServerBinding] =
delegate.bindAndHandleSync(handler.apply(_).asScala,
interface, port, settings, httpsContext, log)(materializer)
.map(new ServerBinding(_))(ec)
@ -145,7 +145,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
*/
def bindAndHandleAsync(handler: Function[HttpRequest, Future[HttpResponse]],
interface: String, port: Int,
materializer: FlowMaterializer): Future[ServerBinding] =
materializer: Materializer): Future[ServerBinding] =
delegate.bindAndHandleAsync(handler.apply(_).asInstanceOf[Future[sm.HttpResponse]], interface, port)(materializer)
.map(new ServerBinding(_))(ec)
@ -161,7 +161,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
interface: String, port: Int,
settings: ServerSettings, httpsContext: Option[HttpsContext],
parallelism: Int, log: LoggingAdapter,
materializer: FlowMaterializer): Future[ServerBinding] =
materializer: Materializer): Future[ServerBinding] =
delegate.bindAndHandleAsync(handler.apply(_).asInstanceOf[Future[sm.HttpResponse]],
interface, port, settings, httpsContext, parallelism, log)(materializer)
.map(new ServerBinding(_))(ec)
@ -233,13 +233,13 @@ 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 newHostConnectionPool[T](host: String, port: Int, materializer: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
def newHostConnectionPool[T](host: String, port: Int, materializer: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
adaptTupleFlow(delegate.newHostConnectionPool[T](host, port)(materializer))
/**
* Same as [[newHostConnectionPool]] but with HTTPS encryption.
*/
def newHostConnectionPoolTls[T](host: String, port: Int, materializer: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
def newHostConnectionPoolTls[T](host: String, port: Int, materializer: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
adaptTupleFlow(delegate.newHostConnectionPoolTls[T](host, port)(materializer))
/**
@ -259,7 +259,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
def newHostConnectionPool[T](host: String, port: Int,
options: JIterable[Inet.SocketOption],
settings: ConnectionPoolSettings,
log: LoggingAdapter, materializer: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
log: LoggingAdapter, materializer: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
adaptTupleFlow(delegate.newHostConnectionPool[T](host, port, settings, log)(materializer))
/**
@ -272,7 +272,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
options: JIterable[Inet.SocketOption],
settings: ConnectionPoolSettings,
httpsContext: Option[HttpsContext],
log: LoggingAdapter, materializer: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
log: LoggingAdapter, materializer: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
adaptTupleFlow(delegate.newHostConnectionPoolTls[T](host, port, settings,
httpsContext.map(_.asInstanceOf[akka.http.scaladsl.HttpsContext]), log)(materializer))
@ -290,7 +290,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 newHostConnectionPool[T](setup: HostConnectionPoolSetup, materializer: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
def newHostConnectionPool[T](setup: HostConnectionPoolSetup, materializer: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
adaptTupleFlow(delegate.newHostConnectionPool[T](setup)(materializer))
/**
@ -310,13 +310,13 @@ 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 cachedHostConnectionPool[T](host: String, port: Int, materializer: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
def cachedHostConnectionPool[T](host: String, port: Int, materializer: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
adaptTupleFlow(delegate.cachedHostConnectionPool[T](host, port)(materializer))
/**
* Same as [[cachedHostConnectionPool]] but with HTTPS encryption.
*/
def cachedHostConnectionPoolTls[T](host: String, port: Int, materializer: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
def cachedHostConnectionPoolTls[T](host: String, port: Int, materializer: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
adaptTupleFlow(delegate.cachedHostConnectionPoolTls[T](host, port)(materializer))
/**
@ -338,7 +338,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
*/
def cachedHostConnectionPool[T](host: String, port: Int,
settings: ConnectionPoolSettings,
log: LoggingAdapter, materializer: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
log: LoggingAdapter, materializer: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
adaptTupleFlow(delegate.cachedHostConnectionPool[T](host, port, settings, log)(materializer))
/**
@ -350,7 +350,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
def cachedHostConnectionPoolTls[T](host: String, port: Int,
settings: ConnectionPoolSettings,
httpsContext: Option[HttpsContext],
log: LoggingAdapter, materializer: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
log: LoggingAdapter, materializer: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
adaptTupleFlow(delegate.cachedHostConnectionPoolTls[T](host, port, settings,
httpsContext.map(_.asInstanceOf[akka.http.scaladsl.HttpsContext]), log)(materializer))
@ -371,7 +371,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 cachedHostConnectionPool[T](setup: HostConnectionPoolSetup, materializer: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
def cachedHostConnectionPool[T](setup: HostConnectionPoolSetup, materializer: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
adaptTupleFlow(delegate.cachedHostConnectionPool[T](setup)(materializer))
/**
@ -386,7 +386,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: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), Unit] =
def superPool[T](materializer: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), Unit] =
adaptTupleFlow(delegate.superPool[T]()(materializer))
/**
@ -406,7 +406,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
*/
def superPool[T](settings: ConnectionPoolSettings,
httpsContext: Option[HttpsContext],
log: LoggingAdapter, materializer: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), Unit] =
log: LoggingAdapter, materializer: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), Unit] =
adaptTupleFlow(delegate.superPool[T](settings, httpsContext, log)(materializer))
/**
@ -416,7 +416,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
* Note that the request must have either an absolute URI or a valid `Host` header, otherwise
* the future will be completed with an error.
*/
def singleRequest(request: HttpRequest, materializer: FlowMaterializer): Future[HttpResponse] =
def singleRequest(request: HttpRequest, materializer: Materializer): Future[HttpResponse] =
delegate.singleRequest(request.asScala)(materializer)
/**
@ -432,7 +432,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
def singleRequest(request: HttpRequest,
settings: ConnectionPoolSettings,
httpsContext: Option[HttpsContext],
log: LoggingAdapter, materializer: FlowMaterializer): Future[HttpResponse] =
log: LoggingAdapter, materializer: Materializer): Future[HttpResponse] =
delegate.singleRequest(request.asScala, settings, httpsContext, log)(materializer)
/**

View file

@ -6,7 +6,7 @@ package akka.http.javadsl
import java.net.InetSocketAddress
import scala.concurrent.Future
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.javadsl.Flow
import akka.http.javadsl.model._
import akka.http.scaladsl.{ model sm }
@ -36,20 +36,20 @@ class IncomingConnection private[http] (delegate: akka.http.scaladsl.Http.Incomi
* Handles the connection with the given flow, which is materialized exactly once
* and the respective materialization result returned.
*/
def handleWith[Mat](handler: Flow[HttpRequest, HttpResponse, Mat], materializer: FlowMaterializer): Mat =
def handleWith[Mat](handler: Flow[HttpRequest, HttpResponse, Mat], materializer: Materializer): Mat =
delegate.handleWith(handler.asInstanceOf[Flow[sm.HttpRequest, sm.HttpResponse, Mat]].asScala)(materializer)
/**
* Handles the connection with the given handler function.
* Returns the materialization result of the underlying flow materialization.
*/
def handleWithSyncHandler(handler: Function[HttpRequest, HttpResponse], materializer: FlowMaterializer): Unit =
def handleWithSyncHandler(handler: Function[HttpRequest, HttpResponse], materializer: Materializer): Unit =
delegate.handleWithSyncHandler(handler.apply(_).asInstanceOf[sm.HttpResponse])(materializer)
/**
* Handles the connection with the given handler function.
* Returns the materialization result of the underlying flow materialization.
*/
def handleWithAsyncHandler(handler: Function[HttpRequest, Future[HttpResponse]], materializer: FlowMaterializer): Unit =
def handleWithAsyncHandler(handler: Function[HttpRequest, Future[HttpResponse]], materializer: Materializer): Unit =
delegate.handleWithAsyncHandler(handler.apply(_).asInstanceOf[Future[sm.HttpResponse]])(materializer)
}

View file

@ -7,7 +7,7 @@ package akka.http.javadsl.model.ws
import java.lang.{ Iterable JIterable }
import akka.http.scaladsl.{ model sm }
import akka.http.javadsl.model._
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.javadsl.{ Sink, Source, Flow }
/**

View file

@ -4,7 +4,7 @@
package akka.http.javadsl.model.ws
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.javadsl.Flow
import akka.http.javadsl.model._
import akka.http.impl.util.JavaMapping.Implicits._

View file

@ -15,7 +15,7 @@ import scala.collection.{ JavaConverters, immutable }
import scala.concurrent.{ ExecutionContext, Promise, Future }
import akka.japi
import akka.event.LoggingAdapter
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.io._
import akka.stream.scaladsl._
import akka.http.impl.util.StreamUtils
@ -54,7 +54,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
def bind(interface: String, port: Int = -1,
settings: ServerSettings = ServerSettings(system),
httpsContext: Option[HttpsContext] = None,
log: LoggingAdapter = system.log)(implicit fm: FlowMaterializer): Source[IncomingConnection, Future[ServerBinding]] = {
log: LoggingAdapter = system.log)(implicit fm: Materializer): Source[IncomingConnection, Future[ServerBinding]] = {
val effectivePort = if (port >= 0) port else if (httpsContext.isEmpty) 80 else 443
val tlsStage = sslTlsStage(httpsContext, Server)
val connections: Source[Tcp.IncomingConnection, Future[Tcp.ServerBinding]] =
@ -80,7 +80,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
interface: String, port: Int = -1,
settings: ServerSettings = ServerSettings(system),
httpsContext: Option[HttpsContext] = None,
log: LoggingAdapter = system.log)(implicit fm: FlowMaterializer): Future[ServerBinding] = {
log: LoggingAdapter = system.log)(implicit fm: Materializer): Future[ServerBinding] = {
def handleOneConnection(incomingConnection: IncomingConnection): Future[Unit] =
try
incomingConnection.flow
@ -111,7 +111,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
interface: String, port: Int = -1,
settings: ServerSettings = ServerSettings(system),
httpsContext: Option[HttpsContext] = None,
log: LoggingAdapter = system.log)(implicit fm: FlowMaterializer): Future[ServerBinding] =
log: LoggingAdapter = system.log)(implicit fm: Materializer): Future[ServerBinding] =
bindAndHandle(Flow[HttpRequest].map(handler), interface, port, settings, httpsContext, log)
/**
@ -127,7 +127,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
settings: ServerSettings = ServerSettings(system),
httpsContext: Option[HttpsContext] = None,
parallelism: Int = 1,
log: LoggingAdapter = system.log)(implicit fm: FlowMaterializer): Future[ServerBinding] =
log: LoggingAdapter = system.log)(implicit fm: Materializer): Future[ServerBinding] =
bindAndHandle(Flow[HttpRequest].mapAsync(parallelism)(handler), interface, port, settings, httpsContext, log)
/**
@ -148,7 +148,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
* Constructs a [[ServerLayer]] stage using the configured default [[ServerSettings]]. The returned [[BidiFlow]]
* can only be materialized once.
*/
def serverLayer()(implicit mat: FlowMaterializer): ServerLayer = serverLayer(ServerSettings(system))
def serverLayer()(implicit mat: Materializer): ServerLayer = serverLayer(ServerSettings(system))
/**
* Constructs a [[ServerLayer]] stage using the given [[ServerSettings]]. The returned [[BidiFlow]] isn't reusable and
@ -156,7 +156,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
*/
def serverLayer(settings: ServerSettings,
remoteAddress: Option[InetSocketAddress] = None,
log: LoggingAdapter = system.log)(implicit mat: FlowMaterializer): ServerLayer =
log: LoggingAdapter = system.log)(implicit mat: Materializer): ServerLayer =
BidiFlow.wrap(HttpServerBluePrint(settings, remoteAddress, log))
/**
@ -241,7 +241,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
*/
def newHostConnectionPool[T](host: String, port: Int = 80,
settings: ConnectionPoolSettings = ConnectionPoolSettings(system),
log: LoggingAdapter = system.log)(implicit fm: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] = {
log: LoggingAdapter = system.log)(implicit fm: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] = {
val cps = ConnectionPoolSetup(settings, None, log)
newHostConnectionPool(HostConnectionPoolSetup(host, port, cps))
}
@ -255,7 +255,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
def newHostConnectionPoolTls[T](host: String, port: Int = 443,
settings: ConnectionPoolSettings = ConnectionPoolSettings(system),
httpsContext: Option[HttpsContext] = None,
log: LoggingAdapter = system.log)(implicit fm: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] = {
log: LoggingAdapter = system.log)(implicit fm: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] = {
val cps = ConnectionPoolSetup(settings, effectiveHttpsContext(httpsContext), log)
newHostConnectionPool(HostConnectionPoolSetup(host, port, cps))
}
@ -275,7 +275,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
* object of type ``T`` from the application which is emitted together with the corresponding response.
*/
def newHostConnectionPool[T](setup: HostConnectionPoolSetup)(
implicit fm: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] = {
implicit fm: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] = {
val gatewayFuture = FastFuture.successful(new PoolGateway(setup, Promise()))
gatewayClientFlow(setup, gatewayFuture)
}
@ -299,7 +299,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
*/
def cachedHostConnectionPool[T](host: String, port: Int = 80,
settings: ConnectionPoolSettings = ConnectionPoolSettings(system),
log: LoggingAdapter = system.log)(implicit fm: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] = {
log: LoggingAdapter = system.log)(implicit fm: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] = {
val cps = ConnectionPoolSetup(settings, None, log)
val setup = HostConnectionPoolSetup(host, port, cps)
cachedHostConnectionPool(setup)
@ -314,7 +314,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
def cachedHostConnectionPoolTls[T](host: String, port: Int = 443,
settings: ConnectionPoolSettings = ConnectionPoolSettings(system),
httpsContext: Option[HttpsContext] = None,
log: LoggingAdapter = system.log)(implicit fm: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] = {
log: LoggingAdapter = system.log)(implicit fm: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] = {
val cps = ConnectionPoolSetup(settings, effectiveHttpsContext(httpsContext), log)
val setup = HostConnectionPoolSetup(host, port, cps)
cachedHostConnectionPool(setup)
@ -338,7 +338,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
* object of type ``T`` from the application which is emitted together with the corresponding response.
*/
def cachedHostConnectionPool[T](setup: HostConnectionPoolSetup)(
implicit fm: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
implicit fm: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
gatewayClientFlow(setup, cachedGateway(setup))
/**
@ -357,7 +357,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
*/
def superPool[T](settings: ConnectionPoolSettings = ConnectionPoolSettings(system),
httpsContext: Option[HttpsContext] = None,
log: LoggingAdapter = system.log)(implicit fm: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), Unit] =
log: LoggingAdapter = system.log)(implicit fm: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), Unit] =
clientFlow[T](settings) { request request -> cachedGateway(request, settings, httpsContext, log) }
/**
@ -372,7 +372,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
def singleRequest(request: HttpRequest,
settings: ConnectionPoolSettings = ConnectionPoolSettings(system),
httpsContext: Option[HttpsContext] = None,
log: LoggingAdapter = system.log)(implicit fm: FlowMaterializer): Future[HttpResponse] =
log: LoggingAdapter = system.log)(implicit fm: Materializer): Future[HttpResponse] =
try {
val gatewayFuture = cachedGateway(request, settings, httpsContext, log)
gatewayFuture.flatMap(_(request))(fm.executionContext)
@ -423,7 +423,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
private def cachedGateway(request: HttpRequest,
settings: ConnectionPoolSettings, httpsContext: Option[HttpsContext],
log: LoggingAdapter)(implicit fm: FlowMaterializer): Future[PoolGateway] =
log: LoggingAdapter)(implicit fm: Materializer): Future[PoolGateway] =
if (request.uri.scheme.nonEmpty && request.uri.authority.nonEmpty) {
val httpsCtx = if (request.uri.scheme.equalsIgnoreCase("https")) effectiveHttpsContext(httpsContext) else None
val setup = ConnectionPoolSetup(settings, httpsCtx, log)
@ -435,7 +435,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
throw new IllegalUriException(ErrorInfo(msg))
}
private[http] def cachedGateway(setup: HostConnectionPoolSetup)(implicit fm: FlowMaterializer): Future[PoolGateway] = {
private[http] def cachedGateway(setup: HostConnectionPoolSetup)(implicit fm: Materializer): Future[PoolGateway] = {
val gatewayPromise = Promise[PoolGateway]()
hostPoolCache.putIfAbsent(setup, gatewayPromise.future) match {
case null // only one thread can get here at a time
@ -460,12 +460,12 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
private def gatewayClientFlow[T](hcps: HostConnectionPoolSetup,
gatewayFuture: Future[PoolGateway])(
implicit fm: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
implicit fm: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), HostConnectionPool] =
clientFlow[T](hcps.setup.settings)(_ -> gatewayFuture)
.mapMaterializedValue(_ HostConnectionPool(hcps)(gatewayFuture))
private def clientFlow[T](settings: ConnectionPoolSettings)(f: HttpRequest (HttpRequest, Future[PoolGateway]))(
implicit system: ActorSystem, fm: FlowMaterializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), Unit] = {
implicit system: ActorSystem, fm: Materializer): Flow[(HttpRequest, T), (Try[HttpResponse], T), Unit] = {
// 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) {
@ -520,21 +520,21 @@ object Http extends ExtensionId[HttpExt] with ExtensionIdProvider {
* Handles the connection with the given flow, which is materialized exactly once
* and the respective materialization result returned.
*/
def handleWith[Mat](handler: Flow[HttpRequest, HttpResponse, Mat])(implicit fm: FlowMaterializer): Mat =
def handleWith[Mat](handler: Flow[HttpRequest, HttpResponse, Mat])(implicit fm: Materializer): Mat =
flow.joinMat(handler)(Keep.right).run()
/**
* Handles the connection with the given handler function.
* Returns the materialization result of the underlying flow materialization.
*/
def handleWithSyncHandler(handler: HttpRequest HttpResponse)(implicit fm: FlowMaterializer): Unit =
def handleWithSyncHandler(handler: HttpRequest HttpResponse)(implicit fm: Materializer): Unit =
handleWith(Flow[HttpRequest].map(handler))
/**
* Handles the connection with the given handler function.
* Returns the materialization result of the underlying flow materialization.
*/
def handleWithAsyncHandler(handler: HttpRequest Future[HttpResponse])(implicit fm: FlowMaterializer): Unit =
def handleWithAsyncHandler(handler: HttpRequest Future[HttpResponse])(implicit fm: Materializer): Unit =
handleWith(Flow[HttpRequest].mapAsync(1)(handler))
}

View file

@ -11,7 +11,7 @@ import scala.concurrent.Future
import scala.concurrent.duration.FiniteDuration
import scala.collection.immutable
import akka.util.ByteString
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.scaladsl._
import akka.stream.io.SynchronousFileSource
import akka.{ japi, stream }
@ -53,7 +53,7 @@ sealed trait HttpEntity extends jm.HttpEntity {
* Collects all possible parts and returns a potentially future Strict entity for easier processing.
* The Future is failed with an TimeoutException if the stream isn't completed after the given timeout.
*/
def toStrict(timeout: FiniteDuration)(implicit fm: FlowMaterializer): Future[HttpEntity.Strict] = {
def toStrict(timeout: FiniteDuration)(implicit fm: Materializer): Future[HttpEntity.Strict] = {
def transformer() =
new TimerTransformer[ByteString, HttpEntity.Strict] {
var bytes = ByteString.newBuilder
@ -174,7 +174,7 @@ object HttpEntity {
def dataBytes: Source[ByteString, Unit] = Source(data :: Nil)
override def toStrict(timeout: FiniteDuration)(implicit fm: FlowMaterializer) =
override def toStrict(timeout: FiniteDuration)(implicit fm: Materializer) =
FastFuture.successful(this)
override def transformDataBytes(transformer: Flow[ByteString, ByteString, Any]): MessageEntity =

View file

@ -12,7 +12,7 @@ import scala.concurrent.{ Future, ExecutionContext }
import scala.collection.immutable
import scala.reflect.{ classTag, ClassTag }
import akka.parboiled2.CharUtils
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.util.ByteString
import akka.http.impl.model.JavaUri
import akka.http.impl.util._
@ -60,7 +60,7 @@ sealed trait HttpMessage extends jm.HttpMessage {
def withEntity(entity: MessageEntity): Self
/** Returns a sharable and serializable copy of this message with a strict entity. */
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Self] =
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: Materializer): Future[Self] =
entity.toStrict(timeout).fast.map(this.withEntity)
/** Returns a copy of this message with the entity and headers set to the given ones. */

View file

@ -13,7 +13,7 @@ import scala.concurrent.duration.FiniteDuration
import scala.concurrent.{ Future, ExecutionContext }
import scala.collection.immutable
import scala.util.{ Failure, Success, Try }
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.io.SynchronousFileSource
import akka.stream.scaladsl.{ FlattenStrategy, Source }
import akka.http.scaladsl.util.FastFuture
@ -30,7 +30,7 @@ trait Multipart {
* The given ``timeout`` denotes the max time that an individual part must be read in.
* The Future is failed with an TimeoutException if one part isn't read completely after the given timeout.
*/
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Multipart.Strict]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: Materializer): Future[Multipart.Strict]
/**
* Creates an entity from this multipart object.
@ -70,7 +70,7 @@ object Multipart {
def dispositionType: Option[ContentDispositionType] =
contentDispositionHeader.map(_.dispositionType)
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[BodyPart.Strict]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: Materializer): Future[BodyPart.Strict]
}
object BodyPart {
@ -79,7 +79,7 @@ object Multipart {
}
}
private def strictify[BP <: Multipart.BodyPart, BPS <: Multipart.BodyPart.Strict](parts: Source[BP, Any])(f: BP Future[BPS])(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Vector[BPS]] =
private def strictify[BP <: Multipart.BodyPart, BPS <: Multipart.BodyPart.Strict](parts: Source[BP, Any])(f: BP Future[BPS])(implicit ec: ExecutionContext, fm: Materializer): Future[Vector[BPS]] =
// TODO: move to Vector `:+` when https://issues.scala-lang.org/browse/SI-8930 is fixed
parts.runFold(new VectorBuilder[Future[BPS]]) {
case (builder, part) builder += f(part)
@ -93,7 +93,7 @@ object Multipart {
sealed abstract class General extends Multipart {
def mediaType: MultipartMediaType
def parts: Source[General.BodyPart, Any]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[General.Strict] =
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: Materializer): Future[General.Strict] =
strictify(parts)(_.toStrict(timeout)).fast.map(General.Strict(mediaType, _))
}
object General {
@ -113,7 +113,7 @@ object Multipart {
*/
case class Strict(mediaType: MultipartMediaType, strictParts: immutable.Seq[BodyPart.Strict]) extends General with Multipart.Strict {
def parts: Source[BodyPart.Strict, Any] = Source(strictParts)
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) =
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: Materializer) =
FastFuture.successful(this)
override def productPrefix = "General.Strict"
}
@ -122,7 +122,7 @@ object Multipart {
* Body part of the [[General]] model.
*/
sealed abstract class BodyPart extends Multipart.BodyPart {
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[BodyPart.Strict] =
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: Materializer): Future[BodyPart.Strict] =
entity.toStrict(timeout).map(BodyPart.Strict(_, headers))
def toFormDataBodyPart: Try[FormData.BodyPart]
def toByteRangesBodyPart: Try[ByteRanges.BodyPart]
@ -156,7 +156,7 @@ object Multipart {
* Strict [[General.BodyPart]].
*/
case class Strict(entity: HttpEntity.Strict, headers: immutable.Seq[HttpHeader] = Nil) extends BodyPart with Multipart.BodyPart.Strict {
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Strict] =
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: Materializer): Future[Strict] =
FastFuture.successful(this)
override def toFormDataBodyPart: Try[FormData.BodyPart.Strict] = tryCreateFormDataBodyPart(FormData.BodyPart.Strict(_, entity, _, _))
override def toByteRangesBodyPart: Try[ByteRanges.BodyPart.Strict] = tryCreateByteRangesBodyPart(ByteRanges.BodyPart.Strict(_, entity, _, _))
@ -172,7 +172,7 @@ object Multipart {
sealed abstract class FormData extends Multipart {
def mediaType = MediaTypes.`multipart/form-data`
def parts: Source[FormData.BodyPart, Any]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[FormData.Strict] =
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: Materializer): Future[FormData.Strict] =
strictify(parts)(_.toStrict(timeout)).fast.map(FormData.Strict(_))
}
object FormData {
@ -202,7 +202,7 @@ object Multipart {
*/
case class Strict(strictParts: immutable.Seq[BodyPart.Strict]) extends FormData with Multipart.Strict {
def parts: Source[BodyPart.Strict, Any] = Source(strictParts)
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) =
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: Materializer) =
FastFuture.successful(this)
override def productPrefix = "FormData.Strict"
}
@ -219,7 +219,7 @@ object Multipart {
override def dispositionParams = additionalDispositionParams.updated("name", name)
override def dispositionType = Some(ContentDispositionTypes.`form-data`)
def filename: Option[String] = additionalDispositionParams.get("filename")
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[BodyPart.Strict] =
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: Materializer): Future[BodyPart.Strict] =
entity.toStrict(timeout).map(BodyPart.Strict(name, _, additionalDispositionParams, additionalHeaders))
}
object BodyPart {
@ -249,7 +249,7 @@ object Multipart {
case class Strict(name: String, entity: HttpEntity.Strict,
additionalDispositionParams: Map[String, String] = Map.empty,
additionalHeaders: immutable.Seq[HttpHeader] = Nil) extends BodyPart with Multipart.BodyPart.Strict {
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Strict] =
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: Materializer): Future[Strict] =
FastFuture.successful(this)
override def productPrefix = "FormData.BodyPart.Strict"
}
@ -263,7 +263,7 @@ object Multipart {
sealed abstract class ByteRanges extends Multipart {
def mediaType = MediaTypes.`multipart/byteranges`
def parts: Source[ByteRanges.BodyPart, Any]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[ByteRanges.Strict] =
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: Materializer): Future[ByteRanges.Strict] =
strictify(parts)(_.toStrict(timeout)).fast.map(ByteRanges.Strict(_))
}
object ByteRanges {
@ -280,7 +280,7 @@ object Multipart {
*/
case class Strict(strictParts: immutable.Seq[BodyPart.Strict]) extends ByteRanges with Multipart.Strict {
def parts: Source[BodyPart.Strict, Any] = Source(strictParts)
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) =
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: Materializer) =
FastFuture.successful(this)
override def productPrefix = "ByteRanges.Strict"
}
@ -294,7 +294,7 @@ object Multipart {
def additionalHeaders: immutable.Seq[HttpHeader]
override def headers = contentRangeHeader +: additionalHeaders
def contentRangeHeader = `Content-Range`(rangeUnit, contentRange)
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[BodyPart.Strict] =
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: Materializer): Future[BodyPart.Strict] =
entity.toStrict(timeout).map(BodyPart.Strict(contentRange, _, rangeUnit, additionalHeaders))
}
object BodyPart {
@ -316,7 +316,7 @@ object Multipart {
*/
case class Strict(contentRange: ContentRange, entity: HttpEntity.Strict, rangeUnit: RangeUnit = RangeUnits.Bytes,
additionalHeaders: immutable.Seq[HttpHeader] = Nil) extends BodyPart with Multipart.BodyPart.Strict {
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Strict] =
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: Materializer): Future[Strict] =
FastFuture.successful(this)
override def productPrefix = "ByteRanges.BodyPart.Strict"
}

View file

@ -12,8 +12,8 @@ import akka.http.javadsl.model.ws.Message;
import akka.http.javadsl.model.ws.TextMessage;
import akka.http.javadsl.model.ws.Websocket;
import akka.japi.JavaPartialFunction;
import akka.stream.ActorFlowMaterializer;
import akka.stream.FlowMaterializer;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
import akka.stream.javadsl.Flow;
import akka.stream.javadsl.Source;
import scala.concurrent.Await;
@ -30,7 +30,7 @@ public class JavaTestServer {
ActorSystem system = ActorSystem.create();
try {
final FlowMaterializer materializer = ActorFlowMaterializer.create(system);
final Materializer materializer = ActorMaterializer.create(system);
Future<ServerBinding> serverBindingFuture =
Http.get(system).bindAndHandleSync(

View file

@ -17,7 +17,7 @@ import akka.http.scaladsl.{ TestUtils, Http }
import akka.http.impl.util.{ SingletonException, StreamUtils }
import akka.http.{ ClientConnectionSettings, ConnectionPoolSettings, ServerSettings }
import akka.stream.io.{ SessionBytes, SendBytes, SslTlsOutbound }
import akka.stream.{ BidiShape, ActorFlowMaterializer }
import akka.stream.{ BidiShape, ActorMaterializer }
import akka.stream.testkit.{ TestPublisher, TestSubscriber, AkkaSpec }
import akka.stream.scaladsl._
import akka.http.scaladsl.model.headers._
@ -28,7 +28,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
akka.loglevel = OFF
akka.io.tcp.trace-logging = off
akka.io.tcp.windows-connection-abort-workaround-enabled=auto""") {
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
// FIXME: Extract into proper util class to be reusable
lazy val ConnectionResetByPeerMessage: String = {

View file

@ -8,12 +8,12 @@ import scala.concurrent.Await
import scala.concurrent.duration._
import akka.http.scaladsl.{ Http, TestUtils }
import akka.http.scaladsl.model._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
class HighLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF") {
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
"The connection-level client implementation" should {

View file

@ -10,7 +10,7 @@ import akka.stream.io.{ SessionBytes, SslTlsOutbound, SendBytes }
import org.scalatest.Inside
import akka.util.ByteString
import akka.event.NoLogging
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.testkit._
import akka.stream.scaladsl._
import akka.http.scaladsl.model.HttpEntity._
@ -20,7 +20,7 @@ import akka.http.scaladsl.model.headers._
import akka.http.impl.util._
class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF") with Inside {
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
"The connection-level client implementation" should {

View file

@ -17,7 +17,7 @@ import akka.http.scaladsl.model._
import akka.http.scaladsl.model.headers._
import akka.http.scaladsl.util.FastFuture
import akka.http.scaladsl.util.FastFuture._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.{ FlattenStrategy, _ }
import akka.util.ByteString
import com.typesafe.config.{ Config, ConfigFactory }
@ -38,7 +38,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
import system.dispatcher
val BOLT = HttpMethod.custom("BOLT", safe = false, idempotent = true, entityAccepted = true)
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
"The request parsing logic should" - {
"properly parse a request" - {

View file

@ -15,7 +15,7 @@ import akka.util.ByteString
import akka.actor.ActorSystem
import akka.stream.scaladsl._
import akka.stream.scaladsl.FlattenStrategy
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.util.FastFuture._
import akka.http.impl.util._
import akka.http.scaladsl.model._
@ -35,7 +35,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
import system.dispatcher
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
val ServerOnTheMove = StatusCodes.custom(331, "Server on the move")
"The response parsing logic should" - {

View file

@ -16,7 +16,7 @@ import akka.http.scaladsl.model._
import akka.http.scaladsl.model.headers._
import akka.http.impl.util._
import akka.stream.scaladsl._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import HttpEntity._
import HttpMethods._
@ -27,7 +27,7 @@ class RequestRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
import system.dispatcher
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
"The request preparation logic should" - {
"properly render an unchunked" - {

View file

@ -16,7 +16,7 @@ import akka.http.scaladsl.model.headers._
import akka.http.impl.util._
import akka.util.ByteString
import akka.stream.scaladsl._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import HttpEntity._
class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
@ -27,7 +27,7 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll
import system.dispatcher
val ServerOnTheMove = StatusCodes.custom(330, "Server on the move")
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
"The response preparation logic should properly render" - {
"a response with no body," - {

View file

@ -14,7 +14,7 @@ import scala.concurrent.duration._
import org.scalatest.Inside
import akka.util.ByteString
import akka.stream.scaladsl._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.testkit._
import akka.http.scaladsl.model._
import akka.http.impl.util._
@ -24,7 +24,7 @@ import MediaTypes._
import HttpMethods._
class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF") with Inside { spec
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
"The server implementation" should {

View file

@ -14,7 +14,7 @@ import akka.actor.ActorSystem
import akka.event.NoLogging
import akka.util.ByteString
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.scaladsl.{ Flow, Sink, Source, FlowGraph }
import akka.stream.testkit.{ TestPublisher, TestSubscriber }
@ -26,7 +26,7 @@ import akka.http.scaladsl.model.{ HttpResponse, HttpRequest }
abstract class HttpServerTestSetupBase {
implicit def system: ActorSystem
implicit def materializer: FlowMaterializer
implicit def materializer: Materializer
val requests = TestSubscriber.manualProbe[HttpRequest]
val responses = TestPublisher.manualProbe[HttpResponse]()

View file

@ -7,7 +7,7 @@ package akka.http.impl.engine.ws
import com.typesafe.config.{ ConfigFactory, Config }
import org.scalatest.{ Suite, BeforeAndAfterAll }
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
trait WithMaterializerSpec extends BeforeAndAfterAll { _: Suite
lazy val testConf: Config = ConfigFactory.parseString("""
@ -15,6 +15,6 @@ trait WithMaterializerSpec extends BeforeAndAfterAll { _: Suite ⇒
akka.loglevel = WARNING""")
implicit lazy val system = ActorSystem(getClass.getSimpleName, testConf)
implicit lazy val materializer = ActorFlowMaterializer()
implicit lazy val materializer = ActorMaterializer()
override def afterAll() = system.shutdown()
}

View file

@ -15,7 +15,7 @@ import scala.concurrent.duration._
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
import akka.actor.ActorSystem
import akka.testkit.EventFilter
import akka.stream.{ StreamTcpException, ActorFlowMaterializer, BindFailedException }
import akka.stream.{ StreamTcpException, ActorMaterializer, BindFailedException }
import akka.stream.scaladsl._
import akka.stream.testkit._
import akka.http.scaladsl.model.HttpEntity._
@ -36,13 +36,13 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
""")
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
import system.dispatcher
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
val testConf2: Config =
ConfigFactory.parseString("akka.stream.materializer.subscription-timeout.timeout = 1 s")
.withFallback(testConf)
val system2 = ActorSystem(getClass.getSimpleName, testConf2)
val materializer2 = ActorFlowMaterializer.create(system2)
val materializer2 = ActorMaterializer.create(system2)
"The low-level HTTP infrastructure" should {

View file

@ -7,7 +7,7 @@ package akka.http.scaladsl
import com.typesafe.config.{ Config, ConfigFactory }
import scala.util.{ Failure, Success }
import akka.actor.{ UnhandledMessage, ActorSystem }
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.{ Sink, Source }
import akka.http.scaladsl.model._
import akka.http.impl.util._
@ -18,7 +18,7 @@ object TestClient extends App {
akka.log-dead-letters = off
akka.io.tcp.trace-logging = off""")
implicit val system = ActorSystem("ServerTest", testConf)
implicit val fm = ActorFlowMaterializer()
implicit val fm = ActorMaterializer()
import system.dispatcher
installEventStreamLoggerFor[UnhandledMessage]

View file

@ -9,7 +9,7 @@ import scala.concurrent.Await
import akka.actor.ActorSystem
import akka.http.scaladsl.model._
import akka.http.scaladsl.model.ws._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.{ Source, Flow }
import com.typesafe.config.{ ConfigFactory, Config }
import HttpMethods._
@ -20,7 +20,7 @@ object TestServer extends App {
akka.log-dead-letters = off
""")
implicit val system = ActorSystem("ServerTest", testConf)
implicit val fm = ActorFlowMaterializer()
implicit val fm = ActorMaterializer()
try {
val binding = Http().bindAndHandleSync({

View file

@ -13,7 +13,7 @@ import org.scalatest.matchers.Matcher
import akka.util.ByteString
import akka.actor.ActorSystem
import akka.stream.scaladsl._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.model.HttpEntity._
import akka.http.impl.util.StreamUtils
@ -30,7 +30,7 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
import system.dispatcher
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
override def afterAll() = system.shutdown()
"HttpEntity" - {

View file

@ -8,7 +8,7 @@ import com.typesafe.config.{ Config, ConfigFactory }
import scala.concurrent.Await
import scala.concurrent.duration._
import org.scalatest.{ BeforeAndAfterAll, Inside, Matchers, WordSpec }
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.Source
import akka.util.ByteString
import akka.actor.ActorSystem
@ -21,7 +21,7 @@ class MultipartSpec extends WordSpec with Matchers with Inside with BeforeAndAft
akka.loglevel = WARNING""")
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
import system.dispatcher
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
override def afterAll() = system.shutdown()
"Multipart.General" should {

View file

@ -11,7 +11,7 @@ import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
import akka.util.ByteString
import akka.actor.ActorSystem
import akka.http.scaladsl.model._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl._
import headers._
@ -42,7 +42,7 @@ class HttpModelIntegrationSpec extends WordSpec with Matchers with BeforeAndAfte
override def afterAll() = system.shutdown()
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
"External HTTP libraries" should {

View file

@ -18,9 +18,9 @@ object Jackson {
def json[T <: AnyRef](objectMapper: ObjectMapper): Marshaller[T] = jsonMarshaller(objectMapper).asInstanceOf[Marshaller[T]]
def jsonAs[T](clazz: Class[T]): Unmarshaller[T] = jsonAs(objectMapper, clazz)
def jsonAs[T](objectMapper: ObjectMapper, clazz: Class[T]): Unmarshaller[T] =
UnmarshallerImpl[T] { (_ec, _flowMaterializer)
UnmarshallerImpl[T] { (_ec, _materializer)
implicit val ec = _ec
implicit val mat = _flowMaterializer
implicit val mat = _materializer
unmarshalling.Unmarshaller.messageUnmarshallerFromEntityUnmarshaller { // isn't implicitly inferred for unknown reasons
unmarshalling.Unmarshaller.stringUnmarshaller

View file

@ -5,7 +5,7 @@
package akka.http.scaladsl.marshallers.sprayjson
import scala.language.implicitConversions
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.http.scaladsl.marshalling.{ ToEntityMarshaller, Marshaller }
import akka.http.scaladsl.unmarshalling.{ FromEntityUnmarshaller, Unmarshaller }
import akka.http.scaladsl.model.{ ContentTypes, HttpCharsets }
@ -16,11 +16,11 @@ import spray.json._
* A trait providing automatic to and from JSON marshalling/unmarshalling using an in-scope *spray-json* protocol.
*/
trait SprayJsonSupport {
implicit def sprayJsonUnmarshallerConverter[T](reader: RootJsonReader[T])(implicit mat: FlowMaterializer): FromEntityUnmarshaller[T] =
implicit def sprayJsonUnmarshallerConverter[T](reader: RootJsonReader[T])(implicit mat: Materializer): FromEntityUnmarshaller[T] =
sprayJsonUnmarshaller(reader, mat)
implicit def sprayJsonUnmarshaller[T](implicit reader: RootJsonReader[T], mat: FlowMaterializer): FromEntityUnmarshaller[T] =
implicit def sprayJsonUnmarshaller[T](implicit reader: RootJsonReader[T], mat: Materializer): FromEntityUnmarshaller[T] =
sprayJsValueUnmarshaller.map(jsonReader[T].read)
implicit def sprayJsValueUnmarshaller(implicit mat: FlowMaterializer): FromEntityUnmarshaller[JsValue] =
implicit def sprayJsValueUnmarshaller(implicit mat: Materializer): FromEntityUnmarshaller[JsValue] =
Unmarshaller.byteStringUnmarshaller.forContentTypes(`application/json`).mapWithCharset { (data, charset)
val input =
if (charset == HttpCharsets.`UTF-8`) ParserInput(data.toArray)

View file

@ -8,7 +8,7 @@ import java.io.{ ByteArrayInputStream, InputStreamReader }
import javax.xml.parsers.{ SAXParserFactory, SAXParser }
import scala.collection.immutable
import scala.xml.{ XML, NodeSeq }
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.http.scaladsl.unmarshalling._
import akka.http.scaladsl.marshalling._
import akka.http.scaladsl.model._
@ -21,10 +21,10 @@ trait ScalaXmlSupport {
def nodeSeqMarshaller(contentType: ContentType): ToEntityMarshaller[NodeSeq] =
Marshaller.StringMarshaller.wrap(contentType)(_.toString())
implicit def defaultNodeSeqUnmarshaller(implicit fm: FlowMaterializer): FromEntityUnmarshaller[NodeSeq] =
implicit def defaultNodeSeqUnmarshaller(implicit fm: Materializer): FromEntityUnmarshaller[NodeSeq] =
nodeSeqUnmarshaller(ScalaXmlSupport.nodeSeqContentTypeRanges: _*)
def nodeSeqUnmarshaller(ranges: ContentTypeRange*)(implicit fm: FlowMaterializer): FromEntityUnmarshaller[NodeSeq] =
def nodeSeqUnmarshaller(ranges: ContentTypeRange*)(implicit fm: Materializer): FromEntityUnmarshaller[NodeSeq] =
Unmarshaller.byteArrayUnmarshaller.forContentTypes(ranges: _*).mapWithCharset { (bytes, charset)
if (bytes.length > 0) {
val reader = new InputStreamReader(new ByteArrayInputStream(bytes), charset.nioCharset)

View file

@ -8,7 +8,7 @@ import org.junit.rules.ExternalResource
import org.junit.{ Rule, Assert }
import scala.concurrent.duration._
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.model.HttpResponse
/**
@ -17,7 +17,7 @@ import akka.http.scaladsl.model.HttpResponse
abstract class JUnitRouteTestBase extends RouteTest {
protected def systemResource: ActorSystemResource
implicit def system: ActorSystem = systemResource.system
implicit def materializer: ActorFlowMaterializer = systemResource.materializer
implicit def materializer: ActorMaterializer = systemResource.materializer
protected def createTestResponse(response: HttpResponse): TestResponse =
new TestResponse(response, awaitDuration)(system.dispatcher, materializer) {
@ -44,18 +44,18 @@ abstract class JUnitRouteTest extends JUnitRouteTestBase {
class ActorSystemResource extends ExternalResource {
protected def createSystem(): ActorSystem = ActorSystem()
protected def createFlowMaterializer(system: ActorSystem): ActorFlowMaterializer = ActorFlowMaterializer()(system)
protected def createMaterializer(system: ActorSystem): ActorMaterializer = ActorMaterializer()(system)
implicit def system: ActorSystem = _system
implicit def materializer: ActorFlowMaterializer = _materializer
implicit def materializer: ActorMaterializer = _materializer
private[this] var _system: ActorSystem = null
private[this] var _materializer: ActorFlowMaterializer = null
private[this] var _materializer: ActorMaterializer = null
override def before(): Unit = {
require((_system eq null) && (_materializer eq null))
_system = createSystem()
_materializer = createFlowMaterializer(_system)
_materializer = createMaterializer(_system)
}
override def after(): Unit = {
_system.shutdown()

View file

@ -7,7 +7,7 @@ package akka.http.javadsl.testkit
import scala.annotation.varargs
import scala.concurrent.ExecutionContext
import scala.concurrent.duration._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.server
import akka.http.javadsl.model.HttpRequest
import akka.http.javadsl.server.{ Route, Directives }
@ -21,7 +21,7 @@ import akka.http.impl.util._
abstract class RouteTest {
implicit def system: ActorSystem
implicit def materializer: ActorFlowMaterializer
implicit def materializer: ActorMaterializer
implicit def executionContext: ExecutionContext = system.dispatcher
protected def awaitDuration: FiniteDuration = 500.millis

View file

@ -8,7 +8,7 @@ import scala.reflect.ClassTag
import scala.concurrent.ExecutionContext
import scala.concurrent.duration.FiniteDuration
import akka.util.ByteString
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.unmarshalling.Unmarshal
import akka.http.scaladsl.model.HttpResponse
import akka.http.impl.util._
@ -20,7 +20,7 @@ import akka.http.javadsl.model._
/**
* A wrapper for responses
*/
abstract class TestResponse(_response: HttpResponse, awaitAtMost: FiniteDuration)(implicit ec: ExecutionContext, materializer: ActorFlowMaterializer) {
abstract class TestResponse(_response: HttpResponse, awaitAtMost: FiniteDuration)(implicit ec: ExecutionContext, materializer: ActorMaterializer) {
lazy val entity: HttpEntityStrict =
_response.entity.toStrict(awaitAtMost).awaitResult(awaitAtMost)
lazy val response: HttpResponse = _response.withEntity(entity)

View file

@ -9,18 +9,18 @@ import scala.concurrent.{ ExecutionContext, Await }
import akka.http.scaladsl.unmarshalling.{ Unmarshal, FromEntityUnmarshaller }
import akka.http.scaladsl.marshalling._
import akka.http.scaladsl.model.HttpEntity
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import scala.util.Try
trait MarshallingTestUtils {
def marshal[T: ToEntityMarshaller](value: T)(implicit ec: ExecutionContext, mat: FlowMaterializer): HttpEntity.Strict =
def marshal[T: ToEntityMarshaller](value: T)(implicit ec: ExecutionContext, mat: Materializer): HttpEntity.Strict =
Await.result(Marshal(value).to[HttpEntity].flatMap(_.toStrict(1.second)), 1.second)
def unmarshalValue[T: FromEntityUnmarshaller](entity: HttpEntity)(implicit ec: ExecutionContext, mat: FlowMaterializer): T =
def unmarshalValue[T: FromEntityUnmarshaller](entity: HttpEntity)(implicit ec: ExecutionContext, mat: Materializer): T =
unmarshal(entity).get
def unmarshal[T: FromEntityUnmarshaller](entity: HttpEntity)(implicit ec: ExecutionContext, mat: FlowMaterializer): Try[T] = {
def unmarshal[T: FromEntityUnmarshaller](entity: HttpEntity)(implicit ec: ExecutionContext, mat: Materializer): Try[T] = {
val fut = Unmarshal(entity).to[T]
Await.ready(fut, 1.second)
fut.value.get

View file

@ -11,7 +11,7 @@ import scala.concurrent.duration._
import scala.util.DynamicVariable
import scala.reflect.ClassTag
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.client.RequestBuilding
import akka.http.scaladsl.util.FastFuture
import akka.http.scaladsl.server._
@ -41,7 +41,7 @@ trait RouteTest extends RequestBuilding with RouteTestResultComponent with Marsh
}
implicit val system = createActorSystem()
implicit def executor = system.dispatcher
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
def cleanUp(): Unit = system.shutdown()

View file

@ -8,7 +8,7 @@ import java.util.concurrent.CountDownLatch
import scala.collection.immutable
import scala.concurrent.duration._
import scala.concurrent.ExecutionContext
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.scaladsl._
import akka.http.scaladsl.model.HttpEntity.ChunkStreamPart
import akka.http.scaladsl.server._
@ -22,7 +22,7 @@ trait RouteTestResultComponent {
/**
* A receptacle for the response or rejections created by a route.
*/
class RouteTestResult(timeout: FiniteDuration)(implicit fm: FlowMaterializer) {
class RouteTestResult(timeout: FiniteDuration)(implicit fm: Materializer) {
private[this] var result: Option[Either[immutable.Seq[Rejection], HttpResponse]] = None
private[this] val latch = new CountDownLatch(1)

View file

@ -9,7 +9,7 @@ import scala.concurrent.{ ExecutionContext, Future, Await }
import scala.concurrent.duration._
import org.scalatest.Suite
import org.scalatest.matchers.Matcher
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.http.scaladsl.model.HttpEntity
import akka.http.scaladsl.unmarshalling.FromEntityUnmarshaller
@ -21,10 +21,10 @@ trait ScalatestUtils extends MarshallingTestUtils {
def haveFailedWith(t: Throwable): Matcher[Future[_]] =
equal(t).matcher[Throwable] compose (x Await.result(x.failed, 1.second))
def unmarshalToValue[T: FromEntityUnmarshaller](value: T)(implicit ec: ExecutionContext, mat: FlowMaterializer): Matcher[HttpEntity] =
def unmarshalToValue[T: FromEntityUnmarshaller](value: T)(implicit ec: ExecutionContext, mat: Materializer): Matcher[HttpEntity] =
equal(value).matcher[T] compose (unmarshalValue(_))
def unmarshalTo[T: FromEntityUnmarshaller](value: Try[T])(implicit ec: ExecutionContext, mat: FlowMaterializer): Matcher[HttpEntity] =
def unmarshalTo[T: FromEntityUnmarshaller](value: Try[T])(implicit ec: ExecutionContext, mat: Materializer): Matcher[HttpEntity] =
equal(value).matcher[Try[T]] compose (unmarshal(_))
}

View file

@ -11,7 +11,7 @@ import akka.http.javadsl.model.HttpRequest;
import akka.http.javadsl.model.headers.AcceptEncoding;
import akka.http.javadsl.model.headers.ContentEncoding;
import akka.http.javadsl.model.headers.HttpEncodings;
import akka.stream.ActorFlowMaterializer;
import akka.stream.ActorMaterializer;
import akka.util.ByteString;
import org.junit.*;
import scala.concurrent.Await;
@ -35,7 +35,7 @@ public class CodingDirectivesTest extends JUnitRouteTest {
system = null;
}
final ActorFlowMaterializer mat = ActorFlowMaterializer.create(system);
final ActorMaterializer mat = ActorMaterializer.create(system);
@Test
public void testAutomaticEncodingWhenNoEncodingRequested() throws Exception {

View file

@ -8,14 +8,14 @@ import scala.concurrent.duration._
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
import org.scalatest.concurrent.ScalaFutures
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.unmarshalling.Unmarshal
import akka.http.scaladsl.marshalling.Marshal
import akka.http.scaladsl.model._
class FormDataSpec extends WordSpec with Matchers with ScalaFutures with BeforeAndAfterAll {
implicit val system = ActorSystem(getClass.getSimpleName)
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
import system.dispatcher
val formData = FormData(Map("surname" -> "Smith", "age" -> "42"))

View file

@ -7,7 +7,7 @@ package akka.http.scaladsl.coding
import scala.concurrent.duration._
import org.scalatest.{ Suite, BeforeAndAfterAll, Matchers }
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.util.ByteString
trait CodecSpecSupport extends Matchers with BeforeAndAfterAll { self: Suite
@ -68,7 +68,7 @@ voluptua. At vero eos et accusam et justo duo dolores et ea rebum. Stet clita ka
est Lorem ipsum dolor sit amet. Lorem ipsum dolor sit amet, consetetur sadipscing elitr, sed diam nonumy e""".replace("\r\n", "\n")
implicit val system = ActorSystem(getClass.getSimpleName)
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
override def afterAll() = {
system.shutdown()

View file

@ -10,7 +10,7 @@ import akka.http.scaladsl.marshallers.xml.ScalaXmlSupport._
import scala.collection.immutable.ListMap
import org.scalatest.{ BeforeAndAfterAll, FreeSpec, Matchers }
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.Source
import akka.http.impl.util._
import akka.http.scaladsl.model._
@ -20,7 +20,7 @@ import MediaTypes._
class MarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll with MultipartMarshallers with MarshallingTestUtils {
implicit val system = ActorSystem(getClass.getSimpleName)
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
import system.dispatcher
"The PredefinedToEntityMarshallers." - {

View file

@ -8,7 +8,7 @@ import akka.http.scaladsl.marshallers.xml.ScalaXmlSupport
import akka.http.scaladsl.server.directives.UserCredentials
import com.typesafe.config.{ ConfigFactory, Config }
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.Http
object TestServer extends App {
@ -17,7 +17,7 @@ object TestServer extends App {
akka.log-dead-letters = off""")
implicit val system = ActorSystem("ServerTest", testConf)
import system.dispatcher
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
import ScalaXmlSupport._
import Directives._

View file

@ -10,7 +10,7 @@ import akka.http.scaladsl.model.headers.{ UpgradeProtocol, Upgrade }
import akka.http.scaladsl.model.{ HttpRequest, StatusCodes, HttpResponse }
import akka.http.scaladsl.model.ws.{ Message, UpgradeToWebsocket }
import akka.http.scaladsl.server.{ Route, RoutingSpec }
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.scaladsl.Flow
class WebsocketDirectivesSpec extends RoutingSpec {

View file

@ -11,7 +11,7 @@ import org.scalatest.{ BeforeAndAfterAll, FreeSpec, Matchers }
import akka.http.scaladsl.testkit.ScalatestUtils
import akka.util.ByteString
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.stream.scaladsl._
import akka.http.scaladsl.model._
import akka.http.scaladsl.util.FastFuture._
@ -21,7 +21,7 @@ import MediaTypes._
class MultipartUnmarshallersSpec extends FreeSpec with Matchers with BeforeAndAfterAll with ScalatestUtils {
implicit val system = ActorSystem(getClass.getSimpleName)
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
import system.dispatcher
"The MultipartUnmarshallers." - {

View file

@ -7,12 +7,12 @@ package akka.http.scaladsl.unmarshalling
import org.scalatest.{ BeforeAndAfterAll, FreeSpec, Matchers }
import akka.http.scaladsl.testkit.ScalatestUtils
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorMaterializer
import akka.http.scaladsl.model._
class UnmarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll with ScalatestUtils {
implicit val system = ActorSystem(getClass.getSimpleName)
implicit val materializer = ActorFlowMaterializer()
implicit val materializer = ActorMaterializer()
import system.dispatcher
"The PredefinedFromEntityUnmarshallers." - {

View file

@ -7,7 +7,7 @@ package akka.http.javadsl.server;
import akka.http.scaladsl.coding.Deflate$;
import akka.http.scaladsl.coding.Gzip$;
import akka.http.scaladsl.coding.NoCoding$;
import akka.stream.FlowMaterializer;
import akka.stream.Materializer;
import akka.util.ByteString;
import scala.concurrent.Future;
@ -26,7 +26,7 @@ public enum Coder {
public ByteString encode(ByteString input) {
return underlying.encode(input);
}
public Future<ByteString> decode(ByteString input, FlowMaterializer mat) {
public Future<ByteString> decode(ByteString input, Materializer mat) {
return underlying.decode(input, mat);
}
public akka.http.scaladsl.coding.Coder _underlyingScalaCoder() {

View file

@ -6,7 +6,7 @@ package akka.http.impl.server
import scala.concurrent.ExecutionContext
import scala.reflect.ClassTag
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.http.javadsl.server.Unmarshaller
import akka.http.scaladsl.unmarshalling.FromMessageUnmarshaller
@ -15,5 +15,5 @@ import akka.http.scaladsl.unmarshalling.FromMessageUnmarshaller
*
*/
// FIXME: too lenient visibility, currently used to implement Java marshallers, needs proper API, see #16439
case class UnmarshallerImpl[T](scalaUnmarshaller: (ExecutionContext, FlowMaterializer) FromMessageUnmarshaller[T])(implicit val classTag: ClassTag[T])
case class UnmarshallerImpl[T](scalaUnmarshaller: (ExecutionContext, Materializer) FromMessageUnmarshaller[T])(implicit val classTag: ClassTag[T])
extends Unmarshaller[T]

View file

@ -9,7 +9,7 @@ import akka.actor.ActorSystem
import akka.http.scaladsl.{ server, Http }
import akka.http.scaladsl.Http.ServerBinding
import akka.http.impl.server.RouteImplementation
import akka.stream.{ ActorFlowMaterializer, FlowMaterializer }
import akka.stream.{ ActorMaterializer, Materializer }
import akka.stream.scaladsl.{ Keep, Sink }
trait HttpServiceBase {
@ -18,26 +18,26 @@ trait HttpServiceBase {
*/
def bindRoute(interface: String, port: Int, route: Route, system: ActorSystem): Future[ServerBinding] = {
implicit val sys = system
implicit val mat = ActorFlowMaterializer()
implicit val mat = ActorMaterializer()
handleConnectionsWithRoute(interface, port, route, system, mat)
}
/**
* Starts a server on the given interface and port and uses the route to handle incoming requests.
*/
def bindRoute(interface: String, port: Int, route: Route, system: ActorSystem, flowMaterializer: FlowMaterializer): Future[ServerBinding] =
handleConnectionsWithRoute(interface, port, route, system, flowMaterializer)
def bindRoute(interface: String, port: Int, route: Route, system: ActorSystem, materializer: Materializer): Future[ServerBinding] =
handleConnectionsWithRoute(interface, port, route, system, materializer)
/**
* Uses the route to handle incoming connections and requests for the ServerBinding.
*/
def handleConnectionsWithRoute(interface: String, port: Int, route: Route, system: ActorSystem, flowMaterializer: FlowMaterializer): Future[ServerBinding] = {
def handleConnectionsWithRoute(interface: String, port: Int, route: Route, system: ActorSystem, materializer: Materializer): Future[ServerBinding] = {
implicit val sys = system
implicit val mat = flowMaterializer
implicit val mat = materializer
import system.dispatcher
val r: server.Route = RouteImplementation(route)
Http(system).bind(interface, port).toMat(Sink.foreach(_.handleWith(r)))(Keep.left).run()(flowMaterializer)
Http(system).bind(interface, port).toMat(Sink.foreach(_.handleWith(r)))(Keep.left).run()(materializer)
}
}

View file

@ -25,7 +25,7 @@ object RequestVals {
def entityAs[T](unmarshaller: Unmarshaller[T]): RequestVal[T] =
new ExtractingStandaloneExtractionImpl[T]()(unmarshaller.classTag) {
def extract(ctx: server.RequestContext): Future[T] = {
val u = unmarshaller.asInstanceOf[UnmarshallerImpl[T]].scalaUnmarshaller(ctx.executionContext, ctx.flowMaterializer)
val u = unmarshaller.asInstanceOf[UnmarshallerImpl[T]].scalaUnmarshaller(ctx.executionContext, ctx.materializer)
u(ctx.request)(ctx.executionContext)
}
}

View file

@ -5,7 +5,7 @@
package akka.http.scaladsl.coding
import akka.http.scaladsl.model._
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.stream.stage.Stage
import akka.util.ByteString
import headers.HttpEncoding
@ -27,7 +27,7 @@ trait Decoder {
def withMaxBytesPerChunk(maxBytesPerChunk: Int): Decoder
def decoderFlow: Flow[ByteString, ByteString, Unit]
def decode(input: ByteString)(implicit mat: FlowMaterializer): Future[ByteString] =
def decode(input: ByteString)(implicit mat: Materializer): Future[ByteString] =
Source.single(input).via(decoderFlow).runWith(Sink.head)
}
object Decoder {

View file

@ -8,7 +8,7 @@ import scala.annotation.implicitNotFound
import scala.collection.immutable
import scala.concurrent.{ ExecutionContext, Future }
import scala.concurrent.duration._
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.http.scaladsl.unmarshalling._
import akka.http.scaladsl.model._
import akka.http.scaladsl.util.FastFuture
@ -87,7 +87,7 @@ object StrictForm {
implicit def unmarshaller(implicit formDataUM: FromEntityUnmarshaller[FormData],
multipartUM: FromEntityUnmarshaller[Multipart.FormData],
fm: FlowMaterializer): FromEntityUnmarshaller[StrictForm] =
fm: Materializer): FromEntityUnmarshaller[StrictForm] =
Unmarshaller { implicit ec
entity

View file

@ -5,7 +5,7 @@
package akka.http.scaladsl.server
import scala.concurrent.{ Future, ExecutionContext }
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.event.LoggingAdapter
import akka.http.scaladsl.marshalling.ToResponseMarshallable
import akka.http.scaladsl.model._
@ -28,9 +28,9 @@ trait RequestContext {
implicit def executionContext: ExecutionContext
/**
* The default FlowMaterializer.
* The default Materializer.
*/
implicit def flowMaterializer: FlowMaterializer
implicit def materializer: Materializer
/**
* The default LoggingAdapter to be used for logging messages related to this request.
@ -47,7 +47,7 @@ trait RequestContext {
*/
def reconfigure(
executionContext: ExecutionContext = executionContext,
flowMaterializer: FlowMaterializer = flowMaterializer,
materializer: Materializer = materializer,
log: LoggingAdapter = log,
settings: RoutingSettings = settings): RequestContext
@ -81,7 +81,7 @@ trait RequestContext {
/**
* Returns a copy of this context with the new HttpRequest.
*/
def withFlowMaterializer(materializer: FlowMaterializer): RequestContext
def withMaterializer(materializer: Materializer): RequestContext
/**
* Returns a copy of this context with the new LoggingAdapter.

View file

@ -5,7 +5,7 @@
package akka.http.scaladsl.server
import scala.concurrent.{ Future, ExecutionContext }
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.event.LoggingAdapter
import akka.http.scaladsl.marshalling.{ Marshal, ToResponseMarshallable }
import akka.http.scaladsl.model._
@ -19,15 +19,15 @@ private[http] class RequestContextImpl(
val request: HttpRequest,
val unmatchedPath: Uri.Path,
val executionContext: ExecutionContext,
val flowMaterializer: FlowMaterializer,
val materializer: Materializer,
val log: LoggingAdapter,
val settings: RoutingSettings) extends RequestContext {
def this(request: HttpRequest, log: LoggingAdapter, settings: RoutingSettings)(implicit ec: ExecutionContext, materializer: FlowMaterializer) =
def this(request: HttpRequest, log: LoggingAdapter, settings: RoutingSettings)(implicit ec: ExecutionContext, materializer: Materializer) =
this(request, request.uri.path, ec, materializer, log, settings)
def reconfigure(executionContext: ExecutionContext, flowMaterializer: FlowMaterializer, log: LoggingAdapter, settings: RoutingSettings): RequestContext =
copy(executionContext = executionContext, flowMaterializer = flowMaterializer, log = log, settings = settings)
def reconfigure(executionContext: ExecutionContext, materializer: Materializer, log: LoggingAdapter, settings: RoutingSettings): RequestContext =
copy(executionContext = executionContext, materializer = materializer, log = log, settings = settings)
override def complete(trm: ToResponseMarshallable): Future[RouteResult] =
trm(request)(executionContext)
@ -50,8 +50,8 @@ private[http] class RequestContextImpl(
override def withExecutionContext(executionContext: ExecutionContext): RequestContext =
if (executionContext != this.executionContext) copy(executionContext = executionContext) else this
override def withFlowMaterializer(flowMaterializer: FlowMaterializer): RequestContext =
if (flowMaterializer != this.flowMaterializer) copy(flowMaterializer = flowMaterializer) else this
override def withMaterializer(materializer: Materializer): RequestContext =
if (materializer != this.materializer) copy(materializer = materializer) else this
override def withLog(log: LoggingAdapter): RequestContext =
if (log != this.log) copy(log = log) else this
@ -84,8 +84,8 @@ private[http] class RequestContextImpl(
private def copy(request: HttpRequest = request,
unmatchedPath: Uri.Path = unmatchedPath,
executionContext: ExecutionContext = executionContext,
flowMaterializer: FlowMaterializer = flowMaterializer,
materializer: Materializer = materializer,
log: LoggingAdapter = log,
settings: RoutingSettings = settings) =
new RequestContextImpl(request, unmatchedPath, executionContext, flowMaterializer, log, settings)
new RequestContextImpl(request, unmatchedPath, executionContext, materializer, log, settings)
}

View file

@ -7,7 +7,7 @@ package akka.http.scaladsl.server
import scala.concurrent.ExecutionContext
import akka.event.LoggingAdapter
import akka.actor.{ ActorSystem, ActorContext }
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.http.scaladsl.Http
import akka.http.scaladsl.model.HttpRequest
@ -34,12 +34,12 @@ class RoutingSetup(
val exceptionHandler: ExceptionHandler,
val rejectionHandler: RejectionHandler,
val executionContext: ExecutionContext,
val flowMaterializer: FlowMaterializer,
val materializer: Materializer,
val routingLog: RoutingLog) {
// enable `import setup._` to properly bring implicits in scope
implicit def executor: ExecutionContext = executionContext
implicit def materializer: FlowMaterializer = flowMaterializer
implicit def implicitMaterializer: Materializer = materializer
}
object RoutingSetup {
@ -47,14 +47,14 @@ object RoutingSetup {
exceptionHandler: ExceptionHandler = null,
rejectionHandler: RejectionHandler = null,
executionContext: ExecutionContext = null,
flowMaterializer: FlowMaterializer,
materializer: Materializer,
routingLog: RoutingLog): RoutingSetup =
new RoutingSetup(
routingSettings,
if (exceptionHandler ne null) exceptionHandler else ExceptionHandler.default(routingSettings),
if (rejectionHandler ne null) rejectionHandler else RejectionHandler.default,
if (executionContext ne null) executionContext else flowMaterializer.executionContext,
flowMaterializer,
if (executionContext ne null) executionContext else materializer.executionContext,
materializer,
routingLog)
}

View file

@ -8,7 +8,7 @@ package directives
import scala.concurrent.{ Future, ExecutionContext }
import scala.collection.immutable
import akka.event.LoggingAdapter
import akka.stream.FlowMaterializer
import akka.stream.Materializer
import akka.http.scaladsl.server.util.Tuple
import akka.http.scaladsl.util.FastFuture
import akka.http.scaladsl.model._
@ -141,15 +141,15 @@ trait BasicDirectives {
def extractExecutionContext: Directive1[ExecutionContext] = BasicDirectives._extractExecutionContext
/**
* Runs its inner route with the given alternative [[FlowMaterializer]].
* Runs its inner route with the given alternative [[Materializer]].
*/
def withFlowMaterializer(materializer: FlowMaterializer): Directive0 =
mapRequestContext(_ withFlowMaterializer materializer)
def withMaterializer(materializer: Materializer): Directive0 =
mapRequestContext(_ withMaterializer materializer)
/**
* Extracts the [[FlowMaterializer]] from the [[RequestContext]].
* Extracts the [[Materializer]] from the [[RequestContext]].
*/
def extractFlowMaterializer: Directive1[FlowMaterializer] = BasicDirectives._extractFlowMaterializer
def extractMaterializer: Directive1[Materializer] = BasicDirectives._extractMaterializer
/**
* Runs its inner route with the given alternative [[LoggingAdapter]].
@ -192,7 +192,7 @@ object BasicDirectives extends BasicDirectives {
private val _extractRequest: Directive1[HttpRequest] = extract(_.request)
private val _extractUri: Directive1[Uri] = extract(_.request.uri)
private val _extractExecutionContext: Directive1[ExecutionContext] = extract(_.executionContext)
private val _extractFlowMaterializer: Directive1[FlowMaterializer] = extract(_.flowMaterializer)
private val _extractMaterializer: Directive1[Materializer] = extract(_.materializer)
private val _extractLog: Directive1[LoggingAdapter] = extract(_.log)
private val _extractSettings: Directive1[RoutingSettings] = extract(_.settings)
private val _extractRequestContext: Directive1[RequestContext] = extract(akka.http.impl.util.identityFunc)

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