Change "Untyped" to "Classic" (#27627)

* Untyped to classic naming convention #24717
* Added entry of naming convention change to migration docs.
* Rename `toUntyped` to `toClassic` in adapters
This commit is contained in:
Helena Edelson 2019-09-05 03:30:52 -07:00 committed by Patrik Nordwall
parent 3fa133d901
commit 29f2eba049
135 changed files with 940 additions and 965 deletions

View file

@ -19,7 +19,7 @@ import akka.actor.typed.Props
import akka.actor.typed.Scheduler
import akka.actor.typed.Settings
import akka.annotation.InternalApi
import akka.{ actor => untyped }
import akka.{ actor => classic }
import akka.Done
import com.typesafe.config.ConfigFactory
@ -39,7 +39,7 @@ import com.github.ghik.silencer.silent
with ActorRefImpl[Nothing]
with InternalRecipientRef[Nothing] {
override val path: untyped.ActorPath = untyped.RootActorPath(untyped.Address("akka", name)) / "user"
override val path: classic.ActorPath = classic.RootActorPath(classic.Address("akka", name)) / "user"
override val settings: Settings = new Settings(getClass.getClassLoader, ConfigFactory.empty, name)
@ -57,7 +57,7 @@ import com.github.ghik.silencer.silent
// stream materialization etc. using stub not supported
override private[akka] def classicSystem =
throw new UnsupportedOperationException("no untyped actor system available")
throw new UnsupportedOperationException("no classic actor system available")
// impl InternalRecipientRef
def isTerminated: Boolean = whenTerminated.isCompleted
@ -72,7 +72,7 @@ import com.github.ghik.silencer.silent
def shutdown(): Unit = ()
}
override def dynamicAccess: untyped.DynamicAccess = new untyped.ReflectiveDynamicAccess(getClass.getClassLoader)
override def dynamicAccess: classic.DynamicAccess = new classic.ReflectiveDynamicAccess(getClass.getClassLoader)
override def logConfiguration(): Unit = log.info(settings.toString)

View file

@ -9,7 +9,7 @@ import java.util.concurrent.ConcurrentLinkedQueue
import akka.actor.typed.ActorRef
import akka.actor.typed.internal.{ ActorRefImpl, SystemMessage }
import akka.annotation.InternalApi
import akka.{ actor => untyped }
import akka.{ actor => classic }
import scala.annotation.tailrec
import akka.actor.ActorRefProvider
@ -18,7 +18,7 @@ import akka.actor.typed.internal.InternalRecipientRef
/**
* INTERNAL API
*/
@InternalApi private[akka] final class DebugRef[T](override val path: untyped.ActorPath, override val isLocal: Boolean)
@InternalApi private[akka] final class DebugRef[T](override val path: classic.ActorPath, override val isLocal: Boolean)
extends ActorRef[T]
with ActorRefImpl[T]
with InternalRecipientRef[T] {

View file

@ -13,7 +13,7 @@ import akka.actor.{ ActorPath, InvalidMessageException }
import akka.annotation.InternalApi
import akka.event.Logging
import akka.util.{ Helpers, OptionVal }
import akka.{ actor => untyped }
import akka.{ actor => classic }
import java.util.concurrent.ThreadLocalRandom.{ current => rnd }
import scala.collection.immutable.TreeMap
@ -194,7 +194,7 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
}
override def spawn[U](behavior: Behavior[U], name: String, props: Props = Props.empty): ActorRef[U] =
_children.get(name) match {
case Some(_) => throw untyped.InvalidActorNameException(s"actor name $name is already taken")
case Some(_) => throw classic.InvalidActorNameException(s"actor name $name is already taken")
case None =>
val btk = new BehaviorTestKitImpl[U]((path / name).withUid(rnd().nextInt()), behavior)
_children += name -> btk
@ -221,8 +221,8 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
override def setReceiveTimeout(d: FiniteDuration, message: T): Unit = ()
override def cancelReceiveTimeout(): Unit = ()
override def scheduleOnce[U](delay: FiniteDuration, target: ActorRef[U], message: U): untyped.Cancellable =
new untyped.Cancellable {
override def scheduleOnce[U](delay: FiniteDuration, target: ActorRef[U], message: U): classic.Cancellable =
new classic.Cancellable {
override def cancel() = false
override def isCancelled = true
}

View file

@ -33,7 +33,7 @@ object ManualTime {
def get[A](system: ActorSystem[A]): ManualTime =
system.scheduler match {
case adapter: SchedulerAdapter =>
adapter.untypedScheduler match {
adapter.classicScheduler match {
case sc: akka.testkit.ExplicitlyTriggeredScheduler => new ManualTime(sc)
case _ =>
throw new IllegalArgumentException(
@ -42,7 +42,7 @@ object ManualTime {
}
case s =>
throw new IllegalArgumentException(
s"ActorSystem.scheduler is not an untyped SchedulerAdapter but a ${s.getClass.getName}, this is not supported")
s"ActorSystem.scheduler is not a classic SchedulerAdapter but a ${s.getClass.getName}, this is not supported")
}
}

View file

@ -32,7 +32,7 @@ object ManualTime {
def apply()(implicit system: ActorSystem[_]): ManualTime =
system.scheduler match {
case adapter: SchedulerAdapter =>
adapter.untypedScheduler match {
adapter.classicScheduler match {
case sc: akka.testkit.ExplicitlyTriggeredScheduler => new ManualTime(sc)
case _ =>
throw new IllegalArgumentException(
@ -41,7 +41,7 @@ object ManualTime {
}
case s =>
throw new IllegalArgumentException(
s"ActorSystem.scheduler is not an untyped SchedulerAdapter but a ${s.getClass.getName}, this is not supported")
s"ActorSystem.scheduler is not a classic SchedulerAdapter but a ${s.getClass.getName}, this is not supported")
}
}

View file

@ -14,7 +14,7 @@ import akka.serialization.Serializers
*/
class SerializationTestKit(system: ActorSystem[_]) {
private val serialization = SerializationExtension(system.toUntyped)
private val serialization = SerializationExtension(system.toClassic)
/**
* Verify serialization roundtrip.

View file

@ -5,14 +5,11 @@
package akka.actor.typed.javadsl;
import org.junit.ClassRule;
import org.junit.Ignore;
import org.junit.Test;
import org.scalatest.junit.JUnitSuite;
import scala.concurrent.duration.FiniteDuration;
import java.time.Duration;
import java.util.concurrent.TimeUnit;
import akka.actor.ActorSystem;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.AkkaSpec;
@ -26,11 +23,11 @@ import static akka.actor.typed.javadsl.Behaviors.*;
public class AdapterTest extends JUnitSuite {
static akka.actor.Props untyped1() {
return akka.actor.Props.create(Untyped1.class, () -> new Untyped1());
static akka.actor.Props classic1() {
return akka.actor.Props.create(Classic1.class, () -> new Classic1());
}
static class Untyped1 extends akka.actor.AbstractActor {
static class Classic1 extends akka.actor.AbstractActor {
@Override
public Receive createReceive() {
return receiveBuilder()
@ -60,28 +57,28 @@ public class AdapterTest extends JUnitSuite {
Behavior<String> onMessage(ActorContext<String> context, String message) {
if (message.equals("send")) {
akka.actor.ActorRef replyTo = Adapter.toUntyped(context.getSelf());
akka.actor.ActorRef replyTo = Adapter.toClassic(context.getSelf());
ref.tell("ping", replyTo);
return same();
} else if (message.equals("pong")) {
probe.tell("ok", akka.actor.ActorRef.noSender());
return same();
} else if (message.equals("actorOf")) {
akka.actor.ActorRef child = Adapter.actorOf(context, untyped1());
child.tell("ping", Adapter.toUntyped(context.getSelf()));
akka.actor.ActorRef child = Adapter.actorOf(context, classic1());
child.tell("ping", Adapter.toClassic(context.getSelf()));
return same();
} else if (message.equals("watch")) {
Adapter.watch(context, ref);
return same();
} else if (message.equals("supervise-restart")) {
// restart is the default, otherwise an intermediate is required
akka.actor.ActorRef child = Adapter.actorOf(context, untyped1());
akka.actor.ActorRef child = Adapter.actorOf(context, classic1());
Adapter.watch(context, child);
child.tell(new ThrowIt3(), Adapter.toUntyped(context.getSelf()));
child.tell("ping", Adapter.toUntyped(context.getSelf()));
child.tell(new ThrowIt3(), Adapter.toClassic(context.getSelf()));
child.tell("ping", Adapter.toClassic(context.getSelf()));
return same();
} else if (message.equals("stop-child")) {
akka.actor.ActorRef child = Adapter.actorOf(context, untyped1());
akka.actor.ActorRef child = Adapter.actorOf(context, classic1());
Adapter.watch(context, child);
Adapter.stop(context, child);
return same();
@ -127,16 +124,16 @@ public class AdapterTest extends JUnitSuite {
static class ThrowIt3 extends ThrowIt {}
static akka.actor.Props untyped2(ActorRef<Ping> ref, akka.actor.ActorRef probe) {
return akka.actor.Props.create(Untyped2.class, () -> new Untyped2(ref, probe));
static akka.actor.Props classic2(ActorRef<Ping> ref, akka.actor.ActorRef probe) {
return akka.actor.Props.create(Classic2.class, () -> new Classic2(ref, probe));
}
static class Untyped2 extends akka.actor.AbstractActor {
static class Classic2 extends akka.actor.AbstractActor {
private final ActorRef<Ping> ref;
private final akka.actor.ActorRef probe;
private final SupervisorStrategy strategy;
Untyped2(ActorRef<Ping> ref, akka.actor.ActorRef probe) {
Classic2(ActorRef<Ping> ref, akka.actor.ActorRef probe) {
this.ref = ref;
this.probe = probe;
this.strategy = strategy();
@ -246,44 +243,44 @@ public class AdapterTest extends JUnitSuite {
private final ActorSystem system = actorSystemResource.getSystem();
@Test
public void shouldSendMessageFromTypedToUntyped() {
public void shouldSendMessageFromTypedToClassic() {
TestKit probe = new TestKit(system);
akka.actor.ActorRef untypedRef = system.actorOf(untyped1());
akka.actor.ActorRef classicRef = system.actorOf(classic1());
ActorRef<String> typedRef =
Adapter.spawnAnonymous(system, Typed1.create(untypedRef, probe.getRef()));
Adapter.spawnAnonymous(system, Typed1.create(classicRef, probe.getRef()));
typedRef.tell("send");
probe.expectMsg("ok");
}
@Test
public void shouldSendMessageFromUntypedToTyped() {
public void shouldSendMessageFromClassicToTyped() {
TestKit probe = new TestKit(system);
ActorRef<Ping> typedRef = Adapter.spawnAnonymous(system, typed2()).narrow();
akka.actor.ActorRef untypedRef = system.actorOf(untyped2(typedRef, probe.getRef()));
untypedRef.tell("send", akka.actor.ActorRef.noSender());
akka.actor.ActorRef classicRef = system.actorOf(classic2(typedRef, probe.getRef()));
classicRef.tell("send", akka.actor.ActorRef.noSender());
probe.expectMsg("ok");
}
@Test
public void shouldSpawnTypedChildFromUntypedParent() {
public void shouldSpawnTypedChildFromClassicParent() {
TestKit probe = new TestKit(system);
ActorRef<Ping> ignore = Adapter.spawnAnonymous(system, ignore());
akka.actor.ActorRef untypedRef = system.actorOf(untyped2(ignore, probe.getRef()));
untypedRef.tell("spawn", akka.actor.ActorRef.noSender());
akka.actor.ActorRef classicRef = system.actorOf(classic2(ignore, probe.getRef()));
classicRef.tell("spawn", akka.actor.ActorRef.noSender());
probe.expectMsg("ok");
}
@Test
public void shouldActorOfTypedChildViaPropsFromUntypedParent() {
public void shouldActorOfTypedChildViaPropsFromClassicParent() {
TestKit probe = new TestKit(system);
ActorRef<Ping> ignore = Adapter.spawnAnonymous(system, ignore());
akka.actor.ActorRef untypedRef = system.actorOf(untyped2(ignore, probe.getRef()));
untypedRef.tell("actorOf-props", akka.actor.ActorRef.noSender());
akka.actor.ActorRef classicRef = system.actorOf(classic2(ignore, probe.getRef()));
classicRef.tell("actorOf-props", akka.actor.ActorRef.noSender());
probe.expectMsg("ok");
}
@Test
public void shouldActorOfUntypedChildFromTypedParent() {
public void shouldActorOfClassicChildFromTypedParent() {
TestKit probe = new TestKit(system);
akka.actor.ActorRef ignore = system.actorOf(akka.actor.Props.empty());
ActorRef<String> typedRef =
@ -293,29 +290,29 @@ public class AdapterTest extends JUnitSuite {
}
@Test
public void shouldWatchTypedFromUntyped() {
public void shouldWatchTypedFromClassic() {
TestKit probe = new TestKit(system);
ActorRef<Typed2Msg> typedRef = Adapter.spawnAnonymous(system, typed2());
ActorRef<Ping> typedRef2 = typedRef.narrow();
akka.actor.ActorRef untypedRef = system.actorOf(untyped2(typedRef2, probe.getRef()));
untypedRef.tell("watch", akka.actor.ActorRef.noSender());
akka.actor.ActorRef classicRef = system.actorOf(classic2(typedRef2, probe.getRef()));
classicRef.tell("watch", akka.actor.ActorRef.noSender());
typedRef.tell(new StopIt());
probe.expectMsg("terminated");
}
@Test
public void shouldWatchUntypedFromTyped() {
public void shouldWatchClassicFromTyped() {
TestKit probe = new TestKit(system);
akka.actor.ActorRef untypedRef = system.actorOf(untyped1());
akka.actor.ActorRef classicRef = system.actorOf(classic1());
ActorRef<String> typedRef =
Adapter.spawnAnonymous(system, Typed1.create(untypedRef, probe.getRef()));
Adapter.spawnAnonymous(system, Typed1.create(classicRef, probe.getRef()));
typedRef.tell("watch");
untypedRef.tell(akka.actor.PoisonPill.getInstance(), akka.actor.ActorRef.noSender());
classicRef.tell(akka.actor.PoisonPill.getInstance(), akka.actor.ActorRef.noSender());
probe.expectMsg("terminated");
}
@Test
public void shouldSuperviseUntypedChildAsRestartFromTypedParent() {
public void shouldSuperviseClassicChildAsRestartFromTypedParent() {
TestKit probe = new TestKit(system);
akka.actor.ActorRef ignore = system.actorOf(akka.actor.Props.empty());
ActorRef<String> typedRef =
@ -335,16 +332,16 @@ public class AdapterTest extends JUnitSuite {
}
@Test
public void shouldStopTypedChildFromUntypedParent() {
public void shouldStopTypedChildFromClassicParent() {
TestKit probe = new TestKit(system);
ActorRef<Ping> ignore = Adapter.spawnAnonymous(system, ignore());
akka.actor.ActorRef untypedRef = system.actorOf(untyped2(ignore, probe.getRef()));
untypedRef.tell("stop-child", akka.actor.ActorRef.noSender());
akka.actor.ActorRef classicRef = system.actorOf(classic2(ignore, probe.getRef()));
classicRef.tell("stop-child", akka.actor.ActorRef.noSender());
probe.expectMsg("terminated");
}
@Test
public void shouldStopUntypedChildFromTypedParent() {
public void shouldStopClassicChildFromTypedParent() {
TestKit probe = new TestKit(system);
akka.actor.ActorRef ignore = system.actorOf(akka.actor.Props.empty());
ActorRef<String> typedRef =

View file

@ -26,7 +26,7 @@ public class BubblingSampleTest extends JUnitSuite {
"akka.loggers = [ akka.testkit.TestEventListener ]\n" + "akka.loglevel=warning"));
// actual exception and then the deathpacts
new EventFilter(Exception.class, ActorSystemAdapter.toUntyped(system))
new EventFilter(Exception.class, ActorSystemAdapter.toClassic(system))
.occurrences(4)
.intercept(
() -> {

View file

@ -10,7 +10,7 @@ import akka.actor.typed.ActorSystem;
import akka.actor.typed.Behavior;
import akka.actor.typed.javadsl.Behaviors;
// #adapter-import
// In java use the static methods on Adapter to convert from typed to untyped
// In java use the static methods on Adapter to convert from typed to classic
import akka.actor.typed.javadsl.Adapter;
// #adapter-import
import akka.testkit.TestProbe;
@ -21,12 +21,12 @@ import scala.concurrent.duration.Duration;
import static akka.actor.typed.javadsl.Behaviors.same;
public class UntypedWatchingTypedTest extends JUnitSuite {
public class ClassicWatchingTypedTest extends JUnitSuite {
// #untyped-watch
public static class Untyped extends AbstractActor {
// #classic-watch
public static class Classic extends AbstractActor {
public static akka.actor.Props props() {
return akka.actor.Props.create(Untyped.class);
return akka.actor.Props.create(Classic.class);
}
private final akka.actor.typed.ActorRef<Typed.Command> second =
@ -54,7 +54,7 @@ public class UntypedWatchingTypedTest extends JUnitSuite {
.build();
}
}
// #untyped-watch
// #classic-watch
// #typed
public abstract static class Typed {
@ -85,22 +85,22 @@ public class UntypedWatchingTypedTest extends JUnitSuite {
@Test
public void testItWorks() {
// #create-untyped
// #create-classic
akka.actor.ActorSystem as = akka.actor.ActorSystem.create();
akka.actor.ActorRef untyped = as.actorOf(Untyped.props());
// #create-untyped
akka.actor.ActorRef classic = as.actorOf(Classic.props());
// #create-classic
TestProbe probe = new TestProbe(as);
probe.watch(untyped);
probe.expectTerminated(untyped, Duration.create(1, "second"));
probe.watch(classic);
probe.expectTerminated(classic, Duration.create(1, "second"));
TestKit.shutdownActorSystem(as);
}
@Test
public void testConversionFromUnTypedSystemToTyped() {
// #convert-untyped
akka.actor.ActorSystem untypedActorSystem = akka.actor.ActorSystem.create();
ActorSystem<Void> typedActorSystem = Adapter.toTyped(untypedActorSystem);
// #convert-untyped
TestKit.shutdownActorSystem(untypedActorSystem);
public void testConversionFromClassicSystemToTyped() {
// #convert-classic
akka.actor.ActorSystem classicActorSystem = akka.actor.ActorSystem.create();
ActorSystem<Void> typedActorSystem = Adapter.toTyped(classicActorSystem);
// #convert-classic
TestKit.shutdownActorSystem(classicActorSystem);
}
}

View file

@ -9,7 +9,7 @@ import akka.actor.ActorSystem;
import akka.actor.typed.ActorRef;
import akka.actor.typed.Behavior;
// #adapter-import
// in Java use the static methods on Adapter to convert from untyped to typed
// in Java use the static methods on Adapter to convert from classic to typed
import akka.actor.typed.javadsl.Adapter;
// #adapter-import
import akka.actor.typed.javadsl.AbstractBehavior;
@ -20,7 +20,7 @@ import akka.testkit.javadsl.TestKit;
import org.junit.Test;
import org.scalatest.junit.JUnitSuite;
public class TypedWatchingUntypedTest extends JUnitSuite {
public class TypedWatchingClassicTest extends JUnitSuite {
// #typed
public static class Typed extends AbstractBehavior<Typed.Command> {
@ -50,12 +50,12 @@ public class TypedWatchingUntypedTest extends JUnitSuite {
public static Behavior<Command> create() {
return akka.actor.typed.javadsl.Behaviors.setup(
context -> {
akka.actor.ActorRef second = Adapter.actorOf(context, Untyped.props(), "second");
akka.actor.ActorRef second = Adapter.actorOf(context, Classic.props(), "second");
Adapter.watch(context, second);
second.tell(
new Typed.Ping(context.getSelf().narrow()), Adapter.toUntyped(context.getSelf()));
new Typed.Ping(context.getSelf().narrow()), Adapter.toClassic(context.getSelf()));
return new Typed(context, second);
});
@ -76,10 +76,10 @@ public class TypedWatchingUntypedTest extends JUnitSuite {
}
// #typed
// #untyped
public static class Untyped extends AbstractActor {
// #classic
public static class Classic extends AbstractActor {
public static akka.actor.Props props() {
return akka.actor.Props.create(Untyped.class);
return akka.actor.Props.create(Classic.class);
}
@Override
@ -91,7 +91,7 @@ public class TypedWatchingUntypedTest extends JUnitSuite {
message.replyTo.tell(Typed.Pong.INSTANCE);
}
}
// #untyped
// #classic
@Test
public void testItWorks() {
@ -100,8 +100,8 @@ public class TypedWatchingUntypedTest extends JUnitSuite {
ActorRef<Typed.Command> typed = Adapter.spawn(as, Typed.create(), "Typed");
// #create
TestKit probe = new TestKit(as);
probe.watch(Adapter.toUntyped(typed));
probe.expectTerminated(Adapter.toUntyped(typed));
probe.watch(Adapter.toClassic(typed));
probe.expectTerminated(Adapter.toClassic(typed));
TestKit.shutdownActorSystem(as);
}
}

View file

@ -69,7 +69,7 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit("""
// FIXME #24348: eventfilter support in typed testkit
import scaladsl.adapter._
implicit val untypedSystem = system.toUntyped
implicit val classicSystem = system.toClassic
import ActorSpecMessages._

View file

@ -36,7 +36,7 @@ class AskSpec extends ScalaTestWithActorTestKit("""
// FIXME #24348: eventfilter support in typed testkit
import scaladsl.adapter._
implicit val untypedSystem = system.toUntyped
implicit val classicSystem = system.toClassic
implicit def executor: ExecutionContext =
system.executionContext
@ -95,7 +95,7 @@ class AskSpec extends ScalaTestWithActorTestKit("""
val noSuchActor: ActorRef[Msg] = system match {
case adaptedSys: ActorSystemAdapter[_] =>
import akka.actor.typed.scaladsl.adapter._
adaptedSys.untypedSystem.provider.resolveActorRef("/foo/bar")
adaptedSys.system.provider.resolveActorRef("/foo/bar")
case _ =>
fail("this test must only run in an adapted actor system")
}
@ -113,7 +113,7 @@ class AskSpec extends ScalaTestWithActorTestKit("""
// It's unlikely but possible that this happens, since the receiving actor would
// have to accept a message with an actoref that accepts AnyRef or be doing crazy casting
// For completeness sake though
implicit val untypedSystem = akka.actor.ActorSystem("AskSpec-untyped-1")
implicit val classicSystem = akka.actor.ActorSystem("AskSpec-classic-1")
try {
case class Ping(respondTo: ActorRef[AnyRef])
val ex = new RuntimeException("not good!")
@ -124,15 +124,15 @@ class AskSpec extends ScalaTestWithActorTestKit("""
}
}
val legacyActor = untypedSystem.actorOf(akka.actor.Props(new LegacyActor))
val legacyActor = classicSystem.actorOf(akka.actor.Props(new LegacyActor))
import scaladsl.AskPattern._
implicit val timeout: Timeout = 3.seconds
implicit val scheduler = untypedSystem.toTyped.scheduler
implicit val scheduler = classicSystem.toTyped.scheduler
val typedLegacy: ActorRef[AnyRef] = legacyActor
(typedLegacy.ask(Ping)).failed.futureValue should ===(ex)
} finally {
akka.testkit.TestKit.shutdownActorSystem(untypedSystem)
akka.testkit.TestKit.shutdownActorSystem(classicSystem)
}
}
@ -175,7 +175,7 @@ class AskSpec extends ScalaTestWithActorTestKit("""
EventFilter[RuntimeException](message = "Exception thrown out of adapter. Stopping myself.", occurrences = 1)
.intercept {
replyRef2 ! 42L
}(system.toUntyped)
}(system.toClassic)
probe.expectTerminated(ref, probe.remainingOrDefault)
}

View file

@ -39,7 +39,7 @@ class DeferredSpec extends ScalaTestWithActorTestKit("""
// FIXME #24348: eventfilter support in typed testkit
import scaladsl.adapter._
implicit val untypedSystem = system.toUntyped
implicit val classicSystem = system.toClassic
"Deferred behavior" must {
"must create underlying" in {

View file

@ -188,13 +188,13 @@ class ExtensionsSpec extends ScalaTestWithActorTestKit with WordSpecLike {
(instance1 should be).theSameInstanceAs(instance2)
}
"load registered typed extensions eagerly even for untyped system" in {
"load registered extensions eagerly even for classic system" in {
import akka.actor.typed.scaladsl.adapter._
val beforeCreation = InstanceCountingExtension.createCount.get()
val untypedSystem = akka.actor.ActorSystem("as", ExtensionsSpec.config)
val classicSystem = akka.actor.ActorSystem("as", ExtensionsSpec.config)
try {
val before = InstanceCountingExtension.createCount.get()
InstanceCountingExtension(untypedSystem.toTyped)
InstanceCountingExtension(classicSystem.toTyped)
val after = InstanceCountingExtension.createCount.get()
// should have been loaded even before it was accessed in the test because InstanceCountingExtension is listed
@ -202,21 +202,21 @@ class ExtensionsSpec extends ScalaTestWithActorTestKit with WordSpecLike {
before shouldEqual beforeCreation + 1
after shouldEqual before
} finally {
untypedSystem.terminate().futureValue
classicSystem.terminate().futureValue
}
}
"not create an extension multiple times when using the ActorSystemAdapter" in {
import akka.actor.typed.scaladsl.adapter._
val untypedSystem = akka.actor.ActorSystem()
val classicSystem = akka.actor.ActorSystem()
try {
val ext1 = DummyExtension1(untypedSystem.toTyped)
val ext2 = DummyExtension1(untypedSystem.toTyped)
val ext1 = DummyExtension1(classicSystem.toTyped)
val ext2 = DummyExtension1(classicSystem.toTyped)
(ext1 should be).theSameInstanceAs(ext2)
} finally {
untypedSystem.terminate().futureValue
classicSystem.terminate().futureValue
}
}

View file

@ -82,7 +82,7 @@ class InterceptSpec extends ScalaTestWithActorTestKit("""
// FIXME #24348: eventfilter support in typed testkit
import scaladsl.adapter._
implicit val untypedSystem = system.toUntyped
implicit val classicSystem = system.toClassic
private def snitchingInterceptor(probe: ActorRef[String]) = new BehaviorInterceptor[String, String] {
override def aroundReceive(

View file

@ -17,7 +17,7 @@ class LogMessagesSpec extends ScalaTestWithActorTestKit("""
akka.loggers = ["akka.testkit.TestEventListener"]
""") with WordSpecLike {
implicit val untyped: actor.ActorSystem = system.toUntyped
implicit val classic: actor.ActorSystem = system.toClassic
"The log messages behavior" should {

View file

@ -30,7 +30,7 @@ class MailboxSelectorSpec extends ScalaTestWithActorTestKit("""
// FIXME #24348: eventfilter support in typed testkit
import scaladsl.adapter._
implicit val untypedSystem = system.toUntyped
implicit val classicSystem = system.toClassic
case class WhatsYourMailbox(replyTo: ActorRef[MessageQueue])
private def behavior: Behavior[WhatsYourMailbox] =
@ -39,7 +39,7 @@ class MailboxSelectorSpec extends ScalaTestWithActorTestKit("""
case WhatsYourMailbox(replyTo) =>
val mailbox = context match {
case adapter: ActorContextAdapter[_] =>
adapter.untypedContext match {
adapter.classicContext match {
case cell: ActorCell =>
cell.mailbox.messageQueue
}

View file

@ -261,7 +261,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
// FIXME #24348: eventfilter support in typed testkit
import akka.actor.typed.scaladsl.adapter._
implicit val untypedSystem = system.toUntyped
implicit val classicSystem = system.toClassic
class FailingConstructorTestSetup(failCount: Int) {
val failCounter = new AtomicInteger(0)
@ -782,7 +782,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
.onFailure[Exception](strategy)
val droppedMessagesProbe = TestProbe[Dropped]()
system.toUntyped.eventStream.subscribe(droppedMessagesProbe.ref.toUntyped, classOf[Dropped])
system.toClassic.eventStream.subscribe(droppedMessagesProbe.ref.toClassic, classOf[Dropped])
val ref = spawn(behv)
EventFilter[Exc1](occurrences = 1).intercept {
startedProbe.expectMessage(Started)
@ -796,8 +796,8 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
ref ! Ping(4)
probe.expectMessage(Pong(1))
probe.expectMessage(Pong(2))
droppedMessagesProbe.expectMessage(Dropped(Ping(3), "Stash is full in [RestartSupervisor]", ref.toUntyped))
droppedMessagesProbe.expectMessage(Dropped(Ping(4), "Stash is full in [RestartSupervisor]", ref.toUntyped))
droppedMessagesProbe.expectMessage(Dropped(Ping(3), "Stash is full in [RestartSupervisor]", ref.toClassic))
droppedMessagesProbe.expectMessage(Dropped(Ping(4), "Stash is full in [RestartSupervisor]", ref.toClassic))
}
}

View file

@ -25,7 +25,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
// FIXME #24348: eventfilter support in typed testkit
import scaladsl.adapter._
implicit val untypedSystem = system.toUntyped
implicit val classicSystem = system.toClassic
sealed trait Command
case class Tick(n: Int) extends Command
@ -313,7 +313,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
ref ! "stop"
}
probe.expectTerminated(ref)
system.toUntyped.eventStream.subscribe(probe.ref.toUntyped, classOf[DeadLetter])
system.toClassic.eventStream.subscribe(probe.ref.toClassic, classOf[DeadLetter])
probe.expectNoMessage(1.second)
}
}

View file

@ -35,7 +35,7 @@ class TransformMessagesSpec extends ScalaTestWithActorTestKit("""
akka.loggers = [akka.testkit.TestEventListener]
""") with WordSpecLike {
implicit val untypedSystem = system.toUntyped
implicit val classicSystem = system.toClassic
def intToString(probe: ActorRef[String]): Behavior[Int] = {
Behaviors

View file

@ -46,7 +46,7 @@ object WatchSpec {
class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpecLike {
implicit def untypedSystem = system.toUntyped
implicit def classicSystem = system.toClassic
import WatchSpec._

View file

@ -7,11 +7,11 @@ import akka.actor.Actor
import akka.actor.testkit.typed.TestException
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed._
import akka.actor.typed.coexistence.UntypedSupervisingTypedSpec.{
SpawnAnonFromUntyped,
SpawnFromUntyped,
TypedSpawnedFromUntypedConext,
UntypedToTyped
import akka.actor.typed.coexistence.ClassicSupervisingTypedSpec.{
ClassicToTyped,
SpawnAnonFromClassic,
SpawnFromClassic,
TypedSpawnedFromClassicContext
}
import akka.testkit.{ AkkaSpec, ImplicitSender, TestProbe }
import akka.actor.typed.scaladsl.adapter._
@ -31,55 +31,55 @@ object ProbedBehavior {
}
}
object UntypedSupervisingTypedSpec {
object ClassicSupervisingTypedSpec {
case class SpawnFromUntyped(behav: Behavior[String], name: String)
case class SpawnAnonFromUntyped(behav: Behavior[String])
case class TypedSpawnedFromUntypedConext(actorRef: ActorRef[String])
case class SpawnFromClassic(behav: Behavior[String], name: String)
case class SpawnAnonFromClassic(behav: Behavior[String])
case class TypedSpawnedFromClassicContext(actorRef: ActorRef[String])
class UntypedToTyped extends Actor {
class ClassicToTyped extends Actor {
override def receive: Receive = {
case SpawnFromUntyped(behav, name) =>
sender() ! TypedSpawnedFromUntypedConext(context.spawn(behav, name))
case SpawnAnonFromUntyped(behav) =>
sender() ! TypedSpawnedFromUntypedConext(context.spawnAnonymous(behav))
case SpawnFromClassic(behav, name) =>
sender() ! TypedSpawnedFromClassicContext(context.spawn(behav, name))
case SpawnAnonFromClassic(behav) =>
sender() ! TypedSpawnedFromClassicContext(context.spawnAnonymous(behav))
}
}
}
class UntypedSupervisingTypedSpec
class ClassicSupervisingTypedSpec
extends AkkaSpec("akka.actor.testkit.typed.expect-no-message-default = 50 ms")
with ImplicitSender {
implicit val typedActorSystem: ActorSystem[Nothing] = system.toTyped
"An untyped actor system that spawns typed actors" should {
"A classic actor system that spawns typed actors" should {
"default to stop for supervision" in {
val probe = TestProbe()
val underTest = system.spawn(ProbedBehavior.behavior(probe.ref), "a1")
watch(underTest.toUntyped)
watch(underTest.toClassic)
underTest ! "throw"
probe.expectMsg(PostStop)
probe.expectNoMessage()
expectTerminated(underTest.toUntyped)
expectTerminated(underTest.toClassic)
}
"default to stop for supervision for spawn anonymous" in {
val probe = TestProbe()
val underTest = system.spawnAnonymous(ProbedBehavior.behavior(probe.ref))
watch(underTest.toUntyped)
watch(underTest.toClassic)
underTest ! "throw"
probe.expectMsg(PostStop)
probe.expectNoMessage()
expectTerminated(underTest.toUntyped)
expectTerminated(underTest.toClassic)
}
"allows overriding the default" in {
val probe = TestProbe()
val value = Behaviors.supervise(ProbedBehavior.behavior(probe.ref)).onFailure(SupervisorStrategy.restart)
val underTest = system.spawn(value, "a2")
watch(underTest.toUntyped)
watch(underTest.toClassic)
underTest ! "throw"
probe.expectMsg(PreRestart)
probe.expectNoMessage()
@ -87,24 +87,24 @@ class UntypedSupervisingTypedSpec
}
"default to stop supervision (from context)" in {
val untyped = system.actorOf(u.Props(new UntypedToTyped()))
val classic = system.actorOf(u.Props(new ClassicToTyped()))
val probe = TestProbe()
untyped ! SpawnFromUntyped(ProbedBehavior.behavior(probe.ref), "a3")
val underTest = expectMsgType[TypedSpawnedFromUntypedConext].actorRef
watch(underTest.toUntyped)
classic ! SpawnFromClassic(ProbedBehavior.behavior(probe.ref), "a3")
val underTest = expectMsgType[TypedSpawnedFromClassicContext].actorRef
watch(underTest.toClassic)
underTest ! "throw"
probe.expectMsg(PostStop)
probe.expectNoMessage()
expectTerminated(underTest.toUntyped)
expectTerminated(underTest.toClassic)
}
"allow overriding the default (from context)" in {
val untyped = system.actorOf(u.Props(new UntypedToTyped()))
val classic = system.actorOf(u.Props(new ClassicToTyped()))
val probe = TestProbe()
val behavior = Behaviors.supervise(ProbedBehavior.behavior(probe.ref)).onFailure(SupervisorStrategy.restart)
untyped ! SpawnFromUntyped(behavior, "a4")
val underTest = expectMsgType[TypedSpawnedFromUntypedConext].actorRef
watch(underTest.toUntyped)
classic ! SpawnFromClassic(behavior, "a4")
val underTest = expectMsgType[TypedSpawnedFromClassicContext].actorRef
watch(underTest.toClassic)
underTest ! "throw"
probe.expectMsg(PreRestart)
probe.expectNoMessage()
@ -112,15 +112,15 @@ class UntypedSupervisingTypedSpec
}
"default to stop supervision for spawn anonymous (from context)" in {
val untyped = system.actorOf(u.Props(new UntypedToTyped()))
val classic = system.actorOf(u.Props(new ClassicToTyped()))
val probe = TestProbe()
untyped ! SpawnAnonFromUntyped(ProbedBehavior.behavior(probe.ref))
val underTest = expectMsgType[TypedSpawnedFromUntypedConext].actorRef
watch(underTest.toUntyped)
classic ! SpawnAnonFromClassic(ProbedBehavior.behavior(probe.ref))
val underTest = expectMsgType[TypedSpawnedFromClassicContext].actorRef
watch(underTest.toClassic)
underTest ! "throw"
probe.expectMsg(PostStop)
probe.expectNoMessage()
expectTerminated(underTest.toUntyped)
expectTerminated(underTest.toClassic)
}
}

View file

@ -10,21 +10,21 @@ import akka.actor.typed.ActorRef
import akka.actor.typed.scaladsl.Behaviors
import org.scalatest.WordSpecLike
import akka.actor.typed.scaladsl.adapter._
import akka.{ actor => untyped }
import akka.{ actor => classic }
object TypedSupervisingUntypedSpec {
object TypedSupervisingClassicSpec {
sealed trait Protocol
final case class SpawnUntypedActor(props: untyped.Props, replyTo: ActorRef[SpawnedUntypedActor]) extends Protocol
final case class SpawnedUntypedActor(ref: untyped.ActorRef)
final case class SpawnClassicActor(props: classic.Props, replyTo: ActorRef[SpawnedClassicActor]) extends Protocol
final case class SpawnedClassicActor(ref: classic.ActorRef)
def untypedActorOf() = Behaviors.receive[Protocol] {
case (ctx, SpawnUntypedActor(props, replyTo)) =>
replyTo ! SpawnedUntypedActor(ctx.actorOf(props))
def classicActorOf() = Behaviors.receive[Protocol] {
case (ctx, SpawnClassicActor(props, replyTo)) =>
replyTo ! SpawnedClassicActor(ctx.actorOf(props))
Behaviors.same
}
class UntypedActor(lifecycleProbe: ActorRef[String]) extends Actor {
class CLassicActor(lifecycleProbe: ActorRef[String]) extends Actor {
override def receive: Receive = {
case "throw" => throw TestException("oh dear")
}
@ -40,22 +40,22 @@ object TypedSupervisingUntypedSpec {
}
class TypedSupervisingUntypedSpec extends ScalaTestWithActorTestKit("""
class TypedSupervisingClassicSpec extends ScalaTestWithActorTestKit("""
akka.loglevel = INFO
""".stripMargin) with WordSpecLike {
import TypedSupervisingUntypedSpec._
import TypedSupervisingClassicSpec._
"Typed supervising untyped" should {
"Typed supervising classic" should {
"default to restart" in {
val ref: ActorRef[Protocol] = spawn(untypedActorOf())
val ref: ActorRef[Protocol] = spawn(classicActorOf())
val lifecycleProbe = TestProbe[String]
val probe = TestProbe[SpawnedUntypedActor]
ref ! SpawnUntypedActor(untyped.Props(new UntypedActor(lifecycleProbe.ref)), probe.ref)
val spawnedUntyped = probe.expectMessageType[SpawnedUntypedActor].ref
val probe = TestProbe[SpawnedClassicActor]
ref ! SpawnClassicActor(classic.Props(new CLassicActor(lifecycleProbe.ref)), probe.ref)
val spawnedClassic = probe.expectMessageType[SpawnedClassicActor].ref
lifecycleProbe.expectMessage("preStart")
spawnedUntyped ! "throw"
spawnedClassic ! "throw"
lifecycleProbe.expectMessage("postStop")
// should be restarted because it is an untyped actor
// should be restarted because it is a classic actor
lifecycleProbe.expectMessage("preStart")
}
}

View file

@ -29,7 +29,7 @@ object ActorRefSerializationSpec {
class ActorRefSerializationSpec extends ScalaTestWithActorTestKit(ActorRefSerializationSpec.config) with WordSpecLike {
val serialization = SerializationExtension(system.toUntyped)
val serialization = SerializationExtension(system.toClassic)
"ActorRef[T]" must {
"be serialized and deserialized by MiscMessageSerializer" in {

View file

@ -57,7 +57,7 @@ class ActorSystemSpec extends WordSpec with Matchers with BeforeAndAfterAll with
}
inbox.receiveAll() should ===("hello" :: Nil)
sys.whenTerminated.futureValue
CoordinatedShutdown(sys.toUntyped).shutdownReason() should ===(
CoordinatedShutdown(sys.toClassic).shutdownReason() should ===(
Some(CoordinatedShutdown.ActorSystemTerminateReason))
}
}
@ -93,7 +93,7 @@ class ActorSystemSpec extends WordSpec with Matchers with BeforeAndAfterAll with
// now we know that the guardian has started, and should receive PostStop
sys.terminate()
sys.whenTerminated.futureValue
CoordinatedShutdown(sys.toUntyped).shutdownReason() should ===(
CoordinatedShutdown(sys.toClassic).shutdownReason() should ===(
Some(CoordinatedShutdown.ActorSystemTerminateReason))
inbox.receiveAll() should ===("done" :: Nil)
}

View file

@ -15,7 +15,7 @@ class ServiceKeySerializationSpec
extends ScalaTestWithActorTestKit(ActorRefSerializationSpec.config)
with WordSpecLike {
val serialization = SerializationExtension(system.toUntyped)
val serialization = SerializationExtension(system.toClassic)
"ServiceKey[T]" must {
"be serialized and deserialized by ServiceKeySerializer" in {

View file

@ -33,7 +33,7 @@ object ActorContextAskSpec {
class ActorContextAskSpec extends ScalaTestWithActorTestKit(ActorContextAskSpec.config) with WordSpecLike {
implicit val untyped = system.toUntyped // FIXME #24348: eventfilter support in typed testkit
implicit val classic = system.toClassic // FIXME #24348: eventfilter support in testkit
"The Scala DSL ActorContext" must {

View file

@ -44,9 +44,9 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
val marker = LogMarker("marker")
val cause = new TestException("böö")
implicit val untyped = system.toUntyped
implicit val classic = system.toClassic
"Logging in a typed actor" must {
"Logging in an actor" must {
"be conveniently available from the context" in {
val actor =

View file

@ -33,7 +33,7 @@ object MessageAdapterSpec {
class MessageAdapterSpec extends ScalaTestWithActorTestKit(MessageAdapterSpec.config) with WordSpecLike {
implicit val untyped = system.toUntyped // FIXME #24348: eventfilter support in typed testkit
implicit val classic = system.toClassic // FIXME #24348: eventfilter support in testkit
"Message adapters" must {

View file

@ -25,7 +25,7 @@ class RoutersSpec extends ScalaTestWithActorTestKit("""
""") with WordSpecLike with Matchers {
// needed for the event filter
implicit val untypedSystem = system.toUntyped
implicit val classicSystem = system.toClassic
def compileOnlyApiCoverage(): Unit = {
Routers.group(ServiceKey[String]("key")).withRandomRouting().withRoundRobinRouting()
@ -152,7 +152,7 @@ class RoutersSpec extends ScalaTestWithActorTestKit("""
val serviceKey = ServiceKey[String]("group-routing-2")
val group = spawn(Routers.group(serviceKey), "group-router-2")
val probe = TestProbe[Dropped]()
system.toUntyped.eventStream.subscribe(probe.ref.toUntyped, classOf[Dropped])
system.toClassic.eventStream.subscribe(probe.ref.toClassic, classOf[Dropped])
(0 to 3).foreach { n =>
val msg = s"message-$n"

View file

@ -257,9 +257,9 @@ class UnstashingSpec extends ScalaTestWithActorTestKit("""
""") with WordSpecLike {
// needed for EventFilter
private implicit val untypedSys: akka.actor.ActorSystem = {
private implicit val classicSys: akka.actor.ActorSystem = {
import akka.actor.typed.scaladsl.adapter._
system.toUntyped
system.toClassic
}
private def slowStoppingChild(latch: CountDownLatch): Behavior[String] =
@ -638,7 +638,7 @@ class UnstashingSpec extends ScalaTestWithActorTestKit("""
"deal with stop" in {
val probe = TestProbe[Any]
import akka.actor.typed.scaladsl.adapter._
untypedSys.eventStream.subscribe(probe.ref.toUntyped, classOf[DeadLetter])
classicSys.eventStream.subscribe(probe.ref.toClassic, classOf[DeadLetter])
val ref = spawn(Behaviors.withStash[String](10) { stash =>
stash.stash("one")
stash.stash("two")

View file

@ -16,53 +16,53 @@ import akka.actor.typed.Terminated
import akka.testkit._
import akka.Done
import akka.NotUsed
import akka.{ actor => untyped }
import akka.{ actor => classic }
object AdapterSpec {
val untyped1: untyped.Props = untyped.Props(new Untyped1)
val classic1: classic.Props = classic.Props(new Classic1)
class Untyped1 extends untyped.Actor {
class Classic1 extends classic.Actor {
def receive = {
case "ping" => sender() ! "pong"
case t: ThrowIt => throw t
}
}
def untypedForwarder(ref: untyped.ActorRef): untyped.Props = untyped.Props(new UntypedForwarder(ref))
def classicForwarder(ref: classic.ActorRef): classic.Props = classic.Props(new ClassicForwarder(ref))
class UntypedForwarder(ref: untyped.ActorRef) extends untyped.Actor {
class ClassicForwarder(ref: classic.ActorRef) extends classic.Actor {
def receive = {
case a: String => ref ! a
}
}
def typed1(ref: untyped.ActorRef, probe: ActorRef[String]): Behavior[String] =
def typed1(ref: classic.ActorRef, probe: ActorRef[String]): Behavior[String] =
Behaviors
.receive[String] { (context, message) =>
message match {
case "send" =>
val replyTo = context.self.toUntyped
val replyTo = context.self.toClassic
ref.tell("ping", replyTo)
Behaviors.same
case "pong" =>
probe ! "ok"
Behaviors.same
case "actorOf" =>
val child = context.actorOf(untyped1)
child.tell("ping", context.self.toUntyped)
val child = context.actorOf(classic1)
child.tell("ping", context.self.toClassic)
Behaviors.same
case "watch" =>
context.watch(ref)
Behaviors.same
case "supervise-restart" =>
// restart is the default
val child = context.actorOf(untyped1)
val child = context.actorOf(classic1)
context.watch(child)
child ! ThrowIt3
child.tell("ping", context.self.toUntyped)
child.tell("ping", context.self.toClassic)
Behaviors.same
case "stop-child" =>
val child = context.actorOf(untyped1)
val child = context.actorOf(classic1)
context.watch(child)
context.stop(child)
Behaviors.same
@ -90,24 +90,24 @@ object AdapterSpec {
case object ThrowIt2 extends ThrowIt
case object ThrowIt3 extends ThrowIt
def untyped2(ref: ActorRef[Ping], probe: ActorRef[String]): untyped.Props =
untyped.Props(new Untyped2(ref, probe))
def classic2(ref: ActorRef[Ping], probe: ActorRef[String]): classic.Props =
classic.Props(new Classic2(ref, probe))
class Untyped2(ref: ActorRef[Ping], probe: ActorRef[String]) extends untyped.Actor {
class Classic2(ref: ActorRef[Ping], probe: ActorRef[String]) extends classic.Actor {
override val supervisorStrategy = untyped.OneForOneStrategy() {
override val supervisorStrategy = classic.OneForOneStrategy() {
({
case ThrowIt1 =>
probe ! "thrown-stop"
untyped.SupervisorStrategy.Stop
classic.SupervisorStrategy.Stop
case ThrowIt2 =>
probe ! "thrown-resume"
untyped.SupervisorStrategy.Resume
classic.SupervisorStrategy.Resume
case ThrowIt3 =>
probe ! "thrown-restart"
// TODO Restart will not really restart the behavior
untyped.SupervisorStrategy.Restart
}: untyped.SupervisorStrategy.Decider).orElse(untyped.SupervisorStrategy.defaultDecider)
classic.SupervisorStrategy.Restart
}: classic.SupervisorStrategy.Decider).orElse(classic.SupervisorStrategy.defaultDecider)
}
def receive = {
@ -122,7 +122,7 @@ object AdapterSpec {
child ! Ping(self)
case "watch" =>
context.watch(ref)
case untyped.Terminated(_) =>
case classic.Terminated(_) =>
probe ! "terminated"
case "supervise-stop" =>
testSupervice(ThrowIt1)
@ -157,7 +157,7 @@ object AdapterSpec {
}
}
def typed2Props: untyped.Props = PropsAdapter(typed2)
def typed2Props: classic.Props = PropsAdapter(typed2)
}
@ -181,7 +181,7 @@ class AdapterSpec extends AkkaSpec("""
system = ActorSystem.create(
Behaviors.setup[NotUsed](_ => Behaviors.stopped[NotUsed]),
"AdapterSpec-stopping-guardian")
} finally if (system != null) shutdown(system.toUntyped)
} finally if (system != null) shutdown(system.toClassic)
}
}
@ -197,84 +197,84 @@ class AdapterSpec extends AkkaSpec("""
}, "AdapterSpec-stopping-guardian-2")
} finally if (system != null) shutdown(system.toUntyped)
} finally if (system != null) shutdown(system.toClassic)
}
}
}
"Adapted actors" must {
"send message from typed to untyped" in {
"send message from typed to classic" in {
val probe = TestProbe()
val untypedRef = system.actorOf(untyped1)
val typedRef = system.spawnAnonymous(typed1(untypedRef, probe.ref))
val classicRef = system.actorOf(classic1)
val typedRef = system.spawnAnonymous(typed1(classicRef, probe.ref))
typedRef ! "send"
probe.expectMsg("ok")
}
"not send null message from typed to untyped" in {
"not send null message from typed to classic" in {
val probe = TestProbe()
val untypedRef = system.actorOf(untyped1)
val typedRef = system.spawnAnonymous(typed1(untypedRef, probe.ref))
val classicRef = system.actorOf(classic1)
val typedRef = system.spawnAnonymous(typed1(classicRef, probe.ref))
intercept[InvalidMessageException] {
typedRef ! null
}
}
"send message from untyped to typed" in {
"send message from classic to typed" in {
val probe = TestProbe()
val typedRef = system.spawnAnonymous(typed2)
val untypedRef = system.actorOf(untyped2(typedRef, probe.ref))
untypedRef ! "send"
val classicRef = system.actorOf(classic2(typedRef, probe.ref))
classicRef ! "send"
probe.expectMsg("ok")
}
"spawn typed child from untyped parent" in {
"spawn typed child from classic parent" in {
val probe = TestProbe()
val ign = system.spawnAnonymous(Behaviors.ignore[Ping])
val untypedRef = system.actorOf(untyped2(ign, probe.ref))
untypedRef ! "spawn"
val classicRef = system.actorOf(classic2(ign, probe.ref))
classicRef ! "spawn"
probe.expectMsg("ok")
}
"actorOf typed child via Props from untyped parent" in {
"actorOf typed child via Props from classic parent" in {
val probe = TestProbe()
val ign = system.spawnAnonymous(Behaviors.ignore[Ping])
val untypedRef = system.actorOf(untyped2(ign, probe.ref))
untypedRef ! "actorOf-props"
val classicRef = system.actorOf(classic2(ign, probe.ref))
classicRef ! "actorOf-props"
probe.expectMsg("ok")
}
"actorOf untyped child from typed parent" in {
"actorOf classic child from typed parent" in {
val probe = TestProbe()
val ignore = system.actorOf(untyped.Props.empty)
val ignore = system.actorOf(classic.Props.empty)
val typedRef = system.spawnAnonymous(typed1(ignore, probe.ref))
typedRef ! "actorOf"
probe.expectMsg("ok")
}
"watch typed from untyped" in {
"watch typed from classic" in {
val probe = TestProbe()
val typedRef = system.spawnAnonymous(typed2)
val untypedRef = system.actorOf(untyped2(typedRef, probe.ref))
untypedRef ! "watch"
val classicRef = system.actorOf(classic2(typedRef, probe.ref))
classicRef ! "watch"
typedRef ! StopIt
probe.expectMsg("terminated")
}
"watch untyped from typed" in {
"watch classic from typed" in {
val probe = TestProbe()
val untypedRef = system.actorOf(untyped1)
val typedRef = system.spawnAnonymous(typed1(untypedRef, probe.ref))
val classicRef = system.actorOf(classic1)
val typedRef = system.spawnAnonymous(typed1(classicRef, probe.ref))
typedRef ! "watch"
untypedRef ! untyped.PoisonPill
classicRef ! classic.PoisonPill
probe.expectMsg("terminated")
}
"supervise untyped child from typed parent" in {
// FIXME there's a warning with null logged from the untyped empty child here, where does that come from?
"supervise classic child from typed parent" in {
// FIXME there's a warning with null logged from the classic empty child here, where does that come from?
val probe = TestProbe()
val ignore = system.actorOf(untyped.Props.empty)
val ignore = system.actorOf(classic.Props.empty)
val typedRef = system.spawnAnonymous(typed1(ignore, probe.ref))
// only stop supervisorStrategy
@ -284,17 +284,17 @@ class AdapterSpec extends AkkaSpec("""
}
}
"stop typed child from untyped parent" in {
"stop typed child from classic parent" in {
val probe = TestProbe()
val ignore = system.spawnAnonymous(Behaviors.ignore[Ping])
val untypedRef = system.actorOf(untyped2(ignore, probe.ref))
untypedRef ! "stop-child"
val classicRef = system.actorOf(classic2(ignore, probe.ref))
classicRef ! "stop-child"
probe.expectMsg("terminated")
}
"stop untyped child from typed parent" in {
"stop classic child from typed parent" in {
val probe = TestProbe()
val ignore = system.actorOf(untyped.Props.empty)
val ignore = system.actorOf(classic.Props.empty)
val typedRef = system.spawnAnonymous(typed1(ignore, probe.ref))
typedRef ! "stop-child"
probe.expectMsg("terminated")

View file

@ -42,7 +42,7 @@ class GuardianStartupSpec extends WordSpec with Matchers with ScalaFutures {
var system: ActorSystem[String] = null
val initialized = new CountDownLatch(1)
val guardianBehavior = Behaviors.setup[String] { ctx =>
ctx.system.toUntyped.asInstanceOf[ActorSystemImpl].assertInitialized()
ctx.system.toClassic.asInstanceOf[ActorSystemImpl].assertInitialized()
initialized.countDown()
Behaviors.empty
}

View file

@ -9,24 +9,24 @@ import akka.actor.typed._
import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.TestKit
//#adapter-import
// adds support for typed actors to an untyped actor system and context
// adds support for actors to a classic actor system and context
import akka.actor.typed.scaladsl.adapter._
//#adapter-import
import akka.testkit.TestProbe
//#import-alias
import akka.{ actor => untyped }
import akka.{ actor => classic }
//#import-alias
import org.scalatest.WordSpec
import scala.concurrent.duration._
object UntypedWatchingTypedSpec {
object Untyped {
def props() = untyped.Props(new Untyped)
object ClassicWatchingTypedSpec {
object Classic {
def props() = classic.Props(new Classic)
}
//#untyped-watch
class Untyped extends untyped.Actor with ActorLogging {
//#classic-watch
class Classic extends classic.Actor with ActorLogging {
// context.spawn is an implicit extension method
val second: ActorRef[Typed.Command] =
context.spawn(Typed(), "second")
@ -44,12 +44,12 @@ object UntypedWatchingTypedSpec {
log.info(s"$self got Pong from ${sender()}")
// context.stop is an implicit extension method
context.stop(second)
case untyped.Terminated(ref) =>
case classic.Terminated(ref) =>
log.info(s"$self observed termination of $ref")
context.stop(self)
}
}
//#untyped-watch
//#classic-watch
//#typed
object Typed {
@ -62,7 +62,7 @@ object UntypedWatchingTypedSpec {
message match {
case Ping(replyTo) =>
context.log.info(s"${context.self} got Ping from $replyTo")
// replyTo is an untyped actor that has been converted for coexistence
// replyTo is a classic actor that has been converted for coexistence
replyTo ! Pong
Behaviors.same
}
@ -71,28 +71,28 @@ object UntypedWatchingTypedSpec {
//#typed
}
class UntypedWatchingTypedSpec extends WordSpec {
class ClassicWatchingTypedSpec extends WordSpec {
import UntypedWatchingTypedSpec._
import ClassicWatchingTypedSpec._
"Untyped -> Typed" must {
"Classic -> Typed" must {
"support creating, watching and messaging" in {
val system = untyped.ActorSystem("Coexistence")
//#create-untyped
val untypedActor = system.actorOf(Untyped.props())
//#create-untyped
val system = classic.ActorSystem("Coexistence")
//#create-classic
val classicActor = system.actorOf(Classic.props())
//#create-classic
val probe = TestProbe()(system)
probe.watch(untypedActor)
probe.expectTerminated(untypedActor, 200.millis)
probe.watch(classicActor)
probe.expectTerminated(classicActor, 200.millis)
TestKit.shutdownActorSystem(system)
}
"support converting an untyped actor system to a typed actor system" in {
//#convert-untyped
"support converting a classic actor system to an actor system" in {
//#convert-classic
val system = akka.actor.ActorSystem("UntypedToTypedSystem")
val system = akka.actor.ActorSystem("ClassicToTypedSystem")
val typedSystem: ActorSystem[Nothing] = system.toTyped
//#convert-untyped
//#convert-classic
typedSystem.scheduler // remove compile warning
TestKit.shutdownActorSystem(system)
}

View file

@ -7,19 +7,19 @@ package docs.akka.typed.coexistence
import akka.actor.typed._
import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.TestKit
import docs.akka.typed.coexistence.TypedWatchingUntypedSpec.Typed
import docs.akka.typed.coexistence.TypedWatchingClassicSpec.Typed
//#adapter-import
// adds support for typed actors to an untyped actor system and context
// adds support for typed actors to a classic actor system and context
import akka.actor.typed.scaladsl.adapter._
//#adapter-import
import akka.testkit.TestProbe
//#import-alias
import akka.{ actor => untyped }
import akka.{ actor => classic }
//#import-alias
import org.scalatest.WordSpec
import scala.concurrent.duration._
object TypedWatchingUntypedSpec {
object TypedWatchingClassicSpec {
//#typed
object Typed {
@ -30,20 +30,20 @@ object TypedWatchingUntypedSpec {
val behavior: Behavior[Command] =
Behaviors.setup { context =>
// context.actorOf is an implicit extension method
val untyped = context.actorOf(Untyped.props(), "second")
val classic = context.actorOf(Classic.props(), "second")
// context.watch is an implicit extension method
context.watch(untyped)
context.watch(classic)
// illustrating how to pass sender, toUntyped is an implicit extension method
untyped.tell(Typed.Ping(context.self), context.self.toUntyped)
// illustrating how to pass sender, toClassic is an implicit extension method
classic.tell(Typed.Ping(context.self), context.self.toClassic)
Behaviors
.receivePartial[Command] {
case (context, Pong) =>
// it's not possible to get the sender, that must be sent in message
// context.stop is an implicit extension method
context.stop(untyped)
context.stop(classic)
Behaviors.same
}
.receiveSignal {
@ -54,32 +54,32 @@ object TypedWatchingUntypedSpec {
}
//#typed
//#untyped
object Untyped {
def props(): untyped.Props = untyped.Props(new Untyped)
//#classic
object Classic {
def props(): classic.Props = classic.Props(new Classic)
}
class Untyped extends untyped.Actor {
class Classic extends classic.Actor {
override def receive = {
case Typed.Ping(replyTo) =>
replyTo ! Typed.Pong
}
}
//#untyped
//#classic
}
class TypedWatchingUntypedSpec extends WordSpec {
class TypedWatchingClassicSpec extends WordSpec {
import TypedWatchingUntypedSpec._
import TypedWatchingClassicSpec._
"Typed -> Untyped" must {
"Typed -> Classic" must {
"support creating, watching and messaging" in {
//#create
val system = untyped.ActorSystem("TypedWatchingUntyped")
val system = classic.ActorSystem("TypedWatchingClassic")
val typed = system.spawn(Typed.behavior, "Typed")
//#create
val probe = TestProbe()(system)
probe.watch(typed.toUntyped)
probe.expectTerminated(typed.toUntyped, 200.millis)
probe.watch(typed.toClassic)
probe.expectTerminated(typed.toClassic, 200.millis)
TestKit.shutdownActorSystem(system)
}
}

View file

@ -25,7 +25,7 @@ akka.actor.typed {
restart-stash-capacity = 1000
}
# Load typed extensions by an untyped unextension.
# Load typed extensions by a classic extension.
akka.library-extensions += "akka.actor.typed.internal.adapter.ActorSystemAdapter$LoadTypedExtensions"
akka.actor {

View file

@ -5,7 +5,7 @@
package akka.actor.typed
import akka.annotation.DoNotInherit
import akka.{ actor => untyped }
import akka.{ actor => classic }
import scala.annotation.unchecked.uncheckedVariance
import akka.actor.typed.internal.InternalRecipientRef
@ -50,7 +50,7 @@ trait ActorRef[-T] extends RecipientRef[T] with java.lang.Comparable[ActorRef[_]
* and more than one Actor instance can exist with the same path at different
* points in time, but not concurrently.
*/
def path: untyped.ActorPath
def path: classic.ActorPath
@throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = SerializedActorRef[T](this)

View file

@ -47,13 +47,13 @@ abstract class ActorRefResolver extends Extension {
@InternalApi private[akka] class ActorRefResolverImpl(system: ActorSystem[_]) extends ActorRefResolver {
import akka.actor.typed.scaladsl.adapter._
private val untypedSystem = system.toUntyped.asInstanceOf[ExtendedActorSystem]
private val classicSystem = system.toClassic.asInstanceOf[ExtendedActorSystem]
override def toSerializationFormat[T](ref: ActorRef[T]): String =
ref.path.toSerializationFormatWithAddress(untypedSystem.provider.getDefaultAddress)
ref.path.toSerializationFormatWithAddress(classicSystem.provider.getDefaultAddress)
override def resolveActorRef[T](serializedActorRef: String): ActorRef[T] =
untypedSystem.provider.resolveActorRef(serializedActorRef)
classicSystem.provider.resolveActorRef(serializedActorRef)
}
object ActorRefResolverSetup {

View file

@ -15,7 +15,7 @@ import akka.actor.typed.internal.adapter.{ ActorSystemAdapter, GuardianStartupBe
import akka.actor.typed.receptionist.Receptionist
import akka.annotation.DoNotInherit
import akka.util.Helpers.Requiring
import akka.{ Done, actor => untyped }
import akka.{ Done, actor => classic }
import com.typesafe.config.{ Config, ConfigFactory }
import scala.concurrent.{ ExecutionContextExecutor, Future }
@ -79,7 +79,7 @@ abstract class ActorSystem[-T] extends ActorRef[T] with Extensions with ClassicA
* set on all threads created by the ActorSystem, if one was set during
* creation.
*/
def dynamicAccess: untyped.DynamicAccess
def dynamicAccess: classic.DynamicAccess
/**
* A generic scheduler that can initiate the execution of tasks after some delay.
@ -227,9 +227,9 @@ object ActorSystem {
create(guardianBehavior, name, ActorSystemSetup.create(bootstrapSetup))
/**
* Create an ActorSystem based on the untyped [[akka.actor.ActorSystem]]
* Create an ActorSystem based on the classic [[akka.actor.ActorSystem]]
* which runs Akka Typed [[Behavior]] on an emulation layer. In this
* system typed and untyped actors can coexist.
* system typed and classic actors can coexist.
*/
private def createInternal[T](
name: String,
@ -245,7 +245,7 @@ object ActorSystem {
val appConfig = bootstrapSettings.flatMap(_.config).getOrElse(ConfigFactory.load(cl))
val executionContext = bootstrapSettings.flatMap(_.defaultExecutionContext)
val system = new untyped.ActorSystemImpl(
val system = new classic.ActorSystemImpl(
name,
appConfig,
cl,
@ -259,28 +259,28 @@ object ActorSystem {
}
/**
* Wrap an untyped [[akka.actor.ActorSystem]] such that it can be used from
* Wrap a classic [[akka.actor.ActorSystem]] such that it can be used from
* Akka Typed [[Behavior]].
*/
def wrap(system: untyped.ActorSystem): ActorSystem[Nothing] =
ActorSystemAdapter.AdapterExtension(system.asInstanceOf[untyped.ActorSystemImpl]).adapter
def wrap(system: classic.ActorSystem): ActorSystem[Nothing] =
ActorSystemAdapter.AdapterExtension(system.asInstanceOf[classic.ActorSystemImpl]).adapter
}
/**
* The configuration settings that were parsed from the config by an [[ActorSystem]].
* This class is immutable.
*/
final class Settings(val config: Config, val untypedSettings: untyped.ActorSystem.Settings, val name: String) {
final class Settings(val config: Config, val classicSettings: classic.ActorSystem.Settings, val name: String) {
def this(classLoader: ClassLoader, config: Config, name: String) =
this({
val cfg = config.withFallback(ConfigFactory.defaultReference(classLoader))
cfg.checkValid(ConfigFactory.defaultReference(classLoader), "akka")
cfg
}, new untyped.ActorSystem.Settings(classLoader, config, name), name)
}, new classic.ActorSystem.Settings(classLoader, config, name), name)
def this(settings: untyped.ActorSystem.Settings) = this(settings.config, settings, settings.name)
def this(settings: classic.ActorSystem.Settings) = this(settings.config, settings, settings.name)
def setup: ActorSystemSetup = untypedSettings.setup
def setup: ActorSystemSetup = classicSettings.setup
/**
* Returns the String representation of the Config that this Settings is backed by

View file

@ -16,7 +16,7 @@ import akka.annotation.DoNotInherit
* The typical usage of this is to use it as the guardian actor of the [[ActorSystem]], possibly combined with
* `Behaviors.setup` to starts some initial tasks or actors. Child actors can then be started from the outside
* by telling or asking [[SpawnProtocol#Spawn]] to the actor reference of the system. When using `ask` this is
* similar to how [[akka.actor.ActorSystem#actorOf]] can be used in untyped actors with the difference that
* similar to how [[akka.actor.ActorSystem#actorOf]] can be used in classic actors with the difference that
* a `Future` / `CompletionStage` of the `ActorRef` is returned.
*
* Stopping children is done through specific support in the protocol of the children, or stopping the entire

View file

@ -17,7 +17,7 @@ class MiscMessageSerializer(val system: akka.actor.ExtendedActorSystem)
extends SerializerWithStringManifest
with BaseSerializer {
// Serializers are initialized early on. `toTyped` might then try to initialize the untyped ActorSystemAdapter extension.
// Serializers are initialized early on. `toTyped` might then try to initialize the classic ActorSystemAdapter extension.
private lazy val resolver = ActorRefResolver(system.toTyped)
private val ActorRefManifest = "a"

View file

@ -14,7 +14,7 @@ import akka.annotation.InternalApi
/**
* INTERNAL API
*
* Note that this is a `Signal` poison pill, not a universal poison pill like the untyped actor one.
* Note that this is a `Signal` poison pill, not a universal poison pill like the classic actor one.
* This requires special handling on the receiving side where it is used (for example with the interceptor below).
*/
@InternalApi private[akka] sealed abstract class PoisonPill extends Signal

View file

@ -186,9 +186,9 @@ import akka.util.{ unused, ConstantFun }
private def unstashRestToDeadLetters(ctx: TypedActorContext[T], messages: Iterator[T]): Unit = {
val scalaCtx = ctx.asScala
import akka.actor.typed.scaladsl.adapter._
val untypedDeadLetters = scalaCtx.system.deadLetters.toUntyped
val classicDeadLetters = scalaCtx.system.deadLetters.toClassic
messages.foreach(msg =>
scalaCtx.system.deadLetters ! DeadLetter(msg, untypedDeadLetters, ctx.asScala.self.toUntyped))
scalaCtx.system.deadLetters ! DeadLetter(msg, classicDeadLetters, ctx.asScala.self.toClassic))
}
override def unstash(behavior: Behavior[T], numberOfMessages: Int, wrap: JFunction[T, T]): Behavior[T] =

View file

@ -91,8 +91,8 @@ private abstract class AbstractSupervisor[I, Thr <: Throwable](strategy: Supervi
def dropped(ctx: TypedActorContext[_], signalOrMessage: Any): Unit = {
import akka.actor.typed.scaladsl.adapter._
ctx.asScala.system.toUntyped.eventStream
.publish(Dropped(signalOrMessage, s"Stash is full in [${getClass.getSimpleName}]", ctx.asScala.self.toUntyped))
ctx.asScala.system.toClassic.eventStream
.publish(Dropped(signalOrMessage, s"Stash is full in [${getClass.getSimpleName}]", ctx.asScala.self.toClassic))
}
protected def handleExceptionOnStart(ctx: TypedActorContext[Any], target: PreStartTarget[I]): Catcher[Behavior[I]]

View file

@ -9,7 +9,7 @@ package adapter
import java.lang.reflect.InvocationTargetException
import akka.actor.{ ActorInitializationException, ActorRefWithCell }
import akka.{ actor => untyped }
import akka.{ actor => classic }
import akka.actor.typed.internal.BehaviorImpl.DeferredBehavior
import akka.actor.typed.internal.BehaviorImpl.StoppedBehavior
import akka.actor.typed.internal.adapter.ActorAdapter.TypedActorFailedException
@ -39,7 +39,7 @@ import akka.util.OptionVal
*/
final case class TypedActorFailedException(cause: Throwable) extends RuntimeException
private val DummyReceive: untyped.Actor.Receive = {
private val DummyReceive: classic.Actor.Receive = {
case _ => throw new RuntimeException("receive should never be called on the typed ActorAdapter")
}
@ -49,8 +49,8 @@ import akka.util.OptionVal
* INTERNAL API
*/
@InternalApi private[typed] final class ActorAdapter[T](_initialBehavior: Behavior[T], rethrowTypedFailure: Boolean)
extends untyped.Actor
with untyped.ActorLogging {
extends classic.Actor
with classic.ActorLogging {
private var behavior: Behavior[T] = _initialBehavior
def currentBehavior: Behavior[T] = behavior
@ -64,19 +64,19 @@ import akka.util.OptionVal
}
/**
* Failures from failed children, that were stopped through untyped supervision, this is what allows us to pass
* Failures from failed children, that were stopped through classic supervision, this is what allows us to pass
* child exception in Terminated for direct children.
*/
private var failures: Map[untyped.ActorRef, Throwable] = Map.empty
private var failures: Map[classic.ActorRef, Throwable] = Map.empty
def receive: Receive = ActorAdapter.DummyReceive
override protected[akka] def aroundReceive(receive: Receive, msg: Any): Unit = {
// as we know we never become in "normal" typed actors, it is just the current behavior that
// changes, we can avoid some overhead with the partial function/behavior stack of untyped entirely
// changes, we can avoid some overhead with the partial function/behavior stack of classic entirely
// we also know that the receive is total, so we can avoid the orElse part as well.
msg match {
case untyped.Terminated(ref) =>
case classic.Terminated(ref) =>
val msg =
if (failures contains ref) {
val ex = failures(ref)
@ -84,7 +84,7 @@ import akka.util.OptionVal
ChildFailed(ActorRefAdapter(ref), ex)
} else Terminated(ActorRefAdapter(ref))
handleSignal(msg)
case untyped.ReceiveTimeout =>
case classic.ReceiveTimeout =>
handleMessage(ctx.receiveTimeoutMsg)
case wrapped: AdaptMessage[Any, T] @unchecked =>
withSafelyAdapted(() => wrapped.adapt()) {
@ -146,7 +146,7 @@ import akka.util.OptionVal
unhandled(msg)
case BehaviorTags.FailedBehavior =>
val f = b.asInstanceOf[BehaviorImpl.FailedBehavior]
// For the parent untyped supervisor to pick up the exception
// For the parent classic supervisor to pick up the exception
if (rethrowTypedFailure) throw TypedActorFailedException(f.cause)
else context.stop(self)
case BehaviorTags.StoppedBehavior =>
@ -196,11 +196,11 @@ import akka.util.OptionVal
super.unhandled(other)
}
override val supervisorStrategy = untyped.OneForOneStrategy(loggingEnabled = false) {
override val supervisorStrategy = classic.OneForOneStrategy(loggingEnabled = false) {
case TypedActorFailedException(cause) =>
// These have already been optionally logged by typed supervision
recordChildFailure(cause)
untyped.SupervisorStrategy.Stop
classic.SupervisorStrategy.Stop
case ex =>
val isTypedActor = sender() match {
case afwc: ActorRefWithCell =>
@ -220,14 +220,14 @@ import akka.util.OptionVal
// log at Error as that is what the supervision strategy would have done.
log.error(ex, logMessage)
if (isTypedActor)
untyped.SupervisorStrategy.Stop
classic.SupervisorStrategy.Stop
else
untyped.SupervisorStrategy.Restart
classic.SupervisorStrategy.Restart
}
private def recordChildFailure(ex: Throwable): Unit = {
val ref = sender()
if (context.asInstanceOf[untyped.ActorCell].isWatching(ref)) {
if (context.asInstanceOf[classic.ActorCell].isWatching(ref)) {
failures = failures.updated(ref, ex)
}
}

View file

@ -11,34 +11,34 @@ import akka.annotation.InternalApi
import akka.event.LoggingFilterWithMarker
import akka.util.OptionVal
import akka.{ actor => untyped }
import akka.{ actor => classic }
import scala.concurrent.ExecutionContextExecutor
import scala.concurrent.duration._
@InternalApi
private[akka] object ActorContextAdapter {
private def toUntypedImp[U](context: TypedActorContext[_]): untyped.ActorContext =
private def toClassicImp[U](context: TypedActorContext[_]): classic.ActorContext =
context match {
case adapter: ActorContextAdapter[_] => adapter.untypedContext
case adapter: ActorContextAdapter[_] => adapter.classicContext
case _ =>
throw new UnsupportedOperationException(
"only adapted untyped ActorContext permissible " +
"Only adapted classic ActorContext permissible " +
s"($context of class ${context.getClass.getName})")
}
def toUntyped[U](context: scaladsl.ActorContext[_]): untyped.ActorContext =
def toClassic[U](context: scaladsl.ActorContext[_]): classic.ActorContext =
context match {
case c: TypedActorContext[_] => toUntypedImp(c)
case c: TypedActorContext[_] => toClassicImp(c)
case _ =>
throw new UnsupportedOperationException(
"unknown ActorContext type " +
s"($context of class ${context.getClass.getName})")
}
def toUntyped[U](context: javadsl.ActorContext[_]): untyped.ActorContext =
def toClassic[U](context: javadsl.ActorContext[_]): classic.ActorContext =
context match {
case c: TypedActorContext[_] => toUntypedImp(c)
case c: TypedActorContext[_] => toClassicImp(c)
case _ =>
throw new UnsupportedOperationException(
"unknown ActorContext type " +
@ -50,35 +50,35 @@ private[akka] object ActorContextAdapter {
* INTERNAL API. Wrapping an [[akka.actor.ActorContext]] as an [[TypedActorContext]].
*/
@InternalApi private[akka] final class ActorContextAdapter[T](
val untypedContext: untyped.ActorContext,
val classicContext: classic.ActorContext,
adapter: ActorAdapter[T])
extends ActorContextImpl[T] {
import ActorRefAdapter.toUntyped
import ActorRefAdapter.toClassic
private[akka] override def currentBehavior: Behavior[T] = adapter.currentBehavior
// lazily initialized
private var actorLogger: OptionVal[Logger] = OptionVal.None
final override val self = ActorRefAdapter(untypedContext.self)
final override val system = ActorSystemAdapter(untypedContext.system)
override def children: Iterable[ActorRef[Nothing]] = untypedContext.children.map(ActorRefAdapter(_))
override def child(name: String): Option[ActorRef[Nothing]] = untypedContext.child(name).map(ActorRefAdapter(_))
final override val self = ActorRefAdapter(classicContext.self)
final override val system = ActorSystemAdapter(classicContext.system)
override def children: Iterable[ActorRef[Nothing]] = classicContext.children.map(ActorRefAdapter(_))
override def child(name: String): Option[ActorRef[Nothing]] = classicContext.child(name).map(ActorRefAdapter(_))
override def spawnAnonymous[U](behavior: Behavior[U], props: Props = Props.empty): ActorRef[U] =
ActorRefFactoryAdapter.spawnAnonymous(untypedContext, behavior, props, rethrowTypedFailure = true)
ActorRefFactoryAdapter.spawnAnonymous(classicContext, behavior, props, rethrowTypedFailure = true)
override def spawn[U](behavior: Behavior[U], name: String, props: Props = Props.empty): ActorRef[U] =
ActorRefFactoryAdapter.spawn(untypedContext, behavior, name, props, rethrowTypedFailure = true)
ActorRefFactoryAdapter.spawn(classicContext, behavior, name, props, rethrowTypedFailure = true)
override def stop[U](child: ActorRef[U]): Unit =
if (child.path.parent == self.path) { // only if a direct child
toUntyped(child) match {
toClassic(child) match {
case f: akka.actor.FunctionRef =>
val cell = untypedContext.asInstanceOf[akka.actor.ActorCell]
val cell = classicContext.asInstanceOf[akka.actor.ActorCell]
cell.removeFunctionRef(f)
case c =>
untypedContext.child(child.path.name) match {
classicContext.child(child.path.name) match {
case Some(`c`) =>
untypedContext.stop(c)
classicContext.stop(c)
case _ =>
// child that was already stopped
}
@ -96,33 +96,33 @@ private[akka] object ActorContextAdapter {
"an explicit stop message that the actor accepts.")
}
override def watch[U](other: ActorRef[U]): Unit = { untypedContext.watch(toUntyped(other)) }
override def watchWith[U](other: ActorRef[U], msg: T): Unit = { untypedContext.watchWith(toUntyped(other), msg) }
override def unwatch[U](other: ActorRef[U]): Unit = { untypedContext.unwatch(toUntyped(other)) }
override def watch[U](other: ActorRef[U]): Unit = { classicContext.watch(toClassic(other)) }
override def watchWith[U](other: ActorRef[U], msg: T): Unit = { classicContext.watchWith(toClassic(other), msg) }
override def unwatch[U](other: ActorRef[U]): Unit = { classicContext.unwatch(toClassic(other)) }
var receiveTimeoutMsg: T = null.asInstanceOf[T]
override def setReceiveTimeout(d: FiniteDuration, msg: T): Unit = {
receiveTimeoutMsg = msg
untypedContext.setReceiveTimeout(d)
classicContext.setReceiveTimeout(d)
}
override def cancelReceiveTimeout(): Unit = {
receiveTimeoutMsg = null.asInstanceOf[T]
untypedContext.setReceiveTimeout(Duration.Undefined)
classicContext.setReceiveTimeout(Duration.Undefined)
}
override def executionContext: ExecutionContextExecutor = untypedContext.dispatcher
override def scheduleOnce[U](delay: FiniteDuration, target: ActorRef[U], msg: U): untyped.Cancellable = {
import untypedContext.dispatcher
untypedContext.system.scheduler.scheduleOnce(delay, toUntyped(target), msg)
override def executionContext: ExecutionContextExecutor = classicContext.dispatcher
override def scheduleOnce[U](delay: FiniteDuration, target: ActorRef[U], msg: U): classic.Cancellable = {
import classicContext.dispatcher
classicContext.system.scheduler.scheduleOnce(delay, toClassic(target), msg)
}
override private[akka] def internalSpawnMessageAdapter[U](f: U => T, _name: String): ActorRef[U] = {
val cell = untypedContext.asInstanceOf[akka.actor.ActorCell]
val cell = classicContext.asInstanceOf[akka.actor.ActorCell]
// apply the function inside the actor by wrapping the msg and f, handled by ActorAdapter
val ref = cell.addFunctionRef((_, msg) => untypedContext.self ! AdaptMessage[U, T](msg.asInstanceOf[U], f), _name)
val ref = cell.addFunctionRef((_, msg) => classicContext.self ! AdaptMessage[U, T](msg.asInstanceOf[U], f), _name)
ActorRefAdapter[U](ref)
}
private def initLoggerWithClass(logClass: Class[_]): LoggerAdapterImpl = {
val logSource = self.path.toString
val system = untypedContext.system.asInstanceOf[ExtendedActorSystem]
val system = classicContext.system.asInstanceOf[ExtendedActorSystem]
val logger =
new LoggerAdapterImpl(system.eventStream, logClass, logSource, LoggingFilterWithMarker.wrap(system.logFilter))
actorLogger = OptionVal.Some(logger)

View file

@ -8,63 +8,63 @@ package adapter
import akka.actor.ActorRefProvider
import akka.actor.InvalidMessageException
import akka.{ actor => untyped }
import akka.{ actor => classic }
import akka.annotation.InternalApi
import akka.dispatch.sysmsg
/**
* INTERNAL API
*/
@InternalApi private[typed] class ActorRefAdapter[-T](val untypedRef: untyped.InternalActorRef)
@InternalApi private[typed] class ActorRefAdapter[-T](val classicRef: classic.InternalActorRef)
extends ActorRef[T]
with internal.ActorRefImpl[T]
with internal.InternalRecipientRef[T] {
override def path: untyped.ActorPath = untypedRef.path
override def path: classic.ActorPath = classicRef.path
override def tell(msg: T): Unit = {
if (msg == null) throw new InvalidMessageException("[null] is not an allowed message")
untypedRef ! msg
classicRef ! msg
}
// impl ActorRefImpl
override def isLocal: Boolean = untypedRef.isLocal
override def isLocal: Boolean = classicRef.isLocal
// impl ActorRefImpl
override def sendSystem(signal: internal.SystemMessage): Unit =
ActorRefAdapter.sendSystemMessage(untypedRef, signal)
ActorRefAdapter.sendSystemMessage(classicRef, signal)
// impl InternalRecipientRef
override def provider: ActorRefProvider = untypedRef.provider
override def provider: ActorRefProvider = classicRef.provider
// impl InternalRecipientRef
def isTerminated: Boolean = untypedRef.isTerminated
def isTerminated: Boolean = classicRef.isTerminated
@throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = SerializedActorRef[T](this)
}
private[akka] object ActorRefAdapter {
def apply[T](ref: untyped.ActorRef): ActorRef[T] = new ActorRefAdapter(ref.asInstanceOf[untyped.InternalActorRef])
def apply[T](ref: classic.ActorRef): ActorRef[T] = new ActorRefAdapter(ref.asInstanceOf[classic.InternalActorRef])
def toUntyped[U](ref: ActorRef[U]): akka.actor.InternalActorRef =
def toClassic[U](ref: ActorRef[U]): akka.actor.InternalActorRef =
ref match {
case adapter: ActorRefAdapter[_] => adapter.untypedRef
case system: ActorSystemAdapter[_] => system.untypedSystem.guardian
case adapter: ActorRefAdapter[_] => adapter.classicRef
case adapter: ActorSystemAdapter[_] => adapter.system.guardian
case _ =>
throw new UnsupportedOperationException(
"only adapted untyped ActorRefs permissible " +
"Only adapted classic ActorRefs permissible " +
s"($ref of class ${ref.getClass.getName})")
}
def sendSystemMessage(untypedRef: akka.actor.InternalActorRef, signal: internal.SystemMessage): Unit =
def sendSystemMessage(classicRef: akka.actor.InternalActorRef, signal: internal.SystemMessage): Unit =
signal match {
case internal.Create() => throw new IllegalStateException("WAT? No, seriously.")
case internal.Terminate() => untypedRef.stop()
case internal.Terminate() => classicRef.stop()
case internal.Watch(watchee, watcher) =>
untypedRef.sendSystemMessage(sysmsg.Watch(toUntyped(watchee), toUntyped(watcher)))
classicRef.sendSystemMessage(sysmsg.Watch(toClassic(watchee), toClassic(watcher)))
case internal.Unwatch(watchee, watcher) =>
untypedRef.sendSystemMessage(sysmsg.Unwatch(toUntyped(watchee), toUntyped(watcher)))
classicRef.sendSystemMessage(sysmsg.Unwatch(toClassic(watchee), toClassic(watcher)))
case internal.DeathWatchNotification(ref, _) =>
untypedRef.sendSystemMessage(sysmsg.DeathWatchNotification(toUntyped(ref), true, false))
classicRef.sendSystemMessage(sysmsg.DeathWatchNotification(toClassic(ref), true, false))
case internal.NoMessage => // just to suppress the warning
}
}

View file

@ -32,104 +32,104 @@ import akka.actor.typed.internal.PropsImpl.DispatcherSameAsParent
import akka.actor.typed.internal.SystemMessage
import akka.annotation.InternalApi
import akka.event.LoggingFilterWithMarker
import akka.{ actor => untyped }
import akka.{ actor => classic }
/**
* INTERNAL API. Lightweight wrapper for presenting an untyped ActorSystem to a Behavior (via the context).
* INTERNAL API. Lightweight wrapper for presenting a classic ActorSystem to a Behavior (via the context).
* Therefore it does not have a lot of vals, only the whenTerminated Future is cached after
* its transformation because redoing that every time will add extra objects that persist for
* a longer time; in all other cases the wrapper will just be spawned for a single call in
* most circumstances.
*/
@InternalApi private[akka] class ActorSystemAdapter[-T](val untypedSystem: untyped.ActorSystemImpl)
@InternalApi private[akka] class ActorSystemAdapter[-T](val system: classic.ActorSystemImpl)
extends ActorSystem[T]
with ActorRef[T]
with ActorRefImpl[T]
with InternalRecipientRef[T]
with ExtensionsImpl {
// note that the untypedSystem may not be initialized yet here, and that is fine because
// note that the (classic) system may not be initialized yet here, and that is fine because
// it is unlikely that anything gets a hold of the extension until the system is started
import ActorRefAdapter.sendSystemMessage
override private[akka] def classicSystem: untyped.ActorSystem = untypedSystem
override private[akka] def classicSystem: classic.ActorSystem = system
// Members declared in akka.actor.typed.ActorRef
override def tell(msg: T): Unit = {
if (msg == null) throw InvalidMessageException("[null] is not an allowed message")
untypedSystem.guardian ! msg
system.guardian ! msg
}
// impl ActorRefImpl
override def isLocal: Boolean = true
// impl ActorRefImpl
override def sendSystem(signal: SystemMessage): Unit = sendSystemMessage(untypedSystem.guardian, signal)
override def sendSystem(signal: SystemMessage): Unit = sendSystemMessage(system.guardian, signal)
// impl InternalRecipientRef
override def provider: ActorRefProvider = untypedSystem.provider
override def provider: ActorRefProvider = system.provider
// impl InternalRecipientRef
def isTerminated: Boolean = whenTerminated.isCompleted
final override val path: untyped.ActorPath =
untyped.RootActorPath(untyped.Address("akka", untypedSystem.name)) / "user"
final override val path: classic.ActorPath =
classic.RootActorPath(classic.Address("akka", system.name)) / "user"
override def toString: String = untypedSystem.toString
override def toString: String = system.toString
// Members declared in akka.actor.typed.ActorSystem
override def deadLetters[U]: ActorRef[U] = ActorRefAdapter(untypedSystem.deadLetters)
override def deadLetters[U]: ActorRef[U] = ActorRefAdapter(system.deadLetters)
override def dispatchers: Dispatchers = new Dispatchers {
override def lookup(selector: DispatcherSelector): ExecutionContextExecutor =
selector match {
case DispatcherDefault(_) => untypedSystem.dispatcher
case DispatcherFromConfig(str, _) => untypedSystem.dispatchers.lookup(str)
case DispatcherSameAsParent(_) => untypedSystem.dispatcher
case DispatcherDefault(_) => system.dispatcher
case DispatcherFromConfig(str, _) => system.dispatchers.lookup(str)
case DispatcherSameAsParent(_) => system.dispatcher
}
override def shutdown(): Unit = () // there was no shutdown in untyped Akka
override def shutdown(): Unit = () // there was no shutdown in classic Akka
}
override def dynamicAccess: untyped.DynamicAccess = untypedSystem.dynamicAccess
implicit override def executionContext: scala.concurrent.ExecutionContextExecutor = untypedSystem.dispatcher
override def dynamicAccess: classic.DynamicAccess = system.dynamicAccess
implicit override def executionContext: scala.concurrent.ExecutionContextExecutor = system.dispatcher
override val log: Logger = new LoggerAdapterImpl(
untypedSystem.eventStream,
system.eventStream,
classOf[ActorSystem[_]],
name,
LoggingFilterWithMarker.wrap(untypedSystem.logFilter))
override def logConfiguration(): Unit = untypedSystem.logConfiguration()
override def name: String = untypedSystem.name
override val scheduler: Scheduler = new SchedulerAdapter(untypedSystem.scheduler)
override def settings: Settings = new Settings(untypedSystem.settings)
override def startTime: Long = untypedSystem.startTime
override def threadFactory: java.util.concurrent.ThreadFactory = untypedSystem.threadFactory
override def uptime: Long = untypedSystem.uptime
override def printTree: String = untypedSystem.printTree
LoggingFilterWithMarker.wrap(system.logFilter))
override def logConfiguration(): Unit = system.logConfiguration()
override def name: String = system.name
override val scheduler: Scheduler = new SchedulerAdapter(system.scheduler)
override def settings: Settings = new Settings(system.settings)
override def startTime: Long = system.startTime
override def threadFactory: java.util.concurrent.ThreadFactory = system.threadFactory
override def uptime: Long = system.uptime
override def printTree: String = system.printTree
import akka.dispatch.ExecutionContexts.sameThreadExecutionContext
override def terminate(): Unit = untypedSystem.terminate()
override def terminate(): Unit = system.terminate()
override lazy val whenTerminated: scala.concurrent.Future[akka.Done] =
untypedSystem.whenTerminated.map(_ => Done)(sameThreadExecutionContext)
system.whenTerminated.map(_ => Done)(sameThreadExecutionContext)
override lazy val getWhenTerminated: CompletionStage[akka.Done] =
FutureConverters.toJava(whenTerminated)
override def systemActorOf[U](behavior: Behavior[U], name: String, props: Props): ActorRef[U] = {
val ref = untypedSystem.systemActorOf(PropsAdapter(() => behavior, props), name)
val ref = system.systemActorOf(PropsAdapter(() => behavior, props), name)
ActorRefAdapter(ref)
}
}
private[akka] object ActorSystemAdapter {
def apply(system: untyped.ActorSystem): ActorSystem[Nothing] = AdapterExtension(system).adapter
def apply(system: classic.ActorSystem): ActorSystem[Nothing] = AdapterExtension(system).adapter
// to make sure we do never create more than one adapter for the same actor system
class AdapterExtension(system: untyped.ExtendedActorSystem) extends untyped.Extension {
val adapter = new ActorSystemAdapter(system.asInstanceOf[untyped.ActorSystemImpl])
class AdapterExtension(system: classic.ExtendedActorSystem) extends classic.Extension {
val adapter = new ActorSystemAdapter(system.asInstanceOf[classic.ActorSystemImpl])
}
object AdapterExtension extends untyped.ExtensionId[AdapterExtension] with untyped.ExtensionIdProvider {
override def get(system: untyped.ActorSystem): AdapterExtension = super.get(system)
object AdapterExtension extends classic.ExtensionId[AdapterExtension] with classic.ExtensionIdProvider {
override def get(system: classic.ActorSystem): AdapterExtension = super.get(system)
override def lookup() = AdapterExtension
override def createExtension(system: untyped.ExtendedActorSystem): AdapterExtension =
override def createExtension(system: classic.ExtendedActorSystem): AdapterExtension =
new AdapterExtension(system)
}
@ -140,19 +140,19 @@ private[akka] object ActorSystemAdapter {
*
* When on the classpath typed extensions will be loaded for classic ActorSystems as well.
*/
class LoadTypedExtensions(system: untyped.ExtendedActorSystem) extends untyped.Extension {
class LoadTypedExtensions(system: classic.ExtendedActorSystem) extends classic.Extension {
ActorSystemAdapter.AdapterExtension(system).adapter.loadExtensions()
}
object LoadTypedExtensions extends untyped.ExtensionId[LoadTypedExtensions] with untyped.ExtensionIdProvider {
object LoadTypedExtensions extends classic.ExtensionId[LoadTypedExtensions] with classic.ExtensionIdProvider {
override def lookup(): actor.ExtensionId[_ <: actor.Extension] = this
override def createExtension(system: ExtendedActorSystem): LoadTypedExtensions =
new LoadTypedExtensions(system)
}
def toUntyped[U](sys: ActorSystem[_]): untyped.ActorSystem =
def toClassic[U](sys: ActorSystem[_]): classic.ActorSystem =
sys match {
case adapter: ActorSystemAdapter[_] => adapter.untypedSystem
case adapter: ActorSystemAdapter[_] => adapter.classicSystem
case _ =>
throw new UnsupportedOperationException(
"Only adapted classic ActorSystem permissible " +

View file

@ -18,7 +18,7 @@ import akka.annotation.InternalApi
private[akka] val behavior: Behavior[EventStream.Command] =
Behaviors.setup { ctx =>
val eventStream = ctx.system.toUntyped.eventStream
val eventStream = ctx.system.toClassic.eventStream
eventStreamBehavior(eventStream)
}
@ -28,10 +28,10 @@ import akka.annotation.InternalApi
eventStream.publish(event)
Behaviors.same
case s @ EventStream.Subscribe(subscriber) =>
eventStream.subscribe(subscriber.toUntyped, s.topic)
eventStream.subscribe(subscriber.toClassic, s.topic)
Behaviors.same
case EventStream.Unsubscribe(subscriber) =>
eventStream.unsubscribe(subscriber.toUntyped)
eventStream.unsubscribe(subscriber.toClassic)
Behaviors.same
}

View file

@ -7,7 +7,7 @@ package akka.actor.typed.internal.adapter
import akka.actor.typed.{ LogMarker, Logger }
import akka.annotation.InternalApi
import akka.event.Logging._
import akka.event.{ LoggingBus, LoggingFilterWithMarker, LogMarker => UntypedLM }
import akka.event.{ LoggingBus, LoggingFilterWithMarker, LogMarker => ClassicLM }
import akka.util.OptionVal
import akka.util.ccompat.JavaConverters._
@ -397,13 +397,13 @@ private[akka] final class LoggerAdapterImpl(
override def isDebugEnabled = loggingFilter.isDebugEnabled(logClass, logSource)
override def isErrorEnabled(marker: LogMarker): Boolean =
loggingFilter.isErrorEnabled(logClass, logSource, marker.asInstanceOf[UntypedLM])
loggingFilter.isErrorEnabled(logClass, logSource, marker.asInstanceOf[ClassicLM])
override def isWarningEnabled(marker: LogMarker): Boolean =
loggingFilter.isWarningEnabled(logClass, logSource, marker.asInstanceOf[UntypedLM])
loggingFilter.isWarningEnabled(logClass, logSource, marker.asInstanceOf[ClassicLM])
override def isInfoEnabled(marker: LogMarker): Boolean =
loggingFilter.isInfoEnabled(logClass, logSource, marker.asInstanceOf[UntypedLM])
loggingFilter.isInfoEnabled(logClass, logSource, marker.asInstanceOf[ClassicLM])
override def isDebugEnabled(marker: LogMarker): Boolean =
loggingFilter.isDebugEnabled(logClass, logSource, marker.asInstanceOf[UntypedLM])
loggingFilter.isDebugEnabled(logClass, logSource, marker.asInstanceOf[ClassicLM])
override def withMdc(mdc: Map[String, Any]): Logger = {
val mdcAdapter = new LoggerAdapterImpl(bus, logClass, logSource, loggingFilter)
@ -427,12 +427,12 @@ private[akka] final class LoggerAdapterImpl(
val error = cause match {
case OptionVal.Some(cause) =>
marker match {
case OptionVal.Some(m) => Error(cause, logSource, logClass, message, mdc, m.asInstanceOf[UntypedLM])
case OptionVal.Some(m) => Error(cause, logSource, logClass, message, mdc, m.asInstanceOf[ClassicLM])
case OptionVal.None => Error(cause, logSource, logClass, message, mdc)
}
case OptionVal.None =>
marker match {
case OptionVal.Some(m) => Error(logSource, logClass, message, mdc, m.asInstanceOf[UntypedLM])
case OptionVal.Some(m) => Error(logSource, logClass, message, mdc, m.asInstanceOf[ClassicLM])
case OptionVal.None => Error(logSource, logClass, message, mdc)
}
}
@ -441,10 +441,10 @@ private[akka] final class LoggerAdapterImpl(
private[akka] def notifyWarning(message: String, cause: OptionVal[Throwable], marker: OptionVal[LogMarker]): Unit = {
val warning =
if (cause.isDefined) Warning(cause.get, logSource, logClass, message, mdc, marker.orNull.asInstanceOf[UntypedLM])
if (cause.isDefined) Warning(cause.get, logSource, logClass, message, mdc, marker.orNull.asInstanceOf[ClassicLM])
else
marker match {
case OptionVal.Some(m) => Warning(logSource, logClass, message, mdc, m.asInstanceOf[UntypedLM])
case OptionVal.Some(m) => Warning(logSource, logClass, message, mdc, m.asInstanceOf[ClassicLM])
case OptionVal.None => Warning(logSource, logClass, message, mdc)
}
bus.publish(warning)
@ -452,7 +452,7 @@ private[akka] final class LoggerAdapterImpl(
private[akka] def notifyInfo(message: String, marker: OptionVal[LogMarker]): Unit = {
val info = marker match {
case OptionVal.Some(m) => Info(logSource, logClass, message, mdc, m.asInstanceOf[UntypedLM])
case OptionVal.Some(m) => Info(logSource, logClass, message, mdc, m.asInstanceOf[ClassicLM])
case OptionVal.None => Info(logSource, logClass, message, mdc)
}
bus.publish(info)
@ -460,7 +460,7 @@ private[akka] final class LoggerAdapterImpl(
private[akka] def notifyDebug(message: String, marker: OptionVal[LogMarker]): Unit = {
val debug = marker match {
case OptionVal.Some(m) => Debug(logSource, logClass, message, mdc, m.asInstanceOf[UntypedLM])
case OptionVal.Some(m) => Debug(logSource, logClass, message, mdc, m.asInstanceOf[ClassicLM])
case OptionVal.None => Debug(logSource, logClass, message, mdc)
}
bus.publish(debug)

View file

@ -32,7 +32,7 @@ import akka.dispatch.Mailboxes
val p2 = deploy.firstOrElse[MailboxSelector](MailboxSelector.default()) match {
case _: DefaultMailboxSelector => p1
case BoundedMailboxSelector(capacity, _) =>
// specific support in untyped Mailboxes
// specific support in classic Mailboxes
p1.withMailbox(s"${Mailboxes.BoundedCapacityPrefix}$capacity")
case MailboxFromConfigSelector(path, _) =>
props.withMailbox(path)

View file

@ -17,34 +17,34 @@ import scala.concurrent.duration.FiniteDuration
* INTERNAL API
*/
@InternalApi
private[akka] final class SchedulerAdapter(private[akka] val untypedScheduler: akka.actor.Scheduler) extends Scheduler {
private[akka] final class SchedulerAdapter(private[akka] val classicScheduler: akka.actor.Scheduler) extends Scheduler {
override def scheduleOnce(delay: FiniteDuration, runnable: Runnable)(
implicit executor: ExecutionContext): Cancellable =
untypedScheduler.scheduleOnce(delay, runnable)
classicScheduler.scheduleOnce(delay, runnable)
override def scheduleOnce(delay: Duration, runnable: Runnable, executor: ExecutionContext): Cancellable =
untypedScheduler.scheduleOnce(delay, runnable)(executor)
classicScheduler.scheduleOnce(delay, runnable)(executor)
override def scheduleWithFixedDelay(initialDelay: FiniteDuration, delay: FiniteDuration)(runnable: Runnable)(
implicit executor: ExecutionContext): Cancellable =
untypedScheduler.scheduleWithFixedDelay(initialDelay, delay)(runnable)
classicScheduler.scheduleWithFixedDelay(initialDelay, delay)(runnable)
override def scheduleWithFixedDelay(
initialDelay: Duration,
delay: Duration,
runnable: Runnable,
executor: ExecutionContext): Cancellable =
untypedScheduler.scheduleWithFixedDelay(initialDelay, delay, runnable, executor)
classicScheduler.scheduleWithFixedDelay(initialDelay, delay, runnable, executor)
override def scheduleAtFixedRate(initialDelay: FiniteDuration, interval: FiniteDuration)(runnable: Runnable)(
implicit executor: ExecutionContext): Cancellable =
untypedScheduler.scheduleAtFixedRate(initialDelay, interval)(runnable)
classicScheduler.scheduleAtFixedRate(initialDelay, interval)(runnable)
override def scheduleAtFixedRate(
initialDelay: Duration,
interval: Duration,
runnable: Runnable,
executor: ExecutionContext): Cancellable =
untypedScheduler.scheduleAtFixedRate(initialDelay, interval, runnable, executor)
classicScheduler.scheduleAtFixedRate(initialDelay, interval, runnable, executor)
}

View file

@ -18,7 +18,7 @@ import akka.annotation.InternalApi
override val ref: ActorRef[Receptionist.Command] = {
val provider: ReceptionistBehaviorProvider =
if (system.settings.untypedSettings.ProviderSelectionType.hasCluster) {
if (system.settings.classicSettings.ProviderSelectionType.hasCluster) {
system.dynamicAccess
.getObjectFor[ReceptionistBehaviorProvider]("akka.cluster.typed.internal.receptionist.ClusterReceptionist")
.recover {

View file

@ -64,7 +64,7 @@ private final class InitialGroupRouterImpl[T](
ctx.system.eventStream ! EventStream.Publish(Dropped(
msg,
s"Stash is full in group router for [$serviceKey]",
ctx.self.toUntyped)) // don't fail on full stash
ctx.self.toClassic)) // don't fail on full stash
this
}
}
@ -99,7 +99,7 @@ private[akka] final class GroupRouterImpl[T](
if (!routeesEmpty) routingLogic.selectRoutee() ! msg
else
ctx.system.eventStream ! EventStream.Publish(
Dropped(msg, s"No routees in group router for [$serviceKey]", ctx.self.toUntyped))
Dropped(msg, s"No routees in group router for [$serviceKey]", ctx.self.toClassic))
this
}
}

View file

@ -269,7 +269,7 @@ trait ActorContext[T] extends TypedActorContext[T] {
* the other actor can send a message back through.
* @param applyToResponse Transforms the response from the `target` into a message this actor understands.
* Will be invoked with either the response message or an AskTimeoutException failed or
* potentially another exception if the remote actor is untyped and sent a
* potentially another exception if the remote actor is classic and sent a
* [[akka.actor.Status.Failure]] as response. The returned message of type `T` is then
* fed into this actor as a message. Should be a pure function but is executed inside
* the actor when the response arrives so can safely touch the actor internals. If this

View file

@ -14,80 +14,80 @@ import akka.actor.typed.internal.adapter.ActorContextAdapter
import akka.japi.Creator
/**
* Java API: Adapters between typed and untyped actors and actor systems.
* The underlying `ActorSystem` is the untyped [[akka.actor.ActorSystem]]
* which runs Akka Typed [[akka.actor.typed.Behavior]] on an emulation layer. In this
* system typed and untyped actors can coexist.
* Java API: Adapters between typed and classic actors and actor systems.
* The underlying `ActorSystem` is the classic [[akka.actor.ActorSystem]]
* which runs Akka [[akka.actor.typed.Behavior]] on an emulation layer. In this
* system typed and classic actors can coexist.
*
* These methods make it possible to create typed child actor from untyped
* parent actor, and the opposite untyped child from typed parent.
* These methods make it possible to create a child actor from classic
* parent actor, and the opposite classic child from typed parent.
* `watch` is also supported in both directions.
*
* There are also converters (`toTyped`, `toUntyped`) between untyped
* [[akka.actor.ActorRef]] and typed [[akka.actor.typed.ActorRef]], and between untyped
* [[akka.actor.ActorSystem]] and typed [[akka.actor.typed.ActorSystem]].
* There are also converters (`toTyped`, `toClassic`) between classic
* [[akka.actor.ActorRef]] and [[akka.actor.typed.ActorRef]], and between classic
* [[akka.actor.ActorSystem]] and [[akka.actor.typed.ActorSystem]].
*/
object Adapter {
/**
* Spawn the given behavior as a child of the user actor in an untyped ActorSystem.
* Typed actors default supervision strategy is to stop. Can be overridden with
* Spawn the given behavior as a child of the user actor in a classic ActorSystem.
* Actor default supervision strategy is to stop. Can be overridden with
* `Behaviors.supervise`.
*/
def spawnAnonymous[T](sys: akka.actor.ActorSystem, behavior: Behavior[T]): ActorRef[T] =
spawnAnonymous(sys, behavior, Props.empty)
/**
* Spawn the given behavior as a child of the user actor in an untyped ActorSystem.
* Typed actors default supervision strategy is to stop. Can be overridden with
* Spawn the given behavior as a child of the user actor in a classic ActorSystem.
* Actor default supervision strategy is to stop. Can be overridden with
* `Behaviors.supervise`.
*/
def spawnAnonymous[T](sys: akka.actor.ActorSystem, behavior: Behavior[T], props: Props): ActorRef[T] =
sys.spawnAnonymous(behavior, props)
/**
* Spawn the given behavior as a child of the user actor in an untyped ActorSystem.
* Typed actors default supervision strategy is to stop. Can be overridden with
* Spawn the given behavior as a child of the user actor in a classic ActorSystem.
* Actor default supervision strategy is to stop. Can be overridden with
* `Behaviors.supervise`.
*/
def spawn[T](sys: akka.actor.ActorSystem, behavior: Behavior[T], name: String): ActorRef[T] =
spawn(sys, behavior, name, Props.empty)
/**
* Spawn the given behavior as a child of the user actor in an untyped ActorSystem.
* Typed actors default supervision strategy is to stop. Can be overridden with
* Spawn the given behavior as a child of the user actor in a classic ActorSystem.
* Actor default supervision strategy is to stop. Can be overridden with
* `Behaviors.supervise`.
*/
def spawn[T](sys: akka.actor.ActorSystem, behavior: Behavior[T], name: String, props: Props): ActorRef[T] =
sys.spawn(behavior, name, props)
/**
* Spawn the given behavior as a child of the user actor in an untyped ActorContext.
* Typed actors default supervision strategy is to stop. Can be overridden with
* Spawn the given behavior as a child of the user actor in a classic ActorContext.
* Actor default supervision strategy is to stop. Can be overridden with
* `Behaviors.supervise`.
*/
def spawnAnonymous[T](ctx: akka.actor.ActorContext, behavior: Behavior[T]): ActorRef[T] =
spawnAnonymous(ctx, behavior, Props.empty)
/**
* Spawn the given behavior as a child of the user actor in an untyped ActorContext.
* Typed actors default supervision strategy is to stop. Can be overridden with
* Spawn the given behavior as a child of the user actor in a classic ActorContext.
* Actor default supervision strategy is to stop. Can be overridden with
* `Behaviors.supervise`.
*/
def spawnAnonymous[T](ctx: akka.actor.ActorContext, behavior: Behavior[T], props: Props): ActorRef[T] =
ctx.spawnAnonymous(behavior, props)
/**
* Spawn the given behavior as a child of the user actor in an untyped ActorContext.
* Typed actors default supervision strategy is to stop. Can be overridden with
* Spawn the given behavior as a child of the user actor in a classic ActorContext.
* Actor default supervision strategy is to stop. Can be overridden with
* `Behaviors.supervise`.
*/
def spawn[T](ctx: akka.actor.ActorContext, behavior: Behavior[T], name: String): ActorRef[T] =
spawn(ctx, behavior, name, Props.empty)
/**
* Spawn the given behavior as a child of the user actor in an untyped ActorContext.
* Typed actors default supervision strategy is to stop. Can be overridden with
* Spawn the given behavior as a child of the user actor in a classic ActorContext.
* Actor default supervision strategy is to stop. Can be overridden with
* `Behaviors.supervise`.
*/
def spawn[T](ctx: akka.actor.ActorContext, behavior: Behavior[T], name: String, props: Props): ActorRef[T] =
@ -96,11 +96,11 @@ object Adapter {
def toTyped(sys: akka.actor.ActorSystem): ActorSystem[Void] =
sys.toTyped.asInstanceOf[ActorSystem[Void]]
def toUntyped(sys: ActorSystem[_]): akka.actor.ActorSystem =
sys.toUntyped
def toClassic(sys: ActorSystem[_]): akka.actor.ActorSystem =
sys.toClassic
def toUntyped(ctx: ActorContext[_]): actor.ActorContext =
ActorContextAdapter.toUntyped(ctx)
def toClassic(ctx: ActorContext[_]): actor.ActorContext =
ActorContextAdapter.toClassic(ctx)
def watch[U](ctx: akka.actor.ActorContext, other: ActorRef[U]): Unit =
ctx.watch(other)
@ -121,36 +121,36 @@ object Adapter {
ctx.stop(child)
def actorOf(ctx: ActorContext[_], props: akka.actor.Props): akka.actor.ActorRef =
ActorContextAdapter.toUntyped(ctx).actorOf(props)
ActorContextAdapter.toClassic(ctx).actorOf(props)
def actorOf(ctx: ActorContext[_], props: akka.actor.Props, name: String): akka.actor.ActorRef =
ActorContextAdapter.toUntyped(ctx).actorOf(props, name)
ActorContextAdapter.toClassic(ctx).actorOf(props, name)
def toUntyped(ref: ActorRef[_]): akka.actor.ActorRef =
ref.toUntyped
def toClassic(ref: ActorRef[_]): akka.actor.ActorRef =
ref.toClassic
def toTyped[T](ref: akka.actor.ActorRef): ActorRef[T] =
ref
/**
* Wrap [[akka.actor.typed.Behavior]] in an untyped [[akka.actor.Props]], i.e. when
* spawning a typed child actor from an untyped parent actor.
* Wrap [[akka.actor.typed.Behavior]] in a classic [[akka.actor.Props]], i.e. when
* spawning a typed child actor from a classic parent actor.
* This is normally not needed because you can use the extension methods
* `spawn` and `spawnAnonymous` with an untyped `ActorContext`, but it's needed
* `spawn` and `spawnAnonymous` with a classic `ActorContext`, but it's needed
* when using typed actors with an existing library/tool that provides an API that
* takes an untyped [[akka.actor.Props]] parameter. Cluster Sharding is an
* takes a classic [[akka.actor.Props]] parameter. Cluster Sharding is an
* example of that.
*/
def props[T](behavior: Creator[Behavior[T]], deploy: Props): akka.actor.Props =
akka.actor.typed.internal.adapter.PropsAdapter(() => behavior.create(), deploy)
/**
* Wrap [[akka.actor.typed.Behavior]] in an untyped [[akka.actor.Props]], i.e. when
* spawning a typed child actor from an untyped parent actor.
* Wrap [[akka.actor.typed.Behavior]] in a classic [[akka.actor.Props]], i.e. when
* spawning a typed child actor from a classic parent actor.
* This is normally not needed because you can use the extension methods
* `spawn` and `spawnAnonymous` with an untyped `ActorContext`, but it's needed
* `spawn` and `spawnAnonymous` with a classic `ActorContext`, but it's needed
* when using typed actors with an existing library/tool that provides an API that
* takes an untyped [[akka.actor.Props]] parameter. Cluster Sharding is an
* takes a classic [[akka.actor.Props]] parameter. Cluster Sharding is an
* example of that.
*/
def props[T](behavior: Creator[Behavior[T]]): akka.actor.Props =

View file

@ -101,7 +101,7 @@ object AskPattern {
// We do not currently use the implicit sched, but want to require it
// because it might be needed when we move to a 'native' typed runtime, see #24219
ref match {
case a: InternalRecipientRef[_] => askUntyped(a, timeout, replyTo)
case a: InternalRecipientRef[_] => askClassic(a, timeout, replyTo)
case a =>
throw new IllegalStateException(
"Only expect references to be RecipientRef, ActorRefAdapter or ActorSystemAdapter until " +
@ -139,7 +139,7 @@ object AskPattern {
val promiseRef: PromiseActorRef = _promiseRef
}
private def askUntyped[T, U](target: InternalRecipientRef[T], timeout: Timeout, f: ActorRef[U] => T): Future[U] = {
private def askClassic[T, U](target: InternalRecipientRef[T], timeout: Timeout, f: ActorRef[U] => T): Future[U] = {
val p = new PromiseRef[U](target, timeout)
val m = f(p.ref)
if (p.promiseRef ne null) p.promiseRef.messageClassName = m.getClass.getName

View file

@ -8,12 +8,12 @@ import akka.actor.typed.Behavior
import akka.actor.typed.Props
/**
* Wrap [[akka.actor.typed.Behavior]] in an untyped [[akka.actor.Props]], i.e. when
* spawning a typed child actor from an untyped parent actor.
* Wrap [[akka.actor.typed.Behavior]] in a classic [[akka.actor.Props]], i.e. when
* spawning a typed child actor from a classic parent actor.
* This is normally not needed because you can use the extension methods
* `spawn` and `spawnAnonymous` on an untyped `ActorContext`, but it's needed
* `spawn` and `spawnAnonymous` on a classic `ActorContext`, but it's needed
* when using typed actors with an existing library/tool that provides an API that
* takes an untyped [[akka.actor.Props]] parameter. Cluster Sharding is an
* takes a classic [[akka.actor.Props]] parameter. Cluster Sharding is an
* example of that.
*/
object PropsAdapter {

View file

@ -19,13 +19,13 @@ import akka.annotation.InternalApi
*
* Implicit extension methods are added to classic and typed `ActorSystem`,
* `ActorContext`. Such methods make it possible to create typed child actor
* from classic parent actor, and the opposite untyped child from typed parent.
* from classic parent actor, and the opposite classic child from typed parent.
* `watch` is also supported in both directions.
*
* There is an implicit conversion from classic [[akka.actor.ActorRef]] to
* typed [[akka.actor.typed.ActorRef]].
*
* There are also converters (`toTyped`, `toUntyped`) from typed
* There are also converters (`toTyped`, `toClassic`) from typed
* [[akka.actor.typed.ActorRef]] to classic [[akka.actor.ActorRef]], and between classic
* [[akka.actor.ActorSystem]] and typed [[akka.actor.typed.ActorSystem]].
*/
@ -36,10 +36,10 @@ package object adapter {
/**
* Extension methods added to [[akka.actor.ActorSystem]].
*/
implicit class UntypedActorSystemOps(val sys: akka.actor.ActorSystem) extends AnyVal {
implicit class ClassicActorSystemOps(val sys: akka.actor.ActorSystem) extends AnyVal {
/**
* Spawn the given behavior as a child of the user actor in an untyped ActorSystem.
* Spawn the given behavior as a child of the user actor in a classic ActorSystem.
*
* Typed actors default supervision strategy is to stop. Can be overridden with
* `Behaviors.supervise`.
@ -53,7 +53,7 @@ package object adapter {
}
/**
* Spawn the given behavior as a child of the user actor in an untyped ActorSystem.
* Spawn the given behavior as a child of the user actor in a classic ActorSystem.
*
* Typed actors default supervision strategy is to stop. Can be overridden with
* `Behaviors.supervise`.
@ -74,7 +74,7 @@ package object adapter {
* Extension methods added to [[akka.actor.typed.ActorSystem]].
*/
implicit class TypedActorSystemOps(val sys: ActorSystem[_]) extends AnyVal {
def toUntyped: akka.actor.ActorSystem = ActorSystemAdapter.toUntyped(sys)
def toClassic: akka.actor.ActorSystem = ActorSystemAdapter.toClassic(sys)
/**
* INTERNAL API
@ -83,17 +83,17 @@ package object adapter {
behavior: Behavior[U],
name: String,
props: Props): ActorRef[U] = {
toUntyped.asInstanceOf[ExtendedActorSystem].systemActorOf(PropsAdapter(behavior, props), name)
toClassic.asInstanceOf[ExtendedActorSystem].systemActorOf(PropsAdapter(behavior, props), name)
}
}
/**
* Extension methods added to [[akka.actor.ActorContext]].
*/
implicit class UntypedActorContextOps(val ctx: akka.actor.ActorContext) extends AnyVal {
implicit class ClassicActorContextOps(val ctx: akka.actor.ActorContext) extends AnyVal {
/**
* Spawn the given behavior as a child of the user actor in an untyped ActorContext.
* Spawn the given behavior as a child of the user actor in a classic ActorContext.
*
* Typed actors default supervision strategy is to stop. Can be overridden with
* `Behaviors.supervise`.
@ -106,7 +106,7 @@ package object adapter {
rethrowTypedFailure = false)
/**
* Spawn the given behavior as a child of the user actor in an untyped ActorContext.
* Spawn the given behavior as a child of the user actor in a classic ActorContext.
*
* Typed actors default supervision strategy is to stop. Can be overridden with
* `Behaviors.supervise`.
@ -119,11 +119,11 @@ package object adapter {
props,
rethrowTypedFailure = false)
def watch[U](other: ActorRef[U]): Unit = ctx.watch(ActorRefAdapter.toUntyped(other))
def unwatch[U](other: ActorRef[U]): Unit = ctx.unwatch(ActorRefAdapter.toUntyped(other))
def watch[U](other: ActorRef[U]): Unit = ctx.watch(ActorRefAdapter.toClassic(other))
def unwatch[U](other: ActorRef[U]): Unit = ctx.unwatch(ActorRefAdapter.toClassic(other))
def stop(child: ActorRef[_]): Unit =
ctx.stop(ActorRefAdapter.toUntyped(child))
ctx.stop(ActorRefAdapter.toClassic(child))
}
/**
@ -131,12 +131,12 @@ package object adapter {
*/
implicit class TypedActorContextOps(val ctx: scaladsl.ActorContext[_]) extends AnyVal {
def actorOf(props: akka.actor.Props): akka.actor.ActorRef =
ActorContextAdapter.toUntyped(ctx).actorOf(props)
ActorContextAdapter.toClassic(ctx).actorOf(props)
def actorOf(props: akka.actor.Props, name: String): akka.actor.ActorRef =
ActorContextAdapter.toUntyped(ctx).actorOf(props, name)
ActorContextAdapter.toClassic(ctx).actorOf(props, name)
def toUntyped: akka.actor.ActorContext = ActorContextAdapter.toUntyped(ctx)
def toClassic: akka.actor.ActorContext = ActorContextAdapter.toClassic(ctx)
// watch, unwatch and stop not needed here because of the implicit ActorRef conversion
}
@ -145,16 +145,16 @@ package object adapter {
* Extension methods added to [[akka.actor.typed.ActorRef]].
*/
implicit class TypedActorRefOps(val ref: ActorRef[_]) extends AnyVal {
def toUntyped: akka.actor.ActorRef = ActorRefAdapter.toUntyped(ref)
def toClassic: akka.actor.ActorRef = ActorRefAdapter.toClassic(ref)
}
/**
* Extension methods added to [[akka.actor.ActorRef]].
*/
implicit class UntypedActorRefOps(val ref: akka.actor.ActorRef) extends AnyVal {
implicit class ClassicActorRefOps(val ref: akka.actor.ActorRef) extends AnyVal {
/**
* Adapt the untyped `ActorRef` to typed `ActorRef[T]`. There is also an
* Adapt the classic `ActorRef` to `akka.actor.typed.ActorRef[T]`. There is also an
* automatic implicit conversion for this, but this more explicit variant might
* sometimes be preferred.
*/
@ -162,7 +162,7 @@ package object adapter {
}
/**
* Implicit conversion from untyped [[akka.actor.ActorRef]] to typed [[akka.actor.typed.ActorRef]].
* Implicit conversion from classic [[akka.actor.ActorRef]] to [[akka.actor.typed.ActorRef]].
*/
implicit def actorRefAdapter[T](ref: akka.actor.ActorRef): ActorRef[T] = ActorRefAdapter(ref)

View file

@ -9,8 +9,8 @@ import akka.actor.NoSerializationVerificationNeeded
import akka.actor.typed.ActorSystem
import akka.annotation.InternalApi
import akka.cluster.ClusterSettings.DataCenter
import akka.cluster.sharding.{ ClusterShardingSettings => UntypedShardingSettings }
import akka.cluster.singleton.{ ClusterSingletonManagerSettings => UntypedClusterSingletonManagerSettings }
import akka.cluster.sharding.{ ClusterShardingSettings => ClassicShardingSettings }
import akka.cluster.singleton.{ ClusterSingletonManagerSettings => ClassicClusterSingletonManagerSettings }
import akka.cluster.typed.Cluster
import akka.cluster.typed.ClusterSingletonManagerSettings
import com.typesafe.config.Config
@ -23,40 +23,40 @@ object ClusterShardingSettings {
fromConfig(system.settings.config.getConfig("akka.cluster.sharding"))
def fromConfig(config: Config): ClusterShardingSettings = {
val untypedSettings = UntypedShardingSettings(config)
val classicSettings = ClassicShardingSettings(config)
val numberOfShards = config.getInt("number-of-shards")
fromUntypedSettings(numberOfShards, untypedSettings)
fromClassicSettings(numberOfShards, classicSettings)
}
/** Java API: Creates new cluster sharding settings object */
def create(system: ActorSystem[_]): ClusterShardingSettings =
apply(system)
/** INTERNAL API: Indended only for internal use, it is not recommended to keep converting between the setting types */
private[akka] def fromUntypedSettings(
/** INTERNAL API: Intended only for internal use, it is not recommended to keep converting between the setting types */
private[akka] def fromClassicSettings(
numberOfShards: Int,
untypedSettings: UntypedShardingSettings): ClusterShardingSettings = {
classicSettings: ClassicShardingSettings): ClusterShardingSettings = {
new ClusterShardingSettings(
numberOfShards,
role = untypedSettings.role,
role = classicSettings.role,
dataCenter = None,
rememberEntities = untypedSettings.rememberEntities,
journalPluginId = untypedSettings.journalPluginId,
snapshotPluginId = untypedSettings.snapshotPluginId,
passivateIdleEntityAfter = untypedSettings.passivateIdleEntityAfter,
shardRegionQueryTimeout = untypedSettings.shardRegionQueryTimeout,
stateStoreMode = StateStoreMode.byName(untypedSettings.stateStoreMode),
new TuningParameters(untypedSettings.tuningParameters),
rememberEntities = classicSettings.rememberEntities,
journalPluginId = classicSettings.journalPluginId,
snapshotPluginId = classicSettings.snapshotPluginId,
passivateIdleEntityAfter = classicSettings.passivateIdleEntityAfter,
shardRegionQueryTimeout = classicSettings.shardRegionQueryTimeout,
stateStoreMode = StateStoreMode.byName(classicSettings.stateStoreMode),
new TuningParameters(classicSettings.tuningParameters),
new ClusterSingletonManagerSettings(
untypedSettings.coordinatorSingletonSettings.singletonName,
untypedSettings.coordinatorSingletonSettings.role,
untypedSettings.coordinatorSingletonSettings.removalMargin,
untypedSettings.coordinatorSingletonSettings.handOverRetryInterval))
classicSettings.coordinatorSingletonSettings.singletonName,
classicSettings.coordinatorSingletonSettings.role,
classicSettings.coordinatorSingletonSettings.removalMargin,
classicSettings.coordinatorSingletonSettings.handOverRetryInterval))
}
/** INTERNAL API: Indended only for internal use, it is not recommended to keep converting between the setting types */
private[akka] def toUntypedSettings(settings: ClusterShardingSettings): UntypedShardingSettings = {
new UntypedShardingSettings(
/** INTERNAL API: Intended only for internal use, it is not recommended to keep converting between the setting types */
private[akka] def toClassicSettings(settings: ClusterShardingSettings): ClassicShardingSettings = {
new ClassicShardingSettings(
role = settings.role,
rememberEntities = settings.rememberEntities,
journalPluginId = settings.journalPluginId,
@ -64,7 +64,7 @@ object ClusterShardingSettings {
stateStoreMode = settings.stateStoreMode.name,
passivateIdleEntityAfter = settings.passivateIdleEntityAfter,
shardRegionQueryTimeout = settings.shardRegionQueryTimeout,
new UntypedShardingSettings.TuningParameters(
new ClassicShardingSettings.TuningParameters(
bufferSize = settings.tuningParameters.bufferSize,
coordinatorFailureBackoff = settings.tuningParameters.coordinatorFailureBackoff,
retryInterval = settings.tuningParameters.retryInterval,
@ -85,7 +85,7 @@ object ClusterShardingSettings {
settings.tuningParameters.entityRecoveryConstantRateStrategyFrequency,
entityRecoveryConstantRateStrategyNumberOfEntities =
settings.tuningParameters.entityRecoveryConstantRateStrategyNumberOfEntities),
new UntypedClusterSingletonManagerSettings(
new ClassicClusterSingletonManagerSettings(
settings.coordinatorSingletonSettings.singletonName,
settings.coordinatorSingletonSettings.role,
settings.coordinatorSingletonSettings.removalMargin,
@ -128,25 +128,25 @@ object ClusterShardingSettings {
val updatingStateTimeout: FiniteDuration,
val waitingForStateTimeout: FiniteDuration) {
def this(untyped: UntypedShardingSettings.TuningParameters) {
def this(classic: ClassicShardingSettings.TuningParameters) {
this(
bufferSize = untyped.bufferSize,
coordinatorFailureBackoff = untyped.coordinatorFailureBackoff,
retryInterval = untyped.retryInterval,
handOffTimeout = untyped.handOffTimeout,
shardStartTimeout = untyped.shardStartTimeout,
shardFailureBackoff = untyped.shardFailureBackoff,
entityRestartBackoff = untyped.entityRestartBackoff,
rebalanceInterval = untyped.rebalanceInterval,
snapshotAfter = untyped.snapshotAfter,
keepNrOfBatches = untyped.keepNrOfBatches,
leastShardAllocationRebalanceThreshold = untyped.leastShardAllocationRebalanceThreshold, // TODO extract it a bit
leastShardAllocationMaxSimultaneousRebalance = untyped.leastShardAllocationMaxSimultaneousRebalance,
waitingForStateTimeout = untyped.waitingForStateTimeout,
updatingStateTimeout = untyped.updatingStateTimeout,
entityRecoveryStrategy = untyped.entityRecoveryStrategy,
entityRecoveryConstantRateStrategyFrequency = untyped.entityRecoveryConstantRateStrategyFrequency,
entityRecoveryConstantRateStrategyNumberOfEntities = untyped.entityRecoveryConstantRateStrategyNumberOfEntities)
bufferSize = classic.bufferSize,
coordinatorFailureBackoff = classic.coordinatorFailureBackoff,
retryInterval = classic.retryInterval,
handOffTimeout = classic.handOffTimeout,
shardStartTimeout = classic.shardStartTimeout,
shardFailureBackoff = classic.shardFailureBackoff,
entityRestartBackoff = classic.entityRestartBackoff,
rebalanceInterval = classic.rebalanceInterval,
snapshotAfter = classic.snapshotAfter,
keepNrOfBatches = classic.keepNrOfBatches,
leastShardAllocationRebalanceThreshold = classic.leastShardAllocationRebalanceThreshold, // TODO extract it a bit
leastShardAllocationMaxSimultaneousRebalance = classic.leastShardAllocationMaxSimultaneousRebalance,
waitingForStateTimeout = classic.waitingForStateTimeout,
updatingStateTimeout = classic.updatingStateTimeout,
entityRecoveryStrategy = classic.entityRecoveryStrategy,
entityRecoveryConstantRateStrategyFrequency = classic.entityRecoveryConstantRateStrategyFrequency,
entityRecoveryConstantRateStrategyNumberOfEntities = classic.entityRecoveryConstantRateStrategyNumberOfEntities)
}

View file

@ -31,7 +31,7 @@ import akka.annotation.InternalApi
import akka.cluster.sharding.ShardCoordinator.LeastShardAllocationStrategy
import akka.cluster.sharding.ShardCoordinator.ShardAllocationStrategy
import akka.cluster.sharding.ShardRegion
import akka.cluster.sharding.ShardRegion.{ StartEntity => UntypedStartEntity }
import akka.cluster.sharding.ShardRegion.{ StartEntity => ClassicStartEntity }
import akka.cluster.sharding.typed.scaladsl.EntityContext
import akka.cluster.typed.Cluster
import akka.event.Logging
@ -52,8 +52,8 @@ import akka.util.Timeout
extends ShardingMessageExtractor[Any, M] {
override def entityId(message: Any): String = {
message match {
case ShardingEnvelope(entityId, _) => entityId //also covers UntypedStartEntity in ShardingEnvelope
case UntypedStartEntity(entityId) => entityId
case ShardingEnvelope(entityId, _) => entityId //also covers ClassicStartEntity in ShardingEnvelope
case ClassicStartEntity(entityId) => entityId
case msg: E @unchecked => delegate.entityId(msg)
}
}
@ -63,9 +63,9 @@ import akka.util.Timeout
override def unwrapMessage(message: Any): M = {
message match {
case ShardingEnvelope(_, msg: M @unchecked) =>
//also covers UntypedStartEntity in ShardingEnvelope
//also covers ClassicStartEntity in ShardingEnvelope
msg
case msg: UntypedStartEntity =>
case msg: ClassicStartEntity =>
// not really of type M, but erased and StartEntity is only handled internally, not delivered to the entity
msg.asInstanceOf[M]
case msg: E @unchecked =>
@ -132,9 +132,9 @@ import akka.util.Timeout
"only adapted classic actor systems can be used for cluster features")
private val cluster = Cluster(system)
private val untypedSystem: ExtendedActorSystem = system.toUntyped.asInstanceOf[ExtendedActorSystem]
private val untypedSharding = akka.cluster.sharding.ClusterSharding(untypedSystem)
private val log: LoggingAdapter = Logging(untypedSystem, classOf[scaladsl.ClusterSharding])
private val classicSystem: ExtendedActorSystem = system.toClassic.asInstanceOf[ExtendedActorSystem]
private val classicSharding = akka.cluster.sharding.ClusterSharding(classicSystem)
private val log: LoggingAdapter = Logging(classicSystem, classOf[scaladsl.ClusterSharding])
// typeKey.name to messageClassName
private val regions: ConcurrentHashMap[String, String] = new ConcurrentHashMap
@ -212,8 +212,8 @@ import akka.util.Timeout
typeKey.name,
new java.util.function.Function[String, ActorRef[scaladsl.ClusterSharding.ShardCommand]] {
override def apply(t: String): ActorRef[scaladsl.ClusterSharding.ShardCommand] = {
// using untyped.systemActorOf to avoid the Future[ActorRef]
system.toUntyped
// using classic.systemActorOf to avoid the Future[ActorRef]
system.toClassic
.asInstanceOf[ExtendedActorSystem]
.systemActorOf(
PropsAdapter(ShardCommandActor.behavior(stopMessage.getOrElse(PoisonPill))),
@ -228,14 +228,14 @@ import akka.util.Timeout
}
}
val untypedEntityPropsFactory: String => akka.actor.Props = { entityId =>
val classicEntityPropsFactory: String => akka.actor.Props = { entityId =>
val behv = behavior(new EntityContext(entityId, shardCommandDelegator))
PropsAdapter(poisonPillInterceptor(behv), entityProps)
}
untypedSharding.internalStart(
classicSharding.internalStart(
typeKey.name,
untypedEntityPropsFactory,
ClusterShardingSettings.toUntypedSettings(settings),
classicEntityPropsFactory,
ClusterShardingSettings.toClassicSettings(settings),
extractEntityId,
extractShardId,
allocationStrategy.getOrElse(defaultShardAllocationStrategy(settings)),
@ -248,7 +248,7 @@ import akka.util.Timeout
settings.role,
settings.dataCenter)
untypedSharding.startProxy(
classicSharding.startProxy(
typeKey.name,
settings.role,
dataCenter = settings.dataCenter,
@ -271,14 +271,14 @@ import akka.util.Timeout
override def entityRefFor[M](typeKey: scaladsl.EntityTypeKey[M], entityId: String): scaladsl.EntityRef[M] = {
new EntityRefImpl[M](
untypedSharding.shardRegion(typeKey.name),
classicSharding.shardRegion(typeKey.name),
entityId,
typeKey.asInstanceOf[EntityTypeKeyImpl[M]])
}
override def entityRefFor[M](typeKey: javadsl.EntityTypeKey[M], entityId: String): javadsl.EntityRef[M] = {
new EntityRefImpl[M](
untypedSharding.shardRegion(typeKey.name),
classicSharding.shardRegion(typeKey.name),
entityId,
typeKey.asInstanceOf[EntityTypeKeyImpl[M]])
}
@ -291,8 +291,8 @@ import akka.util.Timeout
override lazy val shardState: ActorRef[ClusterShardingQuery] = {
import akka.actor.typed.scaladsl.adapter._
val behavior = ShardingState.behavior(untypedSharding)
untypedSystem.systemActorOf(PropsAdapter(behavior), "typedShardState")
val behavior = ShardingState.behavior(classicSharding)
classicSystem.systemActorOf(PropsAdapter(behavior), "typedShardState")
}
}
@ -324,20 +324,20 @@ import akka.util.Timeout
/** Similar to [[akka.actor.typed.scaladsl.AskPattern.PromiseRef]] but for an `EntityRef` target. */
@InternalApi
private final class EntityPromiseRef[U](untyped: InternalActorRef, timeout: Timeout) {
private final class EntityPromiseRef[U](classic: InternalActorRef, timeout: Timeout) {
import akka.actor.typed.internal.{ adapter => adapt }
// Note: _promiseRef mustn't have a type pattern, since it can be null
private[this] val (_ref: ActorRef[U], _future: Future[U], _promiseRef) =
if (untyped.isTerminated)
if (classic.isTerminated)
(
adapt.ActorRefAdapter[U](untyped.provider.deadLetters),
adapt.ActorRefAdapter[U](classic.provider.deadLetters),
Future.failed[U](
new AskTimeoutException(s"Recipient shard region of [${EntityRefImpl.this}] had already been terminated.")),
null)
else if (timeout.duration.length <= 0)
(
adapt.ActorRefAdapter[U](untyped.provider.deadLetters),
adapt.ActorRefAdapter[U](classic.provider.deadLetters),
Future.failed[U](
new IllegalArgumentException(
s"Timeout length must be positive, question not sent to [${EntityRefImpl.this}]")),
@ -345,7 +345,7 @@ import akka.util.Timeout
else {
// note that the real messageClassName will be set afterwards, replyTo pattern
val a =
PromiseActorRef(untyped.provider, timeout, targetName = EntityRefImpl.this, messageClassName = "unknown")
PromiseActorRef(classic.provider, timeout, targetName = EntityRefImpl.this, messageClassName = "unknown")
val b = adapt.ActorRefAdapter[U](a)
(b, a.result.future.asInstanceOf[Future[U]], a)
}
@ -376,21 +376,21 @@ import akka.util.Timeout
*/
@InternalApi private[akka] object ShardCommandActor {
import akka.actor.typed.scaladsl.adapter._
import akka.cluster.sharding.ShardRegion.{ Passivate => UntypedPassivate }
import akka.cluster.sharding.ShardRegion.{ Passivate => ClassicPassivate }
def behavior(stopMessage: Any): Behavior[scaladsl.ClusterSharding.ShardCommand] = {
def sendUntypedPassivate(entity: ActorRef[_], ctx: TypedActorContext[_]): Unit = {
val pathToShard = entity.toUntyped.path.elements.take(4).mkString("/")
ctx.asScala.system.toUntyped.actorSelection(pathToShard).tell(UntypedPassivate(stopMessage), entity.toUntyped)
def sendClassicPassivate(entity: ActorRef[_], ctx: TypedActorContext[_]): Unit = {
val pathToShard = entity.toClassic.path.elements.take(4).mkString("/")
ctx.asScala.system.toClassic.actorSelection(pathToShard).tell(ClassicPassivate(stopMessage), entity.toClassic)
}
Behaviors.receive { (ctx, msg) =>
msg match {
case scaladsl.ClusterSharding.Passivate(entity) =>
sendUntypedPassivate(entity, ctx)
sendClassicPassivate(entity, ctx)
Behaviors.same
case javadsl.ClusterSharding.Passivate(entity) =>
sendUntypedPassivate(entity, ctx)
sendClassicPassivate(entity, ctx)
Behaviors.same
case _ =>
Behaviors.unhandled

View file

@ -17,10 +17,10 @@ import akka.cluster.sharding.typed.{ ClusterShardingQuery, GetShardRegionState }
@InternalApi
object ShardingState {
def behavior(untypedSharding: ClusterSharding): Behavior[ClusterShardingQuery] = Behaviors.receiveMessage {
def behavior(classicSharding: ClusterSharding): Behavior[ClusterShardingQuery] = Behaviors.receiveMessage {
case GetShardRegionState(key, replyTo) =>
if (untypedSharding.getShardTypeNames.contains(key.name)) {
untypedSharding.shardRegion(key.name).tell(ShardRegion.GetShardRegionState, replyTo.toUntyped)
if (classicSharding.getShardTypeNames.contains(key.name)) {
classicSharding.shardRegion(key.name).tell(ShardRegion.GetShardRegionState, replyTo.toClassic)
} else {
replyTo ! CurrentShardRegionState(Set.empty)
}

View file

@ -23,7 +23,7 @@ import akka.annotation.InternalApi
import akka.cluster.sharding.ShardCoordinator.ShardAllocationStrategy
import akka.cluster.sharding.typed.internal.ClusterShardingImpl
import akka.cluster.sharding.typed.internal.EntityTypeKeyImpl
import akka.cluster.sharding.ShardRegion.{ StartEntity => UntypedStartEntity }
import akka.cluster.sharding.ShardRegion.{ StartEntity => ClassicStartEntity }
import akka.persistence.typed.PersistenceId
object ClusterSharding extends ExtensionId[ClusterSharding] {
@ -303,7 +303,7 @@ object StartEntity {
*/
def apply[M](entityId: String): ShardingEnvelope[M] = {
// StartEntity isn't really of type M, but erased and StartEntity is only handled internally, not delivered to the entity
new ShardingEnvelope[M](entityId, UntypedStartEntity(entityId).asInstanceOf[M])
new ShardingEnvelope[M](entityId, ClassicStartEntity(entityId).asInstanceOf[M])
}
}

View file

@ -14,7 +14,7 @@ class ShardingSerializerSpec extends ScalaTestWithActorTestKit with WordSpecLike
"The typed ShardingSerializer" must {
val serialization = SerializationExtension(ActorSystemAdapter.toUntyped(system))
val serialization = SerializationExtension(ActorSystemAdapter.toClassic(system))
def checkSerialization(obj: AnyRef): Unit = {
serialization.findSerializerFor(obj) match {

View file

@ -24,7 +24,7 @@ import akka.cluster.sharding.ShardRegion.CurrentShardRegionState
import akka.cluster.sharding.ShardRegion.GetShardRegionState
import akka.cluster.sharding.typed.scaladsl.ClusterSharding.Passivate
import akka.cluster.sharding.typed.scaladsl.ClusterSharding.ShardCommand
import akka.cluster.sharding.{ ClusterSharding => UntypedClusterSharding }
import akka.cluster.sharding.{ ClusterSharding => ClassicClusterSharding }
import akka.cluster.typed.Cluster
import akka.cluster.typed.Join
import akka.persistence.typed.ExpectingReply
@ -164,9 +164,9 @@ class ClusterShardingPersistenceSpec
// FIXME #24466: rewrite this with Typed API when region queries are supported
import akka.actor.typed.scaladsl.adapter._
val regionStateProbe = TestProbe[CurrentShardRegionState]()
val untypedRegion = UntypedClusterSharding(system.toUntyped)
val classicRegion = ClassicClusterSharding(system.toClassic)
regionStateProbe.awaitAssert {
untypedRegion.shardRegion(typeKey.name).tell(GetShardRegionState, regionStateProbe.ref.toUntyped)
classicRegion.shardRegion(typeKey.name).tell(GetShardRegionState, regionStateProbe.ref.toClassic)
regionStateProbe.receiveMessage().shards.foreach { shardState =>
shardState.entityIds should not contain entityId
}

View file

@ -217,7 +217,7 @@ class ClusterShardingSpec extends ScalaTestWithActorTestKit(ClusterShardingSpec.
import akka.pattern.ask
implicit val timeout: Timeout = Timeout(6.seconds)
val statsBefore =
(shardingRefSystem1WithEnvelope.toUntyped ? akka.cluster.sharding.ShardRegion.GetClusterShardingStats(5.seconds))
(shardingRefSystem1WithEnvelope.toClassic ? akka.cluster.sharding.ShardRegion.GetClusterShardingStats(5.seconds))
.mapTo[akka.cluster.sharding.ShardRegion.ClusterShardingStats]
val totalCount = statsBefore.futureValue.regions.values.flatMap(_.stats.values).sum
totalCount
@ -402,8 +402,8 @@ class ClusterShardingSpec extends ScalaTestWithActorTestKit(ClusterShardingSpec.
val totalCountBefore = totalEntityCount1()
val p = TestProbe[Any]()
shardingRefSystem1WithEnvelope.toUntyped
.tell(akka.cluster.sharding.ShardRegion.StartEntity("startEntity-1"), p.ref.toUntyped)
shardingRefSystem1WithEnvelope.toClassic
.tell(akka.cluster.sharding.ShardRegion.StartEntity("startEntity-1"), p.ref.toClassic)
p.expectMessageType[akka.cluster.sharding.ShardRegion.StartEntityAck]
eventually {

View file

@ -38,12 +38,12 @@ import akka.actor.typed.Terminated
underlyingReplicator: Option[akka.actor.ActorRef]): Behavior[SReplicator.Command] = {
Behaviors.setup { ctx =>
val untypedReplicator = underlyingReplicator match {
val classicReplicator = underlyingReplicator match {
case Some(ref) => ref
case None =>
// FIXME perhaps add supervisor for restarting, see PR https://github.com/akka/akka/pull/25988
val untypedReplicatorProps = dd.Replicator.props(settings)
ctx.actorOf(untypedReplicatorProps, name = "underlying")
val classicReplicatorProps = dd.Replicator.props(settings)
ctx.actorOf(classicReplicatorProps, name = "underlying")
}
def withState(
@ -55,7 +55,7 @@ import akka.actor.typed.Terminated
subscriber: ActorRef[JReplicator.Changed[ReplicatedData]]): Behavior[SReplicator.Command] = {
subscribeAdapters.get(subscriber) match {
case Some(adapter) =>
// will be unsubscribed from untypedReplicator via Terminated
// will be unsubscribed from classicReplicator via Terminated
ctx.stop(adapter)
withState(subscribeAdapters - subscriber)
case None => // already unsubscribed or terminated
@ -67,7 +67,7 @@ import akka.actor.typed.Terminated
.receive[SReplicator.Command] { (ctx, msg) =>
msg match {
case cmd: SReplicator.Get[_] =>
untypedReplicator.tell(dd.Replicator.Get(cmd.key, cmd.consistency), sender = cmd.replyTo.toUntyped)
classicReplicator.tell(dd.Replicator.Get(cmd.key, cmd.consistency), sender = cmd.replyTo.toClassic)
Behaviors.same
case cmd: JReplicator.Get[d] =>
@ -77,7 +77,7 @@ import akka.actor.typed.Terminated
})
import ctx.executionContext
val reply =
(untypedReplicator ? dd.Replicator.Get(cmd.key, cmd.consistency.toUntyped))
(classicReplicator ? dd.Replicator.Get(cmd.key, cmd.consistency.toClassic))
.mapTo[dd.Replicator.GetResponse[d]]
.map {
case rsp: dd.Replicator.GetSuccess[d] =>
@ -92,9 +92,9 @@ import akka.actor.typed.Terminated
Behaviors.same
case cmd: SReplicator.Update[_] =>
untypedReplicator.tell(
classicReplicator.tell(
dd.Replicator.Update(cmd.key, cmd.writeConsistency, None)(cmd.modify),
sender = cmd.replyTo.toUntyped)
sender = cmd.replyTo.toClassic)
Behaviors.same
case cmd: JReplicator.Update[d] =>
@ -104,7 +104,7 @@ import akka.actor.typed.Terminated
})
import ctx.executionContext
val reply =
(untypedReplicator ? dd.Replicator.Update(cmd.key, cmd.writeConsistency.toUntyped, None)(cmd.modify))
(classicReplicator ? dd.Replicator.Update(cmd.key, cmd.writeConsistency.toClassic, None)(cmd.modify))
.mapTo[dd.Replicator.UpdateResponse[d]]
.map {
case rsp: dd.Replicator.UpdateSuccess[d] => JReplicator.UpdateSuccess(rsp.key)
@ -121,9 +121,9 @@ import akka.actor.typed.Terminated
case cmd: SReplicator.Subscribe[_] =>
// For the Scala API the Changed messages can be sent directly to the subscriber
untypedReplicator.tell(
dd.Replicator.Subscribe(cmd.key, cmd.subscriber.toUntyped),
sender = cmd.subscriber.toUntyped)
classicReplicator.tell(
dd.Replicator.Subscribe(cmd.key, cmd.subscriber.toClassic),
sender = cmd.subscriber.toClassic)
Behaviors.same
case cmd: JReplicator.Subscribe[ReplicatedData] @unchecked =>
@ -134,8 +134,8 @@ import akka.actor.typed.Terminated
InternalChanged(chg, cmd.subscriber)
}
untypedReplicator.tell(
dd.Replicator.Subscribe(cmd.key, adapter.toUntyped),
classicReplicator.tell(
dd.Replicator.Subscribe(cmd.key, adapter.toClassic),
sender = akka.actor.ActorRef.noSender)
ctx.watch(cmd.subscriber)
@ -150,7 +150,7 @@ import akka.actor.typed.Terminated
stopSubscribeAdapter(cmd.subscriber)
case cmd: SReplicator.Delete[_] =>
untypedReplicator.tell(dd.Replicator.Delete(cmd.key, cmd.consistency), sender = cmd.replyTo.toUntyped)
classicReplicator.tell(dd.Replicator.Delete(cmd.key, cmd.consistency), sender = cmd.replyTo.toClassic)
Behaviors.same
case cmd: JReplicator.Delete[d] =>
@ -160,7 +160,7 @@ import akka.actor.typed.Terminated
})
import ctx.executionContext
val reply =
(untypedReplicator ? dd.Replicator.Delete(cmd.key, cmd.consistency.toUntyped))
(classicReplicator ? dd.Replicator.Delete(cmd.key, cmd.consistency.toClassic))
.mapTo[dd.Replicator.DeleteResponse[d]]
.map {
case rsp: dd.Replicator.DeleteSuccess[d] => JReplicator.DeleteSuccess(rsp.key)
@ -176,21 +176,21 @@ import akka.actor.typed.Terminated
Behaviors.same
case SReplicator.GetReplicaCount(replyTo) =>
untypedReplicator.tell(dd.Replicator.GetReplicaCount, sender = replyTo.toUntyped)
classicReplicator.tell(dd.Replicator.GetReplicaCount, sender = replyTo.toClassic)
Behaviors.same
case JReplicator.GetReplicaCount(replyTo) =>
implicit val timeout = Timeout(localAskTimeout)
import ctx.executionContext
val reply =
(untypedReplicator ? dd.Replicator.GetReplicaCount)
(classicReplicator ? dd.Replicator.GetReplicaCount)
.mapTo[dd.Replicator.ReplicaCount]
.map(rsp => JReplicator.ReplicaCount(rsp.n))
reply.foreach { replyTo ! _ }
Behaviors.same
case SReplicator.FlushChanges | JReplicator.FlushChanges =>
untypedReplicator.tell(dd.Replicator.FlushChanges, sender = akka.actor.ActorRef.noSender)
classicReplicator.tell(dd.Replicator.FlushChanges, sender = akka.actor.ActorRef.noSender)
Behaviors.same
}

View file

@ -46,60 +46,60 @@ object Replicator {
def timeout: Duration
/** INTERNAL API */
@InternalApi private[akka] def toUntyped: dd.Replicator.ReadConsistency
@InternalApi private[akka] def toClassic: dd.Replicator.ReadConsistency
}
case object ReadLocal extends ReadConsistency {
override def timeout: Duration = Duration.ZERO
/** INTERNAL API */
@InternalApi private[akka] override def toUntyped = dd.Replicator.ReadLocal
@InternalApi private[akka] override def toClassic = dd.Replicator.ReadLocal
}
final case class ReadFrom(n: Int, timeout: Duration) extends ReadConsistency {
require(n >= 2, "ReadFrom n must be >= 2, use ReadLocal for n=1")
/** INTERNAL API */
@InternalApi private[akka] override def toUntyped = dd.Replicator.ReadFrom(n, timeout.asScala)
@InternalApi private[akka] override def toClassic = dd.Replicator.ReadFrom(n, timeout.asScala)
}
final case class ReadMajority(timeout: Duration, minCap: Int = DefaultMajorityMinCap) extends ReadConsistency {
def this(timeout: Duration) = this(timeout, DefaultMajorityMinCap)
/** INTERNAL API */
@InternalApi private[akka] override def toUntyped = dd.Replicator.ReadMajority(timeout.asScala, minCap)
@InternalApi private[akka] override def toClassic = dd.Replicator.ReadMajority(timeout.asScala, minCap)
}
final case class ReadAll(timeout: Duration) extends ReadConsistency {
/** INTERNAL API */
@InternalApi private[akka] override def toUntyped = dd.Replicator.ReadAll(timeout.asScala)
@InternalApi private[akka] override def toClassic = dd.Replicator.ReadAll(timeout.asScala)
}
sealed trait WriteConsistency {
def timeout: Duration
/** INTERNAL API */
@InternalApi private[akka] def toUntyped: dd.Replicator.WriteConsistency
@InternalApi private[akka] def toClassic: dd.Replicator.WriteConsistency
}
case object WriteLocal extends WriteConsistency {
override def timeout: Duration = Duration.ZERO
/** INTERNAL API */
@InternalApi private[akka] override def toUntyped = dd.Replicator.WriteLocal
@InternalApi private[akka] override def toClassic = dd.Replicator.WriteLocal
}
final case class WriteTo(n: Int, timeout: Duration) extends WriteConsistency {
require(n >= 2, "WriteTo n must be >= 2, use WriteLocal for n=1")
/** INTERNAL API */
@InternalApi private[akka] override def toUntyped = dd.Replicator.WriteTo(n, timeout.asScala)
@InternalApi private[akka] override def toClassic = dd.Replicator.WriteTo(n, timeout.asScala)
}
final case class WriteMajority(timeout: Duration, minCap: Int = DefaultMajorityMinCap) extends WriteConsistency {
def this(timeout: Duration) = this(timeout, DefaultMajorityMinCap)
/** INTERNAL API */
@InternalApi private[akka] override def toUntyped = dd.Replicator.WriteMajority(timeout.asScala, minCap)
@InternalApi private[akka] override def toClassic = dd.Replicator.WriteMajority(timeout.asScala, minCap)
}
final case class WriteAll(timeout: Duration) extends WriteConsistency {
/** INTERNAL API */
@InternalApi private[akka] override def toUntyped = dd.Replicator.WriteAll(timeout.asScala)
@InternalApi private[akka] override def toClassic = dd.Replicator.WriteAll(timeout.asScala)
}
/**

View file

@ -16,7 +16,7 @@ object ReplicatorSettings {
* `akka.cluster.distributed-data`.
*/
def create(system: ActorSystem[_]): dd.ReplicatorSettings =
dd.ReplicatorSettings(system.toUntyped)
dd.ReplicatorSettings(system.toClassic)
/**
* Create settings from a configuration with the same layout as

View file

@ -74,9 +74,9 @@ class DistributedData(system: ActorSystem[_]) extends Extension {
.getDuration("akka.cluster.ddata.typed.replicator-message-adapter-unexpected-ask-timeout")
.asScala
private val untypedSystem = system.toUntyped.asInstanceOf[ExtendedActorSystem]
private val classicSystem = system.toClassic.asInstanceOf[ExtendedActorSystem]
implicit val selfUniqueAddress: SelfUniqueAddress = dd.DistributedData(untypedSystem).selfUniqueAddress
implicit val selfUniqueAddress: SelfUniqueAddress = dd.DistributedData(classicSystem).selfUniqueAddress
/**
* `ActorRef` of the [[Replicator]].
@ -86,17 +86,17 @@ class DistributedData(system: ActorSystem[_]) extends Extension {
val replicator: ActorRef[Replicator.Command] =
if (isTerminated) {
val log = system.log.withLoggerClass(getClass)
if (Cluster(untypedSystem).isTerminated)
if (Cluster(classicSystem).isTerminated)
log.warning("Replicator points to dead letters, because Cluster is terminated.")
else
log.warning(
"Replicator points to dead letters. Make sure the cluster node has the proper role. " +
"Node has roles [], Distributed Data is configured for roles []",
Cluster(untypedSystem).selfRoles.mkString(","),
Cluster(classicSystem).selfRoles.mkString(","),
settings.roles.mkString(","))
system.deadLetters
} else {
val underlyingReplicator = dd.DistributedData(untypedSystem).replicator
val underlyingReplicator = dd.DistributedData(classicSystem).replicator
val replicatorBehavior = Replicator.behavior(settings, underlyingReplicator)
system.internalSystemActorOf(
@ -108,6 +108,6 @@ class DistributedData(system: ActorSystem[_]) extends Extension {
/**
* Returns true if this member is not tagged with the role configured for the replicas.
*/
private def isTerminated: Boolean = dd.DistributedData(system.toUntyped).isTerminated
private def isTerminated: Boolean = dd.DistributedData(system.toClassic).isTerminated
}

View file

@ -20,7 +20,7 @@ object ReplicatorSettings {
* `akka.cluster.distributed-data`.
*/
def apply(system: ActorSystem[_]): ReplicatorSettings =
dd.ReplicatorSettings(system.toUntyped)
dd.ReplicatorSettings(system.toClassic)
/**
* Create settings from a configuration with the same layout as
@ -34,5 +34,5 @@ object ReplicatorSettings {
* The name of the actor used in DistributedData extensions.
*/
@InternalApi private[akka] def name(system: ActorSystem[_]): String =
dd.ReplicatorSettings.name(system.toUntyped, Some("typed"))
dd.ReplicatorSettings.name(system.toClassic, Some("typed"))
}

View file

@ -9,7 +9,7 @@ import akka.annotation.{ DoNotInherit, InternalApi }
import akka.cluster.ClusterSettings.DataCenter
import akka.cluster.singleton.{
ClusterSingletonProxySettings,
ClusterSingletonManagerSettings => UntypedClusterSingletonManagerSettings
ClusterSingletonManagerSettings => ClassicClusterSingletonManagerSettings
}
import akka.cluster.typed.internal.AdaptedClusterSingletonImpl
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Extension, ExtensionId, Props }
@ -91,8 +91,8 @@ final class ClusterSingletonSettings(
* INTERNAL API:
*/
@InternalApi
private[akka] def toManagerSettings(singletonName: String): UntypedClusterSingletonManagerSettings =
new UntypedClusterSingletonManagerSettings(singletonName, role, removalMargin, handOverRetryInterval)
private[akka] def toManagerSettings(singletonName: String): ClassicClusterSingletonManagerSettings =
new ClassicClusterSingletonManagerSettings(singletonName, role, removalMargin, handOverRetryInterval)
/**
* INTERNAL API:
@ -213,7 +213,7 @@ object ClusterSingletonManagerSettings {
*/
def apply(system: ActorSystem[_]): ClusterSingletonManagerSettings =
apply(system.settings.config.getConfig("akka.cluster.singleton"))
.withRemovalMargin(akka.cluster.Cluster(system.toUntyped).downingProvider.downRemovalMargin)
.withRemovalMargin(akka.cluster.Cluster(system.toClassic).downingProvider.downRemovalMargin)
/**
* Create settings from a configuration with the same layout as
@ -276,7 +276,7 @@ final class ClusterSingletonManagerSettings(
def withSingletonName(name: String): ClusterSingletonManagerSettings = copy(singletonName = name)
def withRole(role: String): ClusterSingletonManagerSettings =
copy(role = UntypedClusterSingletonManagerSettings.roleOption(role))
copy(role = ClassicClusterSingletonManagerSettings.roleOption(role))
def withRole(role: Option[String]): ClusterSingletonManagerSettings = copy(role = role)

View file

@ -31,7 +31,7 @@ private[akka] object AdapterClusterImpl {
var upSubscribers: List[ActorRef[SelfUp]] = Nil
var removedSubscribers: List[ActorRef[SelfRemoved]] = Nil
adaptedCluster.subscribe(ctx.self.toUntyped, ClusterEvent.initialStateAsEvents, classOf[MemberEvent])
adaptedCluster.subscribe(ctx.self.toClassic, ClusterEvent.initialStateAsEvents, classOf[MemberEvent])
// important to not eagerly refer to it or we get a cycle here
lazy val cluster = Cluster(ctx.system)
@ -77,15 +77,15 @@ private[akka] object AdapterClusterImpl {
case Subscribe(subscriber, eventClass) =>
adaptedCluster
.subscribe(subscriber.toUntyped, initialStateMode = ClusterEvent.initialStateAsEvents, eventClass)
.subscribe(subscriber.toClassic, initialStateMode = ClusterEvent.initialStateAsEvents, eventClass)
Behaviors.same
case Unsubscribe(subscriber) =>
adaptedCluster.unsubscribe(subscriber.toUntyped)
adaptedCluster.unsubscribe(subscriber.toClassic)
Behaviors.same
case GetCurrentState(sender) =>
adaptedCluster.sendCurrentClusterState(sender.toUntyped)
adaptedCluster.sendCurrentClusterState(sender.toClassic)
Behaviors.same
case evt: MemberEvent if evt.member.uniqueAddress == cluster.selfMember.uniqueAddress =>
@ -139,17 +139,17 @@ private[akka] final class AdapterClusterImpl(system: ActorSystem[_]) extends Clu
import AdapterClusterImpl._
require(system.isInstanceOf[ActorSystemAdapter[_]], "only adapted actor systems can be used for cluster features")
private val untypedCluster = akka.cluster.Cluster(system.toUntyped)
private val classicCluster = akka.cluster.Cluster(system.toClassic)
override def selfMember: Member = untypedCluster.selfMember
override def isTerminated: Boolean = untypedCluster.isTerminated
override def state: ClusterEvent.CurrentClusterState = untypedCluster.state
override def selfMember: Member = classicCluster.selfMember
override def isTerminated: Boolean = classicCluster.isTerminated
override def state: ClusterEvent.CurrentClusterState = classicCluster.state
// must not be lazy as it also updates the cached selfMember
override val subscriptions: ActorRef[ClusterStateSubscription] =
system.internalSystemActorOf(subscriptionsBehavior(untypedCluster), "clusterStateSubscriptions", Props.empty)
system.internalSystemActorOf(subscriptionsBehavior(classicCluster), "clusterStateSubscriptions", Props.empty)
override lazy val manager: ActorRef[ClusterCommand] =
system.internalSystemActorOf(managerBehavior(untypedCluster), "clusterCommandManager", Props.empty)
system.internalSystemActorOf(managerBehavior(classicCluster), "clusterCommandManager", Props.empty)
}

View file

@ -31,7 +31,7 @@ private[akka] final class AdaptedClusterSingletonImpl(system: ActorSystem[_]) ex
import akka.actor.typed.scaladsl.adapter._
private lazy val cluster = Cluster(system)
private val untypedSystem = system.toUntyped.asInstanceOf[ExtendedActorSystem]
private val classicSystem = system.toClassic.asInstanceOf[ExtendedActorSystem]
private val proxies = new ConcurrentHashMap[(String, Option[DataCenter]), ActorRef[_]]()
@ -50,11 +50,11 @@ private[akka] final class AdaptedClusterSingletonImpl(system: ActorSystem[_]) ex
if (settings.shouldRunManager(cluster)) {
val managerName = managerNameFor(singleton.name)
// start singleton on this node
val untypedProps = PropsAdapter(poisonPillInterceptor(singleton.behavior), singleton.props)
val classicProps = PropsAdapter(poisonPillInterceptor(singleton.behavior), singleton.props)
try {
untypedSystem.systemActorOf(
classicSystem.systemActorOf(
OldSingletonManager.props(
untypedProps,
classicProps,
singleton.stopMessage.getOrElse(PoisonPill),
settings.toManagerSettings(singleton.name)),
managerName)
@ -72,7 +72,7 @@ private[akka] final class AdaptedClusterSingletonImpl(system: ActorSystem[_]) ex
def apply(singletonNameAndDc: (String, Option[DataCenter])): ActorRef[_] = {
val (singletonName, _) = singletonNameAndDc
val proxyName = s"singletonProxy$singletonName-${settings.dataCenter.getOrElse("no-dc")}"
untypedSystem.systemActorOf(
classicSystem.systemActorOf(
ClusterSingletonProxy
.props(s"/system/${managerNameFor(singletonName)}", settings.toProxySettings(singletonName)),
proxyName)

View file

@ -72,14 +72,14 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
// captures setup/dependencies so we can avoid doing it over and over again
final class Setup(ctx: ActorContext[Command]) {
val untypedSystem = ctx.system.toUntyped
val classicSystem = ctx.system.toClassic
val settings = ClusterReceptionistSettings(ctx.system)
val selfSystemUid = AddressUidExtension(untypedSystem).longAddressUid
val selfSystemUid = AddressUidExtension(classicSystem).longAddressUid
lazy val keepTombstonesFor = cluster.settings.PruneGossipTombstonesAfter match {
case f: FiniteDuration => f
case _ => throw new IllegalStateException("Cannot actually happen")
}
val cluster = Cluster(untypedSystem)
val cluster = Cluster(classicSystem)
// don't use DistributedData.selfUniqueAddress here, because that will initialize extension, which
// isn't used otherwise by the ClusterReceptionist
implicit val selfNodeAddress = SelfUniqueAddress(cluster.selfUniqueAddress)
@ -108,7 +108,7 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
}
registry.allDdataKeys.foreach(key =>
setup.replicator ! Replicator.Subscribe(key, replicatorMessageAdapter.toUntyped))
setup.replicator ! Replicator.Subscribe(key, replicatorMessageAdapter.toClassic))
// keep track of cluster members
// remove entries when members are removed
@ -125,7 +125,7 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
throw new IllegalStateException(s"Unexpected ClusterDomainEvent $other. Please report bug.")
}
setup.cluster.subscribe(
clusterEventMessageAdapter.toUntyped,
clusterEventMessageAdapter.toClassic,
ClusterEvent.InitialStateAsEvents,
classOf[MemberJoined],
classOf[MemberWeaklyUp],

View file

@ -44,7 +44,7 @@ abstract class MultiDcClusterSingletonSpec
import MultiDcClusterActors._
import MultiDcClusterSingletonSpecConfig._
"A typed cluster with multiple data centers" must {
"A cluster with multiple data centers" must {
"be able to form" in {
runOn(first) {
cluster.manager ! Join(cluster.selfMember.address)

View file

@ -69,7 +69,7 @@ class ActorSystemSpec extends WordSpec with Matchers with BeforeAndAfterAll with
}
inbox.receiveAll() should ===("hello" :: Nil)
sys.whenTerminated.futureValue
CoordinatedShutdown(sys.toUntyped).shutdownReason() should ===(
CoordinatedShutdown(sys.toClassic).shutdownReason() should ===(
Some(CoordinatedShutdown.ActorSystemTerminateReason))
}
}
@ -105,7 +105,7 @@ class ActorSystemSpec extends WordSpec with Matchers with BeforeAndAfterAll with
// now we know that the guardian has started, and should receive PostStop
sys.terminate()
sys.whenTerminated.futureValue
CoordinatedShutdown(sys.toUntyped).shutdownReason() should ===(
CoordinatedShutdown(sys.toClassic).shutdownReason() should ===(
Some(CoordinatedShutdown.ActorSystemTerminateReason))
inbox.receiveAll() should ===("done" :: Nil)
}

View file

@ -39,7 +39,7 @@ class ClusterApiSpec extends ScalaTestWithActorTestKit(ClusterApiSpec.config) wi
val testSettings = TestKitSettings(system)
val clusterNode1 = Cluster(system)
val untypedSystem1 = system.toUntyped
val classicSystem1 = system.toClassic
"A typed Cluster" must {

View file

@ -90,7 +90,7 @@ class ClusterSingletonApiSpec extends ScalaTestWithActorTestKit(ClusterSingleton
implicit val testSettings = TestKitSettings(system)
val clusterNode1 = Cluster(system)
val untypedSystem1 = system.toUntyped
val classicSystem1 = system.toClassic
val system2 = akka.actor.ActorSystem(
system.name,

View file

@ -58,12 +58,12 @@ class ClusterSingletonPersistenceSpec
implicit val s = system
implicit val untypedSystem = system.toUntyped
private val untypedCluster = akka.cluster.Cluster(untypedSystem)
implicit val classicSystem = system.toClassic
private val classicCluster = akka.cluster.Cluster(classicSystem)
"A typed cluster singleton with persistent actor" must {
untypedCluster.join(untypedCluster.selfAddress)
classicCluster.join(classicCluster.selfAddress)
"start persistent actor" in {
val ref = ClusterSingleton(system).init(SingletonActor(persistentActor, "singleton").withStopMessage(StopPlz))

View file

@ -32,7 +32,7 @@ class ClusterSingletonPoisonPillSpec
implicit val testSettings = TestKitSettings(system)
val clusterNode1 = Cluster(system)
clusterNode1.manager ! Join(clusterNode1.selfMember.address)
val untypedSystem1 = system.toUntyped
val classicSystem1 = system.toClassic
"A typed cluster singleton" must {
"support using PoisonPill to stop" in {

View file

@ -10,7 +10,7 @@ import akka.Done
import akka.testkit.AkkaSpec
import akka.actor.typed.{ ActorRef, ActorRefResolver }
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.{ ExtendedActorSystem, ActorSystem => UntypedActorSystem }
import akka.actor.{ ExtendedActorSystem, ActorSystem => ClassicActorSystem }
import akka.serialization.SerializerWithStringManifest
import com.typesafe.config.ConfigFactory
import scala.concurrent.Promise
@ -80,7 +80,7 @@ class RemoteMessageSpec extends AkkaSpec(RemoteMessageSpec.config) {
// typed actor on system1
val pingPongActor = system.spawn(ponger, "pingpong")
val system2 = UntypedActorSystem(system.name + "-system2", RemoteMessageSpec.config)
val system2 = ClassicActorSystem(system.name + "-system2", RemoteMessageSpec.config)
val typedSystem2 = system2.toTyped
try {

View file

@ -15,15 +15,15 @@ import org.scalatest.WordSpecLike
class AkkaClusterTypedSerializerSpec extends ScalaTestWithActorTestKit with WordSpecLike {
val ref = spawn(Behaviors.empty[String])
val untypedSystem = system.toUntyped
val serializer = new AkkaClusterTypedSerializer(untypedSystem.asInstanceOf[ExtendedActorSystem])
val classicSystem = system.toClassic
val serializer = new AkkaClusterTypedSerializer(classicSystem.asInstanceOf[ExtendedActorSystem])
"AkkaClusterTypedSerializer" must {
Seq("ReceptionistEntry" -> ClusterReceptionist.Entry(ref, 666L)).foreach {
case (scenario, item) =>
s"resolve serializer for $scenario" in {
val serializer = SerializationExtension(untypedSystem)
val serializer = SerializationExtension(classicSystem)
serializer.serializerFor(item.getClass).getClass should be(classOf[AkkaClusterTypedSerializer])
}

View file

@ -240,7 +240,7 @@ class ClusterReceptionistSpec extends WordSpec with Matchers {
regProbe2.expectMessageType[Listing].serviceInstances(PingKey).size should ===(2)
akka.cluster.Cluster(system1.toUntyped).shutdown()
akka.cluster.Cluster(system1.toClassic).shutdown()
regProbe2.expectNoMessage(3.seconds)

View file

@ -33,7 +33,7 @@ Scala
: @@snip [DispatcherDocSpec.scala](/akka-docs/src/test/scala/docs/dispatcher/DispatcherDocSpec.scala) { #required-mailbox-class }
Java
: @@snip [MyBoundedActor.java](/akka-docs/src/test/java/jdocs/actor/MyBoundedActor.java) { #my-bounded-untyped-actor }
: @@snip [MyBoundedActor.java](/akka-docs/src/test/java/jdocs/actor/MyBoundedActor.java) { #my-bounded-classic-actor }
The type parameter to the `RequiresMessageQueue` @scala[trait]@java[interface] needs to be mapped to a mailbox in
configuration like this:

View file

@ -10,62 +10,39 @@ If you are still using Scala 2.11 then you must upgrade to 2.12 or 2.13
## Removed features that were deprecated
### akka-camel removed
After being deprecated since 2.5.0, the following have been removed in Akka 2.6.
After being deprecated in 2.5.0, the akka-camel module has been removed in 2.6.
As an alternative we recommend [Alpakka](https://doc.akka.io/docs/alpakka/current/).
* akka-camel module
- As an alternative we recommend [Alpakka](https://doc.akka.io/docs/alpakka/current/).
- This is of course not a drop-in replacement. If there is community interest we are open to setting up akka-camel as a separate community-maintained repository.
* akka-agent module
- If there is interest it may be moved to a separate, community-maintained repository.
* akka-contrib module
- To migrate, take the components you are using from [Akka 2.5](https://github.com/akka/akka/tree/release-2.5/akka-contrib) and include them in your own project or library under your own package name.
* Actor DSL
- Actor DSL is a rarely used feature. Use plain `system.actorOf` instead of the DSL to create Actors if you have been using it.
* `akka.stream.extra.Timing` operator
- If you need it you can now find it in `akka.stream.contrib.Timed` from [Akka Stream Contrib](https://github.com/akka/akka-stream-contrib/blob/master/src/main/scala/akka/stream/contrib/Timed.scala).
* Netty UDP (Classic remoting over UDP)
- To continue to use UDP configure @ref[Artery UDP](../remoting-artery.md#configuring-ssl-tls-for-akka-remoting) or migrate to Artery TCP.
- A full cluster restart is required to change to Artery.
* `UntypedActor`
- Use `AbstractActor` instead.
* `JavaTestKit`
- Use `akka.testkit.javadsl.TestKit` instead.
* `UntypedPersistentActor`
- Use `AbstractPersistentActor` instead.
* `UntypedPersistentActorWithAtLeastOnceDelivery`
- Use @apidoc[AbstractPersistentActorWithAtLeastOnceDelivery] instead.
This is of course not a drop-in replacement. If there is community interest we
are open to setting up akka-camel as a separate community-maintained
repository.
After being deprecated since 2.2, the following have been removed in Akka 2.6.
### akka-agent removed
* `actorFor`
- Use `ActorSelection` instead.
### Removed methods
After being deprecated in 2.5.0, the akka-agent module has been removed in 2.6.
If there is interest it may be moved to a separate, community-maintained
repository.
### akka-contrib removed
The akka-contrib module was deprecated in 2.5 and has been removed in 2.6.
To migrate, take the components you are using from [Akka 2.5](https://github.com/akka/akka/tree/release-2.5/akka-contrib)
and include them in your own project or library under your own package name.
### Actor DSL removed
Actor DSL is a rarely used feature and has been deprecated since `2.5.0`.
Use plain `system.actorOf` instead of the DSL to create Actors if you have been using it.
### Timing operator removed
`akka.stream.extra.Timing` has been removed. If you need it you can now find it in `akka.stream.contrib.Timed` from
[Akka Stream Contrib](https://github.com/akka/akka-stream-contrib/blob/master/src/main/scala/akka/stream/contrib/Timed.scala).
### actorFor removed
`actorFor` has been deprecated since `2.2`. Use `ActorSelection` instead.
### Netty UDP removed
Classic remoting over UDP has been deprecated since `2.5.0` and now has been removed.
To continue to use UDP configure @ref[Artery UDP](../remoting-artery.md#configuring-ssl-tls-for-akka-remoting) or migrate to Artery TCP.
A full cluster restart is required to change to Artery.
### Untyped actor removed
`UntypedActor` has been depcated since `2.5.0`. Use `AbstractActor` instead.
### UntypedPersistentActor removed
Use `AbstractPersistentActor` instead.
### UntypedPersistentActorWithAtLeastOnceDelivery removed
Use @apidoc[AbstractPersistentActorWithAtLeastOnceDelivery] instead.
### Various removed methods
* `Logging.getLogger(UntypedActor)` Untyped actor has been removed, use AbstractActor instead.
* `Logging.getLogger(UntypedActor)` `UntypedActor` has been removed, use `AbstractActor` instead.
* `LoggingReceive.create(Receive, ActorContext)` use `AbstractActor.Receive` instead.
* `ActorMaterialzierSettings.withAutoFusing` disabling fusing is no longer possible.
* `AbstractActor.getChild` use `findChild` instead.
@ -76,9 +53,6 @@ Use @apidoc[AbstractPersistentActorWithAtLeastOnceDelivery] instead.
* `Source.actorSubscriber`, use `Source.fromGraph` instead.
* `Source.actorActorPublisher`, use `Source.fromGraph` instead.
### JavaTestKit removed
The `JavaTestKit` has been deprecated since `2.5.0`. Use `akka.testkit.javadsl.TestKit` instead.
## Deprecated features
@ -444,6 +418,12 @@ The materialized value for `StreamRefs.sinkRef` and `StreamRefs.sourceRef` is no
## Akka Typed
### Naming convention changed
In needing a way to distinguish the new APIs in code and docs from the original, Akka used the naming
convention `untyped`. All references of the original have now been changed to `classic`. The
reference of the new APIs as `typed` is going away as it becomes the primary APIs.
### Receptionist has moved
The receptionist had a name clash with the default Cluster Client Receptionist at `/system/receptionist` and will now
@ -472,7 +452,7 @@ made before finalizing the APIs. Compared to Akka 2.5.x the source incompatible
* Factory method `Entity.ofPersistentEntity` is renamed to `Entity.ofEventSourcedEntity` in the Java API for Akka Cluster Sharding Typed.
* New abstract class `EventSourcedEntityWithEnforcedReplies` in Java API for Akka Cluster Sharding Typed and corresponding factory method `Entity.ofEventSourcedEntityWithEnforcedReplies` to ease the creation of `EventSourcedBehavior` with enforced replies.
* New method `EventSourcedEntity.withEnforcedReplies` added to Scala API to ease the creation of `EventSourcedBehavior` with enforced replies.
* `ActorSystem.scheduler` previously gave access to the untyped `akka.actor.Scheduler` but now returns a typed specific `akka.actor.typed.Scheduler`.
* `ActorSystem.scheduler` previously gave access to the classic `akka.actor.Scheduler` but now returns a typed specific `akka.actor.typed.Scheduler`.
Additionally `schedule` method has been replaced by `scheduleWithFixedDelay` and `scheduleAtFixedRate`. Actors that needs to schedule tasks should
prefer `Behaviors.withTimers`.
* `TimerScheduler.startPeriodicTimer`, replaced by `startTimerWithFixedDelay` or `startTimerAtFixedRate`
@ -497,9 +477,9 @@ made before finalizing the APIs. Compared to Akka 2.5.x the source incompatible
#### Akka Typed Stream API changes
* `ActorSource.actorRef` relying on `PartialFunction` has been replaced in the Java API with a variant more suitable to be called by Java.
* `toUntyped` has been renamed to `toClassic`.
## Additional changes
## Materializer changes
### System global Materializer provided

View file

@ -12,7 +12,7 @@ To use Akka Actor Typed, you must add the following dependency in your project:
## Introduction
With @ref:[untyped actors](../general/addressing.md) you would use `ActorSelection` to "lookup" actors. Given an actor path with
With @ref:[classic actors](../general/addressing.md) you would use `ActorSelection` to "lookup" actors. Given an actor path with
address information you can get hold of an `ActorRef` to any actor. `ActorSelection` does not exist in Akka Typed,
so how do you get the actor references? You can send refs in messages but you need something to bootstrap the interaction.

View file

@ -64,7 +64,7 @@ Java
@@@ Note
In the untyped counter part, the @apidoc[akka.actor.ActorSystem], the root actor was provided out of the box and you
In the classic counter part, the @apidoc[akka.actor.ActorSystem], the root actor was provided out of the box and you
could spawn top-level actors from the outside of the `ActorSystem` using `actorOf`. @ref:[SpawnProtocol](#spawnprotocol)
is a tool that mimics the old style of starting up actors.
@ -105,7 +105,7 @@ That is not difficult to implement in your behavior, but since this is a common
message protocol and implementation of a behavior for this. It can be used as the guardian actor of the `ActorSystem`,
possibly combined with `Behaviors.setup` to start some initial tasks or actors. Child actors can then be started from
the outside by telling or asking `SpawnProtocol.Spawn` to the actor reference of the system. When using `ask` this is
similar to how `ActorSystem.actorOf` can be used in untyped actors with the difference that a
similar to how `ActorSystem.actorOf` can be used in classic actors with the difference that a
@scala[`Future`]@java[`CompletionStage`] of the `ActorRef` is returned.
The guardian behavior can be defined as:

View file

@ -13,114 +13,114 @@ To use Akka Actor Typed, you must add the following dependency in your project:
## Introduction
We believe Akka Typed will be adopted in existing systems gradually and therefore it's important to be able to use typed
and untyped actors together, within the same `ActorSystem`. Also, we will not be able to integrate with all existing modules in one big bang release and that is another reason for why these two ways of writing actors must be able to coexist.
and classic actors together, within the same `ActorSystem`. Also, we will not be able to integrate with all existing modules in one big bang release and that is another reason for why these two ways of writing actors must be able to coexist.
There are two different `ActorSystem`s: `akka.actor.ActorSystem` and `akka.actor.typed.ActorSystem`.
Currently the typed actor system is implemented using an untyped actor system under the hood. This may change in the future.
Currently the typed actor system is implemented using the classic actor system under the hood. This may change in the future.
Typed and untyped can interact the following ways:
Typed and classic can interact the following ways:
* untyped actor systems can create typed actors
* typed actors can send messages to untyped actors, and opposite
* spawn and supervise typed child from untyped parent, and opposite
* watch typed from untyped, and opposite
* untyped actor system can be converted to a typed actor system
* classic actor systems can create typed actors
* typed actors can send messages to classic actors, and opposite
* spawn and supervise typed child from classic parent, and opposite
* watch typed from classic, and opposite
* classic actor system can be converted to a typed actor system
@@@ div { .group-scala }
In the examples the `akka.actor` package is aliased to `untyped`.
In the examples the `akka.actor` package is aliased to `classic`.
Scala
: @@snip [UntypedWatchingTypedSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/UntypedWatchingTypedSpec.scala) { #import-alias }
: @@snip [ClassicWatchingTypedSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/ClassicWatchingTypedSpec.scala) { #import-alias }
@@@
@java[The examples use fully qualified class names for the untyped classes to distinguish between typed and untyped classes with the same name.]
@java[The examples use fully qualified class names for the classic classes to distinguish between typed and classic classes with the same name.]
## Untyped to typed
## Classic to typed
While coexisting your application will likely still have an untyped ActorSystem. This can be converted to a typed ActorSystem
so that new code and migrated parts don't rely on the untyped system:
While coexisting your application will likely still have a classic ActorSystem. This can be converted to a typed ActorSystem
so that new code and migrated parts don't rely on the classic system:
Scala
: @@snip [UntypedWatchingTypedSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/UntypedWatchingTypedSpec.scala) { #adapter-import #convert-untyped }
: @@snip [ClassicWatchingTypedSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/ClassicWatchingTypedSpec.scala) { #adapter-import #convert-classic }
Java
: @@snip [UntypedWatchingTypedTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/UntypedWatchingTypedTest.java) { #adapter-import #convert-untyped }
: @@snip [ClassicWatchingTypedTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/ClassicWatchingTypedTest.java) { #adapter-import #convert-classic }
Then for new typed actors here's how you create, watch and send messages to
it from an untyped actor.
it from a classic actor.
Scala
: @@snip [UntypedWatchingTypedSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/UntypedWatchingTypedSpec.scala) { #typed }
: @@snip [ClassicWatchingTypedSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/ClassicWatchingTypedSpec.scala) { #typed }
Java
: @@snip [UntypedWatchingTypedTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/UntypedWatchingTypedTest.java) { #typed }
: @@snip [ClassicWatchingTypedTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/ClassicWatchingTypedTest.java) { #typed }
The top level untyped actor is created in the usual way:
The top level classic actor is created in the usual way:
Scala
: @@snip [UntypedWatchingTypedSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/UntypedWatchingTypedSpec.scala) { #create-untyped }
: @@snip [ClassicWatchingTypedSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/ClassicWatchingTypedSpec.scala) { #create-classic }
Java
: @@snip [UntypedWatchingTypedTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/UntypedWatchingTypedTest.java) { #create-untyped }
: @@snip [ClassicWatchingTypedTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/ClassicWatchingTypedTest.java) { #create-classic }
Then it can create a typed actor, watch it, and send a message to it:
Scala
: @@snip [UntypedWatchingTypedSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/UntypedWatchingTypedSpec.scala) { #untyped-watch }
: @@snip [ClassicWatchingTypedSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/ClassicWatchingTypedSpec.scala) { #classic-watch }
Java
: @@snip [UntypedWatchingTypedTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/UntypedWatchingTypedTest.java) { #untyped-watch }
: @@snip [ClassicWatchingTypedTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/ClassicWatchingTypedTest.java) { #classic-watch }
@scala[There is one `import` that is needed to make that work.] @java[We import the Adapter class and
call static methods for conversion.]
Scala
: @@snip [UntypedWatchingTypedSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/UntypedWatchingTypedSpec.scala) { #adapter-import }
: @@snip [ClassicWatchingTypedSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/ClassicWatchingTypedSpec.scala) { #adapter-import }
Java
: @@snip [UntypedWatchingTypedTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/UntypedWatchingTypedTest.java) { #adapter-import }
: @@snip [ClassicWatchingTypedTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/ClassicWatchingTypedTest.java) { #adapter-import }
@scala[That adds some implicit extension methods that are added to untyped and typed `ActorSystem` and `ActorContext` in both directions.]
@java[To convert between typed and untyped there are adapter methods in `akka.actor.typed.javadsl.Adapter`.] Note the inline comments in the example above.
@scala[That adds some implicit extension methods that are added to classic and typed `ActorSystem` and `ActorContext` in both directions.]
@java[To convert between typed and classic there are adapter methods in `akka.actor.typed.javadsl.Adapter`.] Note the inline comments in the example above.
## Typed to untyped
## Typed to classic
Let's turn the example upside down and first start the typed actor and then the untyped as a child.
Let's turn the example upside down and first start the typed actor and then the classic as a child.
The following will show how to create, watch and send messages back and forth from a typed actor to this
untyped actor:
classic actor:
Scala
: @@snip [TypedWatchingUntypedSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/TypedWatchingUntypedSpec.scala) { #untyped }
: @@snip [TypedWatchingClassicSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/TypedWatchingClassicSpec.scala) { #classic }
Java
: @@snip [TypedWatchingUntypedTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/TypedWatchingUntypedTest.java) { #untyped }
: @@snip [TypedWatchingClassicTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/TypedWatchingClassicTest.java) { #classic }
Creating the actor system and the typed actor:
Scala
: @@snip [TypedWatchingUntypedSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/TypedWatchingUntypedSpec.scala) { #create }
: @@snip [TypedWatchingClassicSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/TypedWatchingClassicSpec.scala) { #create }
Java
: @@snip [TypedWatchingUntypedTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/TypedWatchingUntypedTest.java) { #create }
: @@snip [TypedWatchingClassicTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/TypedWatchingClassicTest.java) { #create }
Then the typed actor creates the untyped actor, watches it and sends and receives a response:
Then the typed actor creates the classic actor, watches it and sends and receives a response:
Scala
: @@snip [TypedWatchingUntypedSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/TypedWatchingUntypedSpec.scala) { #typed }
: @@snip [TypedWatchingClassicSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/coexistence/TypedWatchingClassicSpec.scala) { #typed }
Java
: @@snip [TypedWatchingUntypedTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/TypedWatchingUntypedTest.java) { #typed }
: @@snip [TypedWatchingClassicTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/coexistence/TypedWatchingClassicTest.java) { #typed }
## Supervision
The default supervision for untyped actors is to restart whereas for typed it is to stop.
When combining untyped and typed actors the default supervision is based on the default behavior of
the child i.e. if an untyped actor creates a typed child, its default supervision will be to stop. If a typed
actor creates an untyped child, its default supervision will be to restart.
The default supervision for classic actors is to restart whereas for typed it is to stop.
When combining classic and typed actors the default supervision is based on the default behavior of
the child, for example if a classic actor creates a typed child, its default supervision will be to stop. If a typed
actor creates a classic child, its default supervision will be to restart.

View file

@ -39,7 +39,7 @@ actor provides the API for interacting with the data and is accessed through the
The messages for the replicator, such as `Replicator.Update` are defined in @apidoc[typed.*.Replicator]
but the actual CRDTs are the
same as in untyped, for example `akka.cluster.ddata.GCounter`. This will require a @scala[implicit] `akka.cluster.ddata.SelfUniqueAddress.SelfUniqueAddress`,
same as in classic, for example `akka.cluster.ddata.GCounter`. This will require a @scala[implicit] `akka.cluster.ddata.SelfUniqueAddress.SelfUniqueAddress`,
available from @scala[`implicit val node = DistributedData(system).selfUniqueAddress`]@java[SelfUniqueAddress node = DistributedData.get(system).selfUniqueAddress();].
The replicator can contain multiple entries each containing a replicated data type, we therefore need to create a
@ -79,7 +79,7 @@ the extra interaction with the replicator using the `GetCachedValue` command.
The example also supports asking the replicator using the `GetValue` command. Note how the `replyTo` from the
incoming message can be used when the `GetSuccess` response from the replicator is received.
See the @ref[the untyped Distributed Data documentation](../distributed-data.md#using-the-replicator)
See the @ref:[the classic Distributed Data documentation](../distributed-data.md#using-the-replicator)
for more details about `Get`, `Update` and `Delete` interactions with the replicator.
@@@ div { .group-scala }
@ -98,15 +98,15 @@ Scala
### Replicated data types
Akka contains a set of useful replicated data types and it is fully possible to implement custom replicated data types.
For more details, read @ref[the untyped Distributed Data documentation](../distributed-data.md#data-types)
For more details, read @ref:[the classic Distributed Data documentation](../distributed-data.md#data-types)
### Running separate instances of the replicator
For some use cases, for example when limiting the replicator to certain roles, or using different subsets on different roles,
it makes sense to start separate replicators, this needs to be done on all nodes, or
the group of nodes tagged with a specific role. To do this with the Typed Distributed Data you will first
have to start an untyped `Replicator` and pass it to the `Replicator.behavior` method that takes an untyped
actor ref. All such `Replicator`s must run on the same path in the untyped actor hierarchy.
the group of nodes tagged with a specific role. To do this with Distributed Data you will first
have to start a classic `Replicator` and pass it to the `Replicator.behavior` method that takes a classic
actor ref. All such `Replicator`s must run on the same path in the classic actor hierarchy.
A standalone `ReplicatorMessageAdapter` can also be created for a given `Replicator` instead of creating
one via the `DistributedData` extension.

View file

@ -5,8 +5,8 @@ will by default be stopped.
@@@ note
An important difference between Typed and Untyped actors is that Typed actors are by default stopped if
an exception is thrown and no supervision strategy is defined while in Untyped they are restarted.
An important difference between Typed and Classic actors is that Typed actors are by default stopped if
an exception is thrown and no supervision strategy is defined while in Classic they are restarted.
@@@
@ -26,7 +26,7 @@ with a fresh state that we know is valid.
## Supervision
In Akka Typed this "somewhere else" is called supervision. Supervision allows you to declaratively describe what should happen when a certain type of exceptions are thrown inside an actor. To use supervision the actual Actor behavior is wrapped using `Behaviors.supervise`, for example to restart on `IllegalStateExceptions`:
In Akka this "somewhere else" is called supervision. Supervision allows you to declaratively describe what should happen when a certain type of exceptions are thrown inside an actor. To use supervision the actual Actor behavior is wrapped using `Behaviors.supervise`, for example to restart on `IllegalStateExceptions`:
Scala
@ -114,7 +114,7 @@ restarted.
## Bubble failures up through the hierarchy
In some scenarios it may be useful to push the decision about what to do on a failure upwards in the Actor hierarchy
and let the parent actor handle what should happen on failures (in untyped Akka Actors this is how it works by default).
and let the parent actor handle what should happen on failures (in classic Akka Actors this is how it works by default).
For a parent to be notified when a child is terminated it has to `watch` the child. If the child was stopped because of
a failure the `ChildFailed` signal will be received which will contain the cause. `ChildFailed` extends `Terminated` so if

View file

@ -1,10 +1,10 @@
# Behaviors as Finite state machines
With untyped actors there is explicit support for building @ref[Finite State Machines](../fsm.md). No support
With classic actors there is explicit support for building @ref[Finite State Machines](../fsm.md). No support
is needed in Akka Typed as it is straightforward to represent FSMs with behaviors.
To see how the Akka Typed API can be used to model FSMs here's the Buncher example ported from
the @ref[untyped actor FSM docs](../fsm.md). It demonstrates how to:
the @ref[classic actor FSM docs](../fsm.md). It demonstrates how to:
* Model states using different behaviors
* Model storing data at each state by representing the behavior as a method
@ -22,7 +22,7 @@ Java
`Batches` to be passed on; `Queue` will add to the internal queue while
`Flush` will mark the end of a burst.
Untyped `FSM`s also have a `D` (data) type parameter. Akka Typed doesn't need to be aware of this and it can be stored
Classic `FSM`s also have a `D` (data) type parameter. Akka Typed doesn't need to be aware of this and it can be stored
via defining your behaviors as methods.
Scala

View file

@ -24,7 +24,7 @@ To use Akka Persistence Typed, add the module to your project:
## Introduction
Akka Persistence is a library for building event sourced actors. For background about how it works
see the @ref:[untyped Akka Persistence section](../persistence.md). This documentation shows how the typed API for persistence
see the @ref:[classic Akka Persistence section](../persistence.md). This documentation shows how the typed API for persistence
works and assumes you know what is meant by `Command`, `Event` and `State`.
## Example
@ -323,7 +323,7 @@ command or the reply will be sent later, perhaps after some asynchronous interac
## Serialization
The same @ref:[serialization](../serialization.md) mechanism as for untyped
The same @ref:[serialization](../serialization.md) mechanism as for classic
actors is also used in Akka Typed, also for persistent actors. When picking serialization solution for the events
you should also consider that it must be possible read old events when the application has evolved.
Strategies for that can be found in the @ref:[schema evolution](../persistence-schema-evolution.md).

View file

@ -2,7 +2,7 @@
## Dependency
To use Akka TestKit Typed, add the module to your project:
To use Akka TestKit add the module to your project:
@@dependency[sbt,Maven,Gradle] {
group=com.typesafe.akka
@ -13,7 +13,7 @@ To use Akka TestKit Typed, add the module to your project:
@@@div { .group-scala }
We recommend using Akka TestKit Typed with ScalaTest:
We recommend using Akka TestKit with ScalaTest:
@@dependency[sbt,Maven,Gradle] {
group=org.scalatest
@ -77,8 +77,8 @@ Java
The following demonstrates:
* Creating a typed actor from the `TestKit`'s system using `spawn`
* Creating a typed `TestProbe`
* Creating an actor from the `TestKit`'s system using `spawn`
* Creating a `TestProbe`
* Verifying that the actor under test responds via the `TestProbe`
Scala
@ -117,7 +117,7 @@ run the other actors it depends on. Instead, you might want to create mock behav
messages in the same way the other actor would do but without executing any actual logic.
In addition to this it can also be useful to observe those interactions to assert that the component under test did send
the expected messages.
This allows the same kinds of tests as untyped `TestActor`/`Autopilot`.
This allows the same kinds of tests as classic `TestActor`/`Autopilot`.
As an example, let's assume we'd like to test the following component:

View file

@ -4,10 +4,10 @@
package jdocs.actor;
// #my-bounded-untyped-actor
// #my-bounded-classic-actor
import akka.dispatch.BoundedMessageQueueSemantics;
import akka.dispatch.RequiresMessageQueue;
public class MyBoundedActor extends MyActor
implements RequiresMessageQueue<BoundedMessageQueueSemantics> {}
// #my-bounded-untyped-actor
// #my-bounded-classic-actor

View file

@ -64,7 +64,7 @@ object SnapshotMetadata {
/**
* INTERNAL API
*/
@InternalApi private[akka] def fromUntyped(metadata: akka.persistence.SnapshotMetadata): SnapshotMetadata =
@InternalApi private[akka] def fromClassic(metadata: akka.persistence.SnapshotMetadata): SnapshotMetadata =
new SnapshotMetadata(metadata.persistenceId, metadata.sequenceNr, metadata.timestamp)
}

View file

@ -7,7 +7,7 @@ package akka.persistence.typed
/**
* Facility to convert snapshots from and to a specialized data model.
* Can be used when migration from different state types e.g. when migration
* from Persistent FSM to Typed Persistence.
* from Persistent FSM to Typed EventSourcedBehavior.
*
* @tparam State The state type of the `EventSourcedBehavior`
*/

View file

@ -5,7 +5,7 @@
package akka.persistence.typed
import akka.annotation.InternalApi
import akka.persistence.{ SnapshotSelectionCriteria => UntypedSnapshotSelectionCriteria }
import akka.persistence.{ SnapshotSelectionCriteria => ClassicSnapshotSelectionCriteria }
import akka.util.HashCode
object SnapshotSelectionCriteria {
@ -29,7 +29,7 @@ object SnapshotSelectionCriteria {
/**
* INTERNAL API
*/
@InternalApi private[akka] def fromUntyped(c: UntypedSnapshotSelectionCriteria): SnapshotSelectionCriteria =
@InternalApi private[akka] def fromClassic(c: ClassicSnapshotSelectionCriteria): SnapshotSelectionCriteria =
new SnapshotSelectionCriteria(c.maxSequenceNr, c.maxTimestamp, c.minSequenceNr, c.minTimestamp)
}
@ -82,7 +82,7 @@ final class SnapshotSelectionCriteria @InternalApi private[akka] (
/**
* INTERNAL API
*/
@InternalApi private[akka] def toUntyped: akka.persistence.SnapshotSelectionCriteria =
@InternalApi private[akka] def toClassic: akka.persistence.SnapshotSelectionCriteria =
akka.persistence.SnapshotSelectionCriteria(maxSequenceNr, maxTimestamp, minSequenceNr, minTimestamp)
override def equals(other: Any): Boolean = other match {

View file

@ -55,12 +55,12 @@ private[akka] final class BehaviorSetup[C, E, S](
import akka.actor.typed.scaladsl.adapter._
import BehaviorSetup._
val persistence: Persistence = Persistence(context.system.toUntyped)
val persistence: Persistence = Persistence(context.system.toClassic)
val journal: ActorRef = persistence.journalFor(settings.journalPluginId)
val snapshotStore: ActorRef = persistence.snapshotStoreFor(settings.snapshotPluginId)
def selfUntyped = context.self.toUntyped
def selfClassic: ActorRef = context.self.toClassic
private var mdc: Map[String, Any] = Map.empty
private var _log: OptionVal[Logger] = OptionVal.Some(context.log) // changed when mdc is changed

View file

@ -192,7 +192,7 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State](
override def withSnapshotSelectionCriteria(
selection: SnapshotSelectionCriteria): EventSourcedBehavior[Command, Event, State] = {
copy(recovery = Recovery(selection.toUntyped))
copy(recovery = Recovery(selection.toClassic))
}
override def snapshotWhen(predicate: (State, Event, Long) => Boolean): EventSourcedBehavior[Command, Event, State] =

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