Deprecate actor materializer (#27538)

This commit is contained in:
Johan Andrén 2019-09-05 16:08:37 +02:00 committed by GitHub
parent 8765a4fbe6
commit b68d67008a
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
118 changed files with 1233 additions and 731 deletions

View file

@ -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)
}

View file

@ -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
}

View file

@ -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)
}

View file

@ -29,7 +29,6 @@ public class ActorFlowCompileTest {
{
final ActorSystem<String> system = null;
final ActorMaterializer mat = ActorMaterializerFactory.create(system);
}
static

View file

@ -30,7 +30,6 @@ public class ActorSourceSinkCompileTest {
{
final ActorSystem<String> system = null;
final ActorMaterializer mat = ActorMaterializerFactory.create(system);
}
{

View file

@ -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);
}
}

View file

@ -25,8 +25,6 @@ object ActorSourceSinkSpec {
class ActorSourceSinkSpec extends ScalaTestWithActorTestKit with WordSpecLike {
import ActorSourceSinkSpec._
implicit val mat = ActorMaterializer()
"ActorSink" should {
"accept messages" in {

View file

@ -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])
}
}
}

View file

@ -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])
}
}
}