Deprecate actor materializer (#27538)
This commit is contained in:
parent
8765a4fbe6
commit
b68d67008a
118 changed files with 1233 additions and 731 deletions
|
|
@ -1,83 +0,0 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.stream.typed.javadsl
|
||||
|
||||
import akka.actor.typed.ActorSystem
|
||||
import akka.actor.typed.javadsl.{ ActorContext, Adapter }
|
||||
import akka.stream.ActorMaterializerSettings
|
||||
|
||||
object ActorMaterializerFactory {
|
||||
import akka.actor.typed.scaladsl.adapter._
|
||||
|
||||
/**
|
||||
* Creates an `ActorMaterializer` which will execute every step of a transformation
|
||||
* pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.typed.ActorSystem]]
|
||||
* will be used to create these actors, therefore it is *forbidden* to pass this object
|
||||
* to another actor if the factory is an ActorContext.
|
||||
*
|
||||
* Defaults the actor name prefix used to name actors running the processing steps to `"flow"`.
|
||||
* The actor names are built up of `namePrefix-flowNumber-flowStepNumber-stepName`.
|
||||
*/
|
||||
def create[T](actorSystem: ActorSystem[T]): akka.stream.ActorMaterializer =
|
||||
akka.stream.ActorMaterializer.create(actorSystem.toClassic)
|
||||
|
||||
/**
|
||||
* Creates an `ActorMaterializer` which will execute every step of a transformation
|
||||
* pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.typed.ActorSystem]]
|
||||
* will be used to create one actor that in turn creates actors for the transformation steps.
|
||||
*/
|
||||
def create[T](settings: ActorMaterializerSettings, actorSystem: ActorSystem[T]): akka.stream.ActorMaterializer =
|
||||
akka.stream.ActorMaterializer.create(settings, actorSystem.toClassic)
|
||||
|
||||
/**
|
||||
* Creates an `ActorMaterializer` which will execute every step of a transformation
|
||||
* pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.typed.ActorSystem]]
|
||||
* will be used to create these actors, therefore it is *forbidden* to pass this object
|
||||
* to another actor if the factory is an ActorContext.
|
||||
*
|
||||
* The `namePrefix` is used as the first part of the names of the actors running
|
||||
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
|
||||
* `namePrefix-flowNumber-flowStepNumber-stepName`.
|
||||
*/
|
||||
def create[T](
|
||||
settings: ActorMaterializerSettings,
|
||||
namePrefix: String,
|
||||
actorSystem: ActorSystem[T]): akka.stream.ActorMaterializer =
|
||||
akka.stream.ActorMaterializer.create(settings, actorSystem.toClassic, namePrefix)
|
||||
|
||||
/**
|
||||
* Creates an `ActorMaterializer` which will execute every step of a transformation
|
||||
* pipeline within its own [[akka.actor.Actor]]. The lifecycle of the materialized streams
|
||||
* will be bound to the lifecycle of the provided [[akka.actor.typed.javadsl.ActorContext]]
|
||||
*
|
||||
* Defaults the actor name prefix used to name actors running the processing steps to `"flow"`.
|
||||
* The actor names are built up of `namePrefix-flowNumber-flowStepNumber-stepName`.
|
||||
*/
|
||||
def create[T](ctx: ActorContext[T]): akka.stream.ActorMaterializer =
|
||||
akka.stream.ActorMaterializer.create(Adapter.toClassic(ctx))
|
||||
|
||||
/**
|
||||
* Creates an `ActorMaterializer` which will execute every step of a transformation
|
||||
* pipeline within its own [[akka.actor.Actor]]. The lifecycle of the materialized streams
|
||||
* will be bound to the lifecycle of the provided [[akka.actor.typed.javadsl.ActorContext]]
|
||||
*/
|
||||
def create[T](settings: ActorMaterializerSettings, ctx: ActorContext[T]): akka.stream.ActorMaterializer =
|
||||
akka.stream.ActorMaterializer.create(settings, Adapter.toClassic(ctx))
|
||||
|
||||
/**
|
||||
* Creates an `ActorMaterializer` which will execute every step of a transformation
|
||||
* pipeline within its own [[akka.actor.Actor]]. The lifecycle of the materialized streams
|
||||
* will be bound to the lifecycle of the provided [[akka.actor.typed.javadsl.ActorContext]]
|
||||
*
|
||||
* The `namePrefix` is used as the first part of the names of the actors running
|
||||
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
|
||||
* `namePrefix-flowNumber-flowStepNumber-stepName`.
|
||||
*/
|
||||
def create[T](
|
||||
settings: ActorMaterializerSettings,
|
||||
namePrefix: String,
|
||||
ctx: ActorContext[T]): akka.stream.ActorMaterializer =
|
||||
akka.stream.ActorMaterializer.create(settings, Adapter.toClassic(ctx), namePrefix)
|
||||
}
|
||||
|
|
@ -1,9 +0,0 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.stream.typed
|
||||
|
||||
package object scaladsl {
|
||||
type ActorMaterializer = akka.stream.ActorMaterializer
|
||||
}
|
||||
|
|
@ -1,48 +0,0 @@
|
|||
/*
|
||||
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.stream.typed.scaladsl
|
||||
|
||||
import akka.actor.typed.ActorSystem
|
||||
import akka.actor.typed.scaladsl.ActorContext
|
||||
import akka.stream.ActorMaterializerSettings
|
||||
|
||||
object ActorMaterializer {
|
||||
import akka.actor.typed.scaladsl.adapter._
|
||||
|
||||
/**
|
||||
* Creates an `ActorMaterializer` which will execute every step of a transformation
|
||||
* pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.typed.ActorSystem]]
|
||||
* will be used to create one actor that in turn creates actors for the transformation steps.
|
||||
*
|
||||
* The materializer's [[akka.stream.ActorMaterializerSettings]] will be obtained from the
|
||||
* configuration of the `context`'s underlying [[akka.actor.typed.ActorSystem]].
|
||||
*
|
||||
* The `namePrefix` is used as the first part of the names of the actors running
|
||||
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
|
||||
* `namePrefix-flowNumber-flowStepNumber-stepName`.
|
||||
*/
|
||||
def apply[T](materializerSettings: Option[ActorMaterializerSettings] = None, namePrefix: Option[String] = None)(
|
||||
implicit actorSystem: ActorSystem[T]): ActorMaterializer =
|
||||
akka.stream.ActorMaterializer(materializerSettings, namePrefix)(actorSystem.toClassic)
|
||||
|
||||
/**
|
||||
* Creates an `ActorMaterializer` which will execute every step of a transformation
|
||||
* pipeline within its own [[akka.actor.Actor]]. The lifecycle of the materialized streams
|
||||
* will be bound to the lifecycle of the provided [[akka.actor.typed.scaladsl.ActorContext]]
|
||||
*
|
||||
* The materializer's [[akka.stream.ActorMaterializerSettings]] will be obtained from the
|
||||
* configuration of the `context`'s underlying [[akka.actor.typed.ActorSystem]].
|
||||
*
|
||||
* The `namePrefix` is used as the first part of the names of the actors running
|
||||
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
|
||||
* `namePrefix-flowNumber-flowStepNumber-stepName`.
|
||||
*/
|
||||
def boundToActor[T](
|
||||
ctx: ActorContext[T],
|
||||
materializerSettings: Option[ActorMaterializerSettings] = None,
|
||||
namePrefix: Option[String] = None): ActorMaterializer =
|
||||
akka.stream.ActorMaterializer(materializerSettings, namePrefix)(ctx.toClassic)
|
||||
|
||||
}
|
||||
|
|
@ -29,7 +29,6 @@ public class ActorFlowCompileTest {
|
|||
|
||||
{
|
||||
final ActorSystem<String> system = null;
|
||||
final ActorMaterializer mat = ActorMaterializerFactory.create(system);
|
||||
}
|
||||
|
||||
static
|
||||
|
|
|
|||
|
|
@ -30,7 +30,6 @@ public class ActorSourceSinkCompileTest {
|
|||
|
||||
{
|
||||
final ActorSystem<String> system = null;
|
||||
final ActorMaterializer mat = ActorMaterializerFactory.create(system);
|
||||
}
|
||||
|
||||
{
|
||||
|
|
|
|||
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.stream.typed.javadsl;
|
||||
|
||||
import akka.Done;
|
||||
import akka.actor.AbstractActor;
|
||||
import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
|
||||
import akka.actor.testkit.typed.javadsl.TestProbe;
|
||||
import akka.actor.typed.ActorRef;
|
||||
import akka.actor.typed.Behavior;
|
||||
import akka.actor.typed.TimerSpec;
|
||||
import akka.actor.typed.javadsl.Behaviors;
|
||||
import akka.stream.AbruptStageTerminationException;
|
||||
import akka.stream.AbruptTerminationException;
|
||||
import akka.stream.Materializer;
|
||||
import akka.stream.javadsl.Sink;
|
||||
import akka.stream.javadsl.Source;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
import org.scalatest.junit.JUnitSuite;
|
||||
|
||||
import java.util.concurrent.CompletionStage;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
public class CustomGuardianAndMaterializerTest extends JUnitSuite {
|
||||
|
||||
@ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource();
|
||||
|
||||
@Test
|
||||
public void useSystemWideMaterialiser() throws Exception {
|
||||
CompletionStage<String> result = Source.single("hello").runWith(Sink.head(), testKit.system());
|
||||
|
||||
assertEquals("hello", result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void createCustomSystemLevelMaterialiser() throws Exception {
|
||||
Materializer materializer = Materializer.create(testKit.system());
|
||||
|
||||
CompletionStage<String> result = Source.single("hello").runWith(Sink.head(), materializer);
|
||||
|
||||
assertEquals("hello", result.toCompletableFuture().get(3, TimeUnit.SECONDS));
|
||||
}
|
||||
|
||||
private static Behavior<String> actorStreamBehavior(ActorRef<Object> probe) {
|
||||
return Behaviors.setup(
|
||||
(context) -> {
|
||||
Materializer materializer = Materializer.create(context);
|
||||
|
||||
CompletionStage<Done> done = Source.repeat("hello").runWith(Sink.ignore(), materializer);
|
||||
done.whenComplete(
|
||||
(success, failure) -> {
|
||||
if (success != null) probe.tell(success);
|
||||
else probe.tell(failure);
|
||||
});
|
||||
|
||||
return Behaviors.receive(String.class)
|
||||
.onMessageEquals("stop", () -> Behaviors.stopped())
|
||||
.build();
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void createCustomActorLevelMaterializer() throws Exception {
|
||||
TestProbe<Object> probe = testKit.createTestProbe();
|
||||
ActorRef<String> actor = testKit.spawn(actorStreamBehavior(probe.getRef()));
|
||||
|
||||
actor.tell("stop");
|
||||
|
||||
probe.expectMessageClass(AbruptStageTerminationException.class);
|
||||
}
|
||||
}
|
||||
|
|
@ -25,8 +25,6 @@ object ActorSourceSinkSpec {
|
|||
class ActorSourceSinkSpec extends ScalaTestWithActorTestKit with WordSpecLike {
|
||||
import ActorSourceSinkSpec._
|
||||
|
||||
implicit val mat = ActorMaterializer()
|
||||
|
||||
"ActorSink" should {
|
||||
|
||||
"accept messages" in {
|
||||
|
|
|
|||
|
|
@ -1,44 +0,0 @@
|
|||
/*
|
||||
* Copyright (C) 2017-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.stream.typed.scaladsl
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.stream.AbruptStageTerminationException
|
||||
import akka.stream.scaladsl.{ Sink, Source }
|
||||
import org.scalatest.WordSpecLike
|
||||
|
||||
import scala.concurrent.Future
|
||||
|
||||
class CustomGuardianAndMaterializerSpec extends ScalaTestWithActorTestKit with WordSpecLike {
|
||||
|
||||
implicit val mat = ActorMaterializer()
|
||||
|
||||
"ActorMaterializer" should {
|
||||
|
||||
"work with typed ActorSystem with custom guardian" in {
|
||||
val it: Future[String] = Source.single("hello").runWith(Sink.head)
|
||||
|
||||
it.futureValue should ===("hello")
|
||||
}
|
||||
|
||||
"should kill streams with bound actor context" in {
|
||||
var doneF: Future[Done] = null
|
||||
val behavior =
|
||||
Behaviors.setup[String] { ctx =>
|
||||
implicit val mat: ActorMaterializer = ActorMaterializer.boundToActor(ctx)
|
||||
doneF = Source.repeat("hello").runWith(Sink.ignore)
|
||||
|
||||
Behaviors.receiveMessage[String](_ => Behaviors.stopped)
|
||||
}
|
||||
|
||||
val actorRef = spawn(behavior)
|
||||
|
||||
actorRef ! "kill"
|
||||
eventually(doneF.failed.futureValue shouldBe an[AbruptStageTerminationException])
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,68 @@
|
|||
/*
|
||||
* Copyright (C) 2017-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.stream.typed.scaladsl
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.stream.AbruptStageTerminationException
|
||||
import akka.stream.Materializer
|
||||
import akka.stream.scaladsl.Sink
|
||||
import akka.stream.scaladsl.Source
|
||||
import org.scalatest.WordSpecLike
|
||||
|
||||
import scala.concurrent.Future
|
||||
import scala.util.Success
|
||||
|
||||
class MaterializerForTypedSpec extends ScalaTestWithActorTestKit with WordSpecLike {
|
||||
|
||||
"Materialization in typed" should {
|
||||
|
||||
"use system materializer by default" in {
|
||||
val it: Future[String] = Source.single("hello").runWith(Sink.head)
|
||||
it.futureValue should ===("hello")
|
||||
}
|
||||
|
||||
"allow for custom instances for special cases" in {
|
||||
val customMaterializer = Materializer(system)
|
||||
val it: Future[String] = Source.single("hello").runWith(Sink.head)(customMaterializer)
|
||||
|
||||
it.futureValue should ===("hello")
|
||||
}
|
||||
|
||||
"allow for actor context bound instances" in {
|
||||
val probe = testKit.createTestProbe[Any]()
|
||||
val actor = testKit.spawn(Behaviors.setup[String] { context =>
|
||||
val materializerForActor = Materializer(context)
|
||||
|
||||
Behaviors.receiveMessage[String] {
|
||||
case "run" =>
|
||||
val f = Source.single("hello").runWith(Sink.head)(materializerForActor)
|
||||
f.onComplete(probe.ref ! _)(system.executionContext)
|
||||
Behaviors.same
|
||||
}
|
||||
})
|
||||
actor ! "run"
|
||||
probe.expectMessage(Success("hello"))
|
||||
|
||||
}
|
||||
|
||||
"should kill streams with bound actor context" in {
|
||||
var doneF: Future[Done] = null
|
||||
val behavior =
|
||||
Behaviors.setup[String] { ctx =>
|
||||
implicit val mat: Materializer = Materializer(ctx)
|
||||
doneF = Source.repeat("hello").runWith(Sink.ignore)
|
||||
|
||||
Behaviors.receiveMessage[String](_ => Behaviors.stopped)
|
||||
}
|
||||
|
||||
val actorRef = spawn(behavior)
|
||||
|
||||
actorRef ! "kill"
|
||||
eventually(doneF.failed.futureValue shouldBe an[AbruptStageTerminationException])
|
||||
}
|
||||
}
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue