From 6c446b36f23a6879cca75329711e297681f2cfc6 Mon Sep 17 00:00:00 2001 From: Matthew de Detrich Date: Wed, 12 Nov 2025 08:34:39 +0100 Subject: [PATCH] Split ActorSystem creation into javadsl and scaladsl --- .../apache/pekko/actor/ActorSystemTest.java | 3 +- .../actor/ActorSystemDispatcherSpec.scala | 9 +- .../pekko/actor/CoordinatedShutdownSpec.scala | 17 +-- .../org/apache/pekko/actor/DeployerSpec.scala | 4 +- .../pekko/actor/DynamicAccessSpec.scala | 6 +- .../apache/pekko/actor/ExtensionSpec.scala | 16 +-- .../org/apache/pekko/actor/FSMActorSpec.scala | 4 +- .../actor/setup/ActorSystemSetupSpec.scala | 2 +- .../dispatch/DispatcherShutdownSpec.scala | 5 +- .../AddressTerminatedTopicBenchSpec.scala | 3 +- .../apache/pekko/event/EventStreamSpec.scala | 13 +- .../org/apache/pekko/event/LoggerSpec.scala | 10 +- .../pekko/event/LoggingReceiveSpec.scala | 1 + .../apache/pekko/io/TcpConnectionSpec.scala | 2 +- .../pekko/io/TcpIntegrationSpecSupport.scala | 9 +- .../apache/pekko/routing/RoutingSpec.scala | 1 + .../SerializationSetupSpec.scala | 11 +- .../pekko/serialization/SerializeSpec.scala | 10 +- .../ClassicWatchingTypedSpec.scala | 4 +- .../TypedWatchingClassicSpec.scala | 2 +- .../apache/pekko/actor/typed/AskSpec.scala | 2 +- .../pekko/actor/typed/ExtensionsSpec.scala | 8 +- .../ClassicSupervisingTypedSpec.scala | 2 +- .../typed/scaladsl/ActorLoggingSpec.scala | 2 +- .../typed/scaladsl/adapter/AdapterSpec.scala | 2 +- .../internal/adapter/ActorSystemAdapter.scala | 4 +- actor/src/main/resources/reference.conf | 3 + .../org/apache/pekko/actor/ActorSystem.scala | 97 ++++++++++--- .../pekko/actor/CoordinatedShutdown.scala | 6 +- .../pekko/actor/javadsl/ActorSystem.scala | 127 +++++++++++++++++ .../pekko/actor/scaladsl/ActorSystem.scala | 128 ++++++++++++++++++ .../apache/pekko/actor/ActorBenchmark.scala | 8 +- .../pekko/actor/ActorCreationBenchmark.scala | 8 +- .../AffinityPoolComparativeBenchmark.scala | 2 +- .../AffinityPoolIdleCPULevelBenchmark.scala | 5 +- ...AffinityPoolRequestResponseBenchmark.scala | 2 +- .../apache/pekko/actor/BenchmarkActors.scala | 4 +- .../pekko/actor/ForkJoinActorBenchmark.scala | 10 +- .../actor/RouterPoolCreationBenchmark.scala | 6 +- .../pekko/actor/ScheduleBenchmark.scala | 8 +- .../pekko/actor/StashCreationBenchmark.scala | 11 +- .../pekko/actor/TellOnlyBenchmark.scala | 6 +- .../ddata/ORSetSerializationBenchmark.scala | 9 +- .../pekko/dispatch/NodeQueueBenchmark.scala | 6 +- .../LevelDbBatchingBenchmark.scala | 5 +- .../PersistenceActorDeferBenchmark.scala | 7 +- .../PersistentActorBenchmark.scala | 8 +- ...ctorWithAtLeastOnceDeliveryBenchmark.scala | 5 +- .../pekko/remote/artery/CodecBenchmark.scala | 5 +- .../remote/artery/SendQueueBenchmark.scala | 7 +- .../jackson/JacksonSerializationBench.scala | 5 +- .../apache/pekko/stream/AskBenchmark.scala | 5 +- .../pekko/stream/EmptySourceBenchmark.scala | 4 +- .../pekko/stream/FlatMapConcatBenchmark.scala | 7 +- .../pekko/stream/FlatMapMergeBenchmark.scala | 4 +- .../pekko/stream/FlowMapBenchmark.scala | 6 +- .../pekko/stream/FramingBenchmark.scala | 6 +- .../pekko/stream/FusedGraphsBenchmark.scala | 7 +- .../stream/InvokeWithFeedbackBenchmark.scala | 4 +- .../stream/LazyFutureSourceBenchmark.scala | 4 +- .../pekko/stream/MapAsyncBenchmark.scala | 6 +- .../stream/MaterializationBenchmark.scala | 4 +- .../pekko/stream/PartitionHubBenchmark.scala | 7 +- .../pekko/stream/SourceRefBenchmark.scala | 6 +- .../OutputStreamSourceStageBenchmark.scala | 4 +- .../stream/io/FileSourcesBenchmark.scala | 4 +- .../stream/io/FileSourcesScaleBenchmark.scala | 4 +- .../RemoveInternalClusterShardingData.scala | 4 +- ...ndomizedBrainResolverIntegrationSpec.scala | 1 + .../SplitBrainResolverIntegrationSpec.scala | 1 + ...dingRememberEntitiesNewExtractorSpec.scala | 2 +- .../ClusterShardingRememberEntitiesSpec.scala | 3 +- .../ClusterShardingSettingsSpec.scala | 2 +- .../CoordinatedShutdownShardingSpec.scala | 4 +- .../JoinConfigCompatCheckShardingSpec.scala | 2 +- .../PersistentShardingMigrationSpec.scala | 6 +- ...erEntitiesShardIdExtractorChangeSpec.scala | 7 +- .../cluster/sharding/ShardRegionSpec.scala | 2 +- .../passivation/simulator/Simulator.scala | 6 +- .../cluster/client/ClusterClientSpec.scala | 2 +- .../pubsub/DistributedPubSubRestartSpec.scala | 10 +- .../ClusterSingletonLeavingSpeedSpec.scala | 2 +- .../singleton/ClusterSingletonProxySpec.scala | 1 + .../ClusterSingletonRestart2Spec.scala | 2 +- .../ClusterSingletonRestartSpec.scala | 2 +- .../pekko/cluster/typed/ClusterApiSpec.scala | 2 +- .../typed/ClusterSingletonApiSpec.scala | 5 +- .../cluster/typed/RemoteMessageSpec.scala | 4 +- .../pekko/cluster/ClusterDeathWatchSpec.scala | 3 +- .../pekko/cluster/MultiDcSplitBrainSpec.scala | 6 +- .../apache/pekko/cluster/NodeChurnSpec.scala | 1 + .../pekko/cluster/QuickRestartSpec.scala | 5 +- .../cluster/RestartFirstSeedNodeSpec.scala | 2 +- .../pekko/cluster/RestartNode2Spec.scala | 2 +- .../pekko/cluster/RestartNode3Spec.scala | 2 +- .../pekko/cluster/RestartNodeSpec.scala | 2 +- .../org/apache/pekko/cluster/StressSpec.scala | 3 +- .../UnreachableNodeJoinsAgainSpec.scala | 2 +- .../ClusterDeathWatchNotificationSpec.scala | 2 +- .../apache/pekko/cluster/ClusterSpec.scala | 2 +- .../apache/pekko/cluster/ClusterTestKit.scala | 2 +- .../pekko/cluster/DowningProviderSpec.scala | 2 +- .../ShutdownAfterJoinSeedNodesSpec.scala | 2 +- .../pekko/discovery/CompileOnlyTest.java | 2 +- .../pekko/discovery/CompileOnlySpec.scala | 2 +- .../DiscoveryConfigurationSpec.scala | 2 +- .../AggregateServiceDiscoverySpec.scala | 3 +- .../config/ConfigServiceDiscoverySpec.scala | 2 +- .../discovery/dns/DnsDiscoverySpec.scala | 2 +- .../pekko/cluster/ddata/DurableDataSpec.scala | 9 +- .../cluster/ddata/DurablePruningSpec.scala | 8 +- .../cluster/ddata/LocalConcurrencySpec.scala | 2 +- .../pekko/cluster/ddata/LotsOfDataBot.scala | 2 +- .../cluster/ddata/VersionVectorSpec.scala | 2 +- .../ReplicatedDataSerializerSpec.scala | 2 +- .../ReplicatorMessageSerializerSpec.scala | 2 +- .../pekko/remote/testconductor/Player.scala | 4 +- .../pekko/remote/testkit/MultiNodeSpec.scala | 8 +- .../pekko/osgi/ActorSystemActivator.scala | 2 +- .../pekko/osgi/OsgiActorSystemFactory.scala | 2 +- .../pekko/osgi/ActorSystemActivatorTest.scala | 6 +- .../persistence/query/PersistenceQuery.scala | 21 +-- .../query/PersistenceQuerySpec.scala | 7 +- .../leveldb/PersistencePluginProxySpec.scala | 5 +- .../leveldb/SharedLeveldbJournalSpec.scala | 1 + .../serialization/SerializerSpec.scala | 6 +- .../persistence/journal/JournalSpec.scala | 1 + .../snapshot/SnapshotStoreSpec.scala | 1 + .../persistence/testkit/CommonUtils.scala | 3 +- .../PersistentFsmToTypedMigrationSpec.scala | 10 +- .../EndToEndEventAdapterSpec.scala | 6 +- .../SnapshotSerializerMigrationAkkaSpec.scala | 4 +- .../RemoteDeploymentDeathWatchSpec.scala | 2 +- .../RemoteNodeRestartDeathWatchSpec.scala | 9 +- .../remote/RemoteQuarantinePiercingSpec.scala | 7 +- .../pekko/remote/RemoteReDeploymentSpec.scala | 5 +- .../artery/HandshakeRestartReceiverSpec.scala | 6 +- .../RemoteRestartedQuarantinedSpec.scala | 5 +- .../classic/RemoteGatePiercingSpec.scala | 2 +- .../classic/RemoteNodeRestartGateSpec.scala | 4 +- .../RemoteNodeShutdownAndComesBackSpec.scala | 7 +- .../artery/ArteryFailedToBindSpec.scala | 2 +- .../classic/RemotingFailedToBindSpec.scala | 4 +- .../pekko/remote/artery/ArteryTransport.scala | 2 +- .../pekko/remote/artery/Association.scala | 2 +- .../aeron/ArteryAeronUdpTransport.scala | 2 +- .../apache/pekko/remote/DaemonicSpec.scala | 3 +- .../pekko/remote/MessageLoggingSpec.scala | 3 +- .../pekko/remote/RemoteRouterSpec.scala | 1 + .../TransientSerializationErrorSpec.scala | 3 +- .../remote/TypedActorRemoteDeploySpec.scala | 3 +- .../remote/artery/ArteryMultiNodeSpec.scala | 6 +- .../artery/BindCanonicalAddressSpec.scala | 14 +- .../remote/artery/FlushOnShutdownSpec.scala | 2 +- .../remote/artery/RemoteRouterSpec.scala | 1 + .../artery/SystemMessageDeliverySpec.scala | 4 +- .../pekko/remote/artery/tcp/TlsTcpSpec.scala | 4 +- .../RotatingKeysSSLEngineProviderSpec.scala | 2 +- .../pekko/remote/classic/ActorsLeakSpec.scala | 16 +-- .../remote/classic/RemoteInitErrorSpec.scala | 2 +- .../pekko/remote/classic/UntrustedSpec.scala | 2 +- .../transport/PekkoProtocolStressTest.scala | 1 + .../AllowJavaSerializationOffSpec.scala | 7 +- ...erializationTransportInformationSpec.scala | 3 +- .../remote/transport/NettyTransportSpec.scala | 26 ++-- .../jackson/SerializationDocSpec.scala | 2 +- .../jackson/JacksonFactorySpec.scala | 5 +- .../jackson/JacksonSerializerSpec.scala | 9 +- .../pekko/stream/testkit/StreamSpec.scala | 3 +- .../stream/tck/ActorSystemLifecycle.scala | 4 +- .../pekko/stream/tck/FilePublisherTest.scala | 2 +- .../org/apache/pekko/stream/io/TcpSpec.scala | 14 +- .../io/compression/CodecSpecSupport.scala | 2 +- .../scaladsl/AggregateWithBoundarySpec.scala | 2 +- .../scaladsl/SourceFromPublisherSpec.scala | 2 +- .../stream/scaladsl/StreamRefsSpec.scala | 3 +- .../org/apache/pekko/testkit/TestKit.scala | 4 +- .../PekkoJUnitActorSystemResource.java | 2 +- .../pekko/testkit/DefaultTimeoutSpec.scala | 4 +- .../pekko/testkit/ImplicitSenderSpec.scala | 4 +- .../org/apache/pekko/testkit/PekkoSpec.scala | 5 +- 181 files changed, 738 insertions(+), 448 deletions(-) create mode 100644 actor/src/main/scala/org/apache/pekko/actor/javadsl/ActorSystem.scala create mode 100644 actor/src/main/scala/org/apache/pekko/actor/scaladsl/ActorSystem.scala diff --git a/actor-tests/src/test/java/org/apache/pekko/actor/ActorSystemTest.java b/actor-tests/src/test/java/org/apache/pekko/actor/ActorSystemTest.java index bf2ddd042bc..52a799e2252 100644 --- a/actor-tests/src/test/java/org/apache/pekko/actor/ActorSystemTest.java +++ b/actor-tests/src/test/java/org/apache/pekko/actor/ActorSystemTest.java @@ -13,6 +13,7 @@ package org.apache.pekko.actor; +import org.apache.pekko.actor.javadsl.ActorSystem; import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import org.junit.Before; import org.junit.Rule; @@ -39,7 +40,7 @@ public void beforeEach() { @Test public void testGetWhenTerminated() throws Exception { - system.terminate(); + system.terminateAsync(); final CompletionStage cs = system.getWhenTerminated(); cs.toCompletableFuture().get(2, SECONDS); } diff --git a/actor-tests/src/test/scala/org/apache/pekko/actor/ActorSystemDispatcherSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/actor/ActorSystemDispatcherSpec.scala index 82620268eb0..737b9143f96 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/actor/ActorSystemDispatcherSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/actor/ActorSystemDispatcherSpec.scala @@ -54,7 +54,8 @@ class ActorSystemDispatchersSpec extends PekkoSpec(ConfigFactory.parseString(""" val ecProbe = TestProbe() val ec = new SnitchingExecutionContext(ecProbe.ref, ExecutionContexts.global()) - val system2 = ActorSystem(name = "ActorSystemDispatchersSpec-passed-in-ec", defaultExecutionContext = Some(ec)) + val system2 = pekko.actor.scaladsl.ActorSystem(name = "ActorSystemDispatchersSpec-passed-in-ec", + defaultExecutionContext = Some(ec)) try { val ref = system2.actorOf(Props(new Actor { @@ -79,7 +80,7 @@ class ActorSystemDispatchersSpec extends PekkoSpec(ConfigFactory.parseString(""" val ec = new SnitchingExecutionContext(ecProbe.ref, ExecutionContexts.global()) val config = ConfigFactory.parseString("pekko.actor.default-dispatcher.executor = \"fork-join-executor\"") - val system2 = ActorSystem( + val system2 = pekko.actor.scaladsl.ActorSystem( name = "ActorSystemDispatchersSpec-ec-configured", config = Some(config), defaultExecutionContext = Some(ec)) @@ -103,7 +104,7 @@ class ActorSystemDispatchersSpec extends PekkoSpec(ConfigFactory.parseString(""" } "provide a single place to override the internal dispatcher" in { - val sys = ActorSystem( + val sys = pekko.actor.scaladsl.ActorSystem( "ActorSystemDispatchersSpec-override-internal-disp", ConfigFactory.parseString(""" pekko.actor.internal-dispatcher = pekko.actor.default-dispatcher @@ -124,7 +125,7 @@ class ActorSystemDispatchersSpec extends PekkoSpec(ConfigFactory.parseString(""" // using the default for internal dispatcher and passing a pre-existing execution context val system2 = - ActorSystem( + pekko.actor.scaladsl.ActorSystem( name = "ActorSystemDispatchersSpec-passed-in-ec-for-internal", config = Some(ConfigFactory.parseString(""" pekko.actor.internal-dispatcher = pekko.actor.default-dispatcher diff --git a/actor-tests/src/test/scala/org/apache/pekko/actor/CoordinatedShutdownSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/actor/CoordinatedShutdownSpec.scala index d2d75544212..0e49f2a5140 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/actor/CoordinatedShutdownSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/actor/CoordinatedShutdownSpec.scala @@ -533,7 +533,7 @@ class CoordinatedShutdownSpec } "terminate ActorSystem" in { - val sys = ActorSystem(system.name, system.settings.config) + val sys = pekko.actor.scaladsl.ActorSystem(system.name, system.settings.config) try { Await.result(CoordinatedShutdown(sys).run(CustomReason), 10.seconds) should ===(Done) sys.whenTerminated.isCompleted should ===(true) @@ -544,9 +544,9 @@ class CoordinatedShutdownSpec } "be run by ActorSystem.terminate" in { - val sys = ActorSystem(system.name, system.settings.config) + val sys = pekko.actor.scaladsl.ActorSystem(system.name, system.settings.config) try { - Await.result(sys.terminate(), 10.seconds) + Await.result(sys.terminateAsync(), 10.seconds) sys.whenTerminated.isCompleted should ===(true) CoordinatedShutdown(sys).shutdownReason() should ===(Some(CoordinatedShutdown.ActorSystemTerminateReason)) } finally { @@ -555,13 +555,13 @@ class CoordinatedShutdownSpec } "not be run by ActorSystem.terminate when run-by-actor-system-terminate=off" in { - val sys = ActorSystem( + val sys = pekko.actor.scaladsl.ActorSystem( system.name, ConfigFactory .parseString("pekko.coordinated-shutdown.run-by-actor-system-terminate = off") .withFallback(system.settings.config)) try { - Await.result(sys.terminate(), 10.seconds) + Await.result(sys.terminateAsync(), 10.seconds) sys.whenTerminated.isCompleted should ===(true) CoordinatedShutdown(sys).shutdownReason() should ===(None) } finally { @@ -571,7 +571,7 @@ class CoordinatedShutdownSpec "not allow terminate-actor-system=off && run-by-actor-system-terminate=on" in { intercept[ConfigurationException] { - val sys = ActorSystem( + val sys = pekko.actor.scaladsl.ActorSystem( system.name, ConfigFactory .parseString("pekko.coordinated-shutdown.terminate-actor-system = off") @@ -659,7 +659,7 @@ class CoordinatedShutdownSpec } def withCoordinatedShutdown(block: (ActorSystem, CoordinatedShutdown) => Unit): Unit = { - val system = ActorSystem( + val system = pekko.actor.scaladsl.ActorSystem( s"CoordinatedShutdownSpec-terminated-${System.currentTimeMillis()}", ConfigFactory.parseString(""" pekko.coordinated-shutdown.phases { @@ -761,7 +761,8 @@ class CoordinatedShutdownSpec def withSystemRunning(system: ActorSystem, cs: CoordinatedShutdown): Unit private val newSystem = - ActorSystem(systemName, systemConfig.withFallback(system.settings.config)).asInstanceOf[ExtendedActorSystem] + pekko.actor.scaladsl.ActorSystem(systemName, systemConfig.withFallback(system.settings.config)).asInstanceOf[ + ExtendedActorSystem] private var shutdownHooks = Set.empty[Thread] private val mockRuntime = new JVMShutdownHooks { override def addHook(t: Thread): Unit = synchronized { diff --git a/actor-tests/src/test/scala/org/apache/pekko/actor/DeployerSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/actor/DeployerSpec.scala index 1a49950f885..83a46d7f20c 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/actor/DeployerSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/actor/DeployerSpec.scala @@ -156,7 +156,7 @@ class DeployerSpec extends PekkoSpec(DeployerSpec.deployerConf) { ConfigParseOptions.defaults) .withFallback(PekkoSpec.testConf) - shutdown(ActorSystem("invalid-number-of-instances", invalidDeployerConf)) + shutdown(pekko.actor.scaladsl.ActorSystem("invalid-number-of-instances", invalidDeployerConf)) } } @@ -175,7 +175,7 @@ class DeployerSpec extends PekkoSpec(DeployerSpec.deployerConf) { ConfigParseOptions.defaults) .withFallback(PekkoSpec.testConf) - shutdown(ActorSystem("invalid-path", invalidDeployerConf)) + shutdown(pekko.actor.scaladsl.ActorSystem("invalid-path", invalidDeployerConf)) } e.getMessage should include("[ubåt]") e.getMessage should include("[/gul/ubåt]") diff --git a/actor-tests/src/test/scala/org/apache/pekko/actor/DynamicAccessSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/actor/DynamicAccessSpec.scala index 2f1bf94099d..7b13a5bc347 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/actor/DynamicAccessSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/actor/DynamicAccessSpec.scala @@ -14,8 +14,6 @@ package org.apache.pekko.actor import scala.collection.immutable -import scala.concurrent.Await -import scala.concurrent.duration._ import scala.util.{ Failure, Success, Try } import org.scalatest.BeforeAndAfterAll @@ -32,7 +30,7 @@ class TestClassWithDefaultConstructor extends TestSuperclass { } class DynamicAccessSpec extends AnyWordSpec with Matchers with BeforeAndAfterAll { - val system = ActorSystem() + val system = org.apache.pekko.actor.scaladsl.ActorSystem() "The DynamicAccess of a system" should { val dynamicAccess = system.asInstanceOf[ExtendedActorSystem].dynamicAccess @@ -88,7 +86,7 @@ class DynamicAccessSpec extends AnyWordSpec with Matchers with BeforeAndAfterAll } override def afterAll() = { - Await.result(system.terminate(), 10.seconds) + system.close() super.afterAll() } } diff --git a/actor-tests/src/test/scala/org/apache/pekko/actor/ExtensionSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/actor/ExtensionSpec.scala index 62084fb17d4..5721d98dfac 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/actor/ExtensionSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/actor/ExtensionSpec.scala @@ -69,7 +69,7 @@ class ExtensionSpec extends AnyWordSpec with Matchers { "support extensions" in { val config = ConfigFactory.parseString("""pekko.extensions = ["org.apache.pekko.actor.TestExtension"]""") - val system = ActorSystem("extensions", config) + val system = pekko.actor.scaladsl.ActorSystem("extensions", config) // TestExtension is configured and should be loaded at startup system.hasExtension(TestExtension) should ===(true) @@ -80,7 +80,7 @@ class ExtensionSpec extends AnyWordSpec with Matchers { } "handle extensions that fail to initialize" in { - val system = ActorSystem("extensions") + val system = pekko.actor.scaladsl.ActorSystem("extensions") // First attempt, an actor is created and after that // an exception is thrown: @@ -98,7 +98,7 @@ class ExtensionSpec extends AnyWordSpec with Matchers { "fail the actor system if an extension listed in pekko.extensions fails to start" in { intercept[RuntimeException] { - val system = ActorSystem( + val system = pekko.actor.scaladsl.ActorSystem( "failing", ConfigFactory.parseString(""" pekko.extensions = ["org.apache.pekko.actor.FailingTestExtension"] @@ -109,7 +109,7 @@ class ExtensionSpec extends AnyWordSpec with Matchers { } "log an error if an extension listed in pekko.extensions cannot be loaded" in { - val system = ActorSystem( + val system = pekko.actor.scaladsl.ActorSystem( "failing", ConfigFactory.parseString(""" pekko.extensions = ["org.apache.pekko.actor.MissingExtension"] @@ -123,7 +123,7 @@ class ExtensionSpec extends AnyWordSpec with Matchers { import pekko.util.ccompat.JavaConverters._ // could be initialized by other tests, but assuming tests are not running in parallel val countBefore = InstanceCountingExtension.createCount.get() - val system = ActorSystem("extensions") + val system = pekko.actor.scaladsl.ActorSystem("extensions") val listedExtensions = system.settings.config.getStringList("pekko.library-extensions").asScala listedExtensions.count(_.contains("InstanceCountingExtension")) should ===(1) @@ -136,7 +136,7 @@ class ExtensionSpec extends AnyWordSpec with Matchers { import pekko.util.ccompat.JavaConverters._ // could be initialized by other tests, but assuming tests are not running in parallel val countBefore = InstanceCountingExtension.createCount.get() - val system = ActorSystem( + val system = pekko.actor.scaladsl.ActorSystem( "extensions", ConfigFactory.parseString( """ @@ -152,7 +152,7 @@ class ExtensionSpec extends AnyWordSpec with Matchers { "fail the actor system if a library-extension fails to start" in { intercept[FailingTestExtension.TestException] { - ActorSystem( + pekko.actor.scaladsl.ActorSystem( "failing", ConfigFactory.parseString(""" pekko.library-extensions += "org.apache.pekko.actor.FailingTestExtension" @@ -163,7 +163,7 @@ class ExtensionSpec extends AnyWordSpec with Matchers { "fail the actor system if a library-extension cannot be loaded" in { intercept[RuntimeException] { - ActorSystem( + pekko.actor.scaladsl.ActorSystem( "failing", ConfigFactory.parseString(""" pekko.library-extensions += "org.apache.pekko.actor.MissingExtension" diff --git a/actor-tests/src/test/scala/org/apache/pekko/actor/FSMActorSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/actor/FSMActorSpec.scala index e04ebdaa4e2..24c22dcd264 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/actor/FSMActorSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/actor/FSMActorSpec.scala @@ -264,7 +264,7 @@ class FSMActorSpec extends PekkoSpec(Map("pekko.actor.debug.fsm" -> true)) with val config = ConfigFactory .parseMap(Map("pekko.loglevel" -> "DEBUG", "pekko.actor.debug.fsm" -> true).asJava) .withFallback(system.settings.config) - val fsmEventSystem = ActorSystem("fsmEvent", config) + val fsmEventSystem = pekko.actor.scaladsl.ActorSystem("fsmEvent", config) try { new TestKit(fsmEventSystem) { EventFilter.debug(occurrences = 5).intercept { @@ -351,7 +351,7 @@ class FSMActorSpec extends PekkoSpec(Map("pekko.actor.debug.fsm" -> true)) with } "allow cancelling stateTimeout by issuing forMax(Duration.Inf)" taggedAs TimingTest in { - val sys = ActorSystem("fsmEvent") + val sys = pekko.actor.scaladsl.ActorSystem("fsmEvent") val p = TestProbe()(sys) val OverrideTimeoutToInf = "override-timeout-to-inf" diff --git a/actor-tests/src/test/scala/org/apache/pekko/actor/setup/ActorSystemSetupSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/actor/setup/ActorSystemSetupSpec.scala index b4dbcd31ef4..945cd9590b8 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/actor/setup/ActorSystemSetupSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/actor/setup/ActorSystemSetupSpec.scala @@ -68,7 +68,7 @@ class ActorSystemSetupSpec extends AnyWordSpec with Matchers { var system: ActorSystem = null try { val setup = DummySetup("Tad Moore") - system = ActorSystem("name", ActorSystemSetup(setup)) + system = pekko.actor.scaladsl.ActorSystem("name", ActorSystemSetup(setup)) (system.settings.setup.get[DummySetup]: Option[Setup]) should ===(Some(setup)) diff --git a/actor-tests/src/test/scala/org/apache/pekko/dispatch/DispatcherShutdownSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/dispatch/DispatcherShutdownSpec.scala index 465f4041e88..def5334c067 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/dispatch/DispatcherShutdownSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/dispatch/DispatcherShutdownSpec.scala @@ -22,7 +22,6 @@ import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec import org.apache.pekko -import pekko.actor.ActorSystem import pekko.testkit.TestKit class DispatcherShutdownSpec extends AnyWordSpec with Matchers { @@ -42,10 +41,10 @@ class DispatcherShutdownSpec extends AnyWordSpec with Matchers { "DispatcherShutdownSpec-pekko.actor.internal")) // nothing is run on default without any user actors started .size - val system = ActorSystem("DispatcherShutdownSpec") + val system = pekko.actor.scaladsl.ActorSystem("DispatcherShutdownSpec") threadCount should be > 0 - Await.ready(system.terminate(), 1.second) + Await.ready(system.terminateAsync(), 1.second) Await.ready(Future(pekko.Done)(system.dispatcher), 1.second) TestKit.awaitCond(threadCount == 0, 3.second) diff --git a/actor-tests/src/test/scala/org/apache/pekko/event/AddressTerminatedTopicBenchSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/event/AddressTerminatedTopicBenchSpec.scala index 92155843844..929d38ebc62 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/event/AddressTerminatedTopicBenchSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/event/AddressTerminatedTopicBenchSpec.scala @@ -18,7 +18,6 @@ import scala.concurrent.duration._ import org.apache.pekko import pekko.actor.Actor import pekko.actor.ActorRef -import pekko.actor.ActorSystem import pekko.actor.Props import pekko.testkit._ @@ -42,7 +41,7 @@ class AddressTerminatedTopicBenchSpec extends PekkoSpec("pekko.loglevel=INFO") { "Subscribe and unsubscribe of AddressTerminated" must { "be quick" in { - val sys = ActorSystem(system.name + "2", system.settings.config) + val sys = pekko.actor.scaladsl.ActorSystem(system.name + "2", system.settings.config) try { val num = 20000 diff --git a/actor-tests/src/test/scala/org/apache/pekko/event/EventStreamSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/event/EventStreamSpec.scala index ac19fb1c182..5855dce44f8 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/event/EventStreamSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/event/EventStreamSpec.scala @@ -113,7 +113,7 @@ class EventStreamSpec extends PekkoSpec(EventStreamSpec.config) { } "be able to log unhandled messages" in { - val sys = ActorSystem("EventStreamSpecUnhandled", configUnhandled) + val sys = pekko.actor.scaladsl.ActorSystem("EventStreamSpecUnhandled", configUnhandled) try { sys.eventStream.subscribe(testActor, classOf[AnyRef]) val m = UnhandledMessage(42, sys.deadLetters, sys.deadLetters) @@ -296,7 +296,7 @@ class EventStreamSpec extends PekkoSpec(EventStreamSpec.config) { } "unsubscribe an actor on its termination" in { - val sys = ActorSystem("EventStreamSpecUnsubscribeOnTerminated", configUnhandledWithDebug) + val sys = pekko.actor.scaladsl.ActorSystem("EventStreamSpecUnsubscribeOnTerminated", configUnhandledWithDebug) try { val es = sys.eventStream @@ -325,7 +325,7 @@ class EventStreamSpec extends PekkoSpec(EventStreamSpec.config) { } "unsubscribe the actor, when it subscribes already in terminated state" in { - val sys = ActorSystem("EventStreamSpecUnsubscribeTerminated", configUnhandledWithDebug) + val sys = pekko.actor.scaladsl.ActorSystem("EventStreamSpecUnsubscribeTerminated", configUnhandledWithDebug) try { val es = sys.eventStream @@ -356,7 +356,7 @@ class EventStreamSpec extends PekkoSpec(EventStreamSpec.config) { } "not allow initializing a TerminatedUnsubscriber twice" in { - val sys = ActorSystem("MustNotAllowDoubleInitOfTerminatedUnsubscriber") + val sys = pekko.actor.scaladsl.ActorSystem("MustNotAllowDoubleInitOfTerminatedUnsubscriber") // initializes an TerminatedUnsubscriber during start try { @@ -373,7 +373,7 @@ class EventStreamSpec extends PekkoSpec(EventStreamSpec.config) { } "unwatch an actor from unsubscriber when that actor unsubscribes from the stream" in { - val sys = ActorSystem("MustUnregisterDuringUnsubscribe", configUnhandledWithDebug) + val sys = pekko.actor.scaladsl.ActorSystem("MustUnregisterDuringUnsubscribe", configUnhandledWithDebug) try { val es = sys.eventStream @@ -393,7 +393,8 @@ class EventStreamSpec extends PekkoSpec(EventStreamSpec.config) { } "unwatch an actor from unsubscriber when that actor unsubscribes from channels it subscribed" in { - val sys = ActorSystem("MustUnregisterWhenNoMoreChannelSubscriptions", configUnhandledWithDebug) + val sys = + pekko.actor.scaladsl.ActorSystem("MustUnregisterWhenNoMoreChannelSubscriptions", configUnhandledWithDebug) try { val es = sys.eventStream diff --git a/actor-tests/src/test/scala/org/apache/pekko/event/LoggerSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/event/LoggerSpec.scala index 409027a5e9c..84a2185fe8d 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/event/LoggerSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/event/LoggerSpec.scala @@ -157,7 +157,7 @@ class LoggerSpec extends AnyWordSpec with Matchers { private def createSystemAndLogToBuffer(name: String, config: Config, shouldLog: Boolean) = { val out = new java.io.ByteArrayOutputStream() Console.withOut(out) { - implicit val system = ActorSystem(name, config) + implicit val system = pekko.actor.scaladsl.ActorSystem(name, config) try { val probe = TestProbe() system.eventStream.publish(SetTarget(probe.ref, qualifier = 1)) @@ -189,7 +189,7 @@ class LoggerSpec extends AnyWordSpec with Matchers { "drain logger queue on system.terminate" in { val out = new java.io.ByteArrayOutputStream() Console.withOut(out) { - val sys = ActorSystem("defaultLogger", slowConfig) + val sys = pekko.actor.scaladsl.ActorSystem("defaultLogger", slowConfig) sys.log.error("msg1") sys.log.error("msg2") sys.log.error("msg3") @@ -218,7 +218,7 @@ class LoggerSpec extends AnyWordSpec with Matchers { "use several loggers" in { Console.withOut(new java.io.ByteArrayOutputStream()) { - implicit val system = ActorSystem("multipleLoggers", multipleConfig) + implicit val system = pekko.actor.scaladsl.ActorSystem("multipleLoggers", multipleConfig) try { val probe1 = TestProbe() val probe2 = TestProbe() @@ -240,7 +240,7 @@ class LoggerSpec extends AnyWordSpec with Matchers { "Ticket 3671" must { "log message with given MDC values" in { - implicit val system = ActorSystem("ticket-3671", ticket3671Config) + implicit val system = pekko.actor.scaladsl.ActorSystem("ticket-3671", ticket3671Config) try { val probe = TestProbe() system.eventStream.publish(SetTarget(probe.ref, qualifier = 1)) @@ -346,7 +346,7 @@ class LoggerSpec extends AnyWordSpec with Matchers { "Ticket 3165 - serialize-messages and dual-entry serialization of LogEvent" must { "not cause StackOverflowError" in { - implicit val s = ActorSystem("foo", ticket3165Config) + implicit val s = pekko.actor.scaladsl.ActorSystem("foo", ticket3165Config) try { SerializationExtension(s).serialize(Warning("foo", classOf[String])) } finally { diff --git a/actor-tests/src/test/scala/org/apache/pekko/event/LoggingReceiveSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/event/LoggingReceiveSpec.scala index 910aa7f814d..72ddf77c28c 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/event/LoggingReceiveSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/event/LoggingReceiveSpec.scala @@ -22,6 +22,7 @@ import org.scalatest.wordspec.AnyWordSpec import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.testkit._ import pekko.util.ccompat.JavaConverters._ diff --git a/actor-tests/src/test/scala/org/apache/pekko/io/TcpConnectionSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/io/TcpConnectionSpec.scala index 1c654892143..5ab2fa30fe8 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/io/TcpConnectionSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/io/TcpConnectionSpec.scala @@ -387,7 +387,7 @@ class TcpConnectionSpec extends PekkoSpec(""" // override config to decrease default buffer size def config = ConfigFactory.parseString("pekko.io.tcp.direct-buffer-size = 1k").withFallback(PekkoSpec.testConf) - override implicit lazy val system: ActorSystem = ActorSystem("respectPullModeTest", config) + override implicit lazy val system: ActorSystem = pekko.actor.scaladsl.ActorSystem("respectPullModeTest", config) try run { val maxBufferSize = 1 * 1024 diff --git a/actor-tests/src/test/scala/org/apache/pekko/io/TcpIntegrationSpecSupport.scala b/actor-tests/src/test/scala/org/apache/pekko/io/TcpIntegrationSpecSupport.scala index 25f76c5b81f..712691eef35 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/io/TcpIntegrationSpecSupport.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/io/TcpIntegrationSpecSupport.scala @@ -20,8 +20,7 @@ import Tcp._ import org.apache.pekko import pekko.actor.ActorRef -import pekko.actor.ActorSystem -import pekko.dispatch.ExecutionContexts +import pekko.actor.scaladsl.ActorSystem import pekko.io.Inet.SocketOption import pekko.testkit.{ PekkoSpec, TestProbe } import pekko.testkit.SocketUtil.temporaryServerAddress @@ -33,9 +32,9 @@ trait TcpIntegrationSpecSupport { this: PekkoSpec => if (runClientInExtraSystem) { val res = ActorSystem("TcpIntegrationSpec-client", system.settings.config) // terminate clientSystem after server system - system.whenTerminated.onComplete { _ => - res.terminate() - }(ExecutionContexts.parasitic) + system.registerOnTermination { + res.terminateAsync() + } res } else system val bindHandler = TestProbe() diff --git a/actor-tests/src/test/scala/org/apache/pekko/routing/RoutingSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/routing/RoutingSpec.scala index eed7e6bb81f..0e9b29ee259 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/routing/RoutingSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/routing/RoutingSpec.scala @@ -23,6 +23,7 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.ConfigurationException import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.pattern.{ ask, pipe } import pekko.testkit._ diff --git a/actor-tests/src/test/scala/org/apache/pekko/serialization/SerializationSetupSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/serialization/SerializationSetupSpec.scala index 9b8aac0896f..78ae6abcf65 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/serialization/SerializationSetupSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/serialization/SerializationSetupSpec.scala @@ -80,7 +80,7 @@ object SerializationSetupSpec { None) val actorSystemSettings = ActorSystemSetup(bootstrapSettings, serializationSettings) - val noJavaSerializationSystem = ActorSystem( + val noJavaSerializationSystem = pekko.actor.scaladsl.ActorSystem( "SerializationSettingsSpec" + "NoJavaSerialization", ConfigFactory.parseString(""" pekko { @@ -96,7 +96,8 @@ object SerializationSetupSpec { } class SerializationSetupSpec - extends PekkoSpec(ActorSystem("SerializationSettingsSpec", SerializationSetupSpec.actorSystemSettings)) { + extends PekkoSpec(pekko.actor.scaladsl.ActorSystem("SerializationSettingsSpec", + SerializationSetupSpec.actorSystemSettings)) { import SerializationSetupSpec._ @@ -129,8 +130,8 @@ class SerializationSetupSpec """).withFallback(ConfigFactory.load()) a[ClassNotFoundException] should be thrownBy { - val system = ActorSystem("SerializationSetupSpec-FailingSystem", config) - system.terminate() + val system = pekko.actor.scaladsl.ActorSystem("SerializationSetupSpec-FailingSystem", config) + system.terminateAsync() } } @@ -163,7 +164,7 @@ class SerializationSetupSpec None) val addedJavaSerializationViaSettingsSystem = - ActorSystem( + pekko.actor.scaladsl.ActorSystem( "addedJavaSerializationSystem", ActorSystemSetup(addedJavaSerializationProgramaticallyButDisabledSettings, addedJavaSerializationSettings)) diff --git a/actor-tests/src/test/scala/org/apache/pekko/serialization/SerializeSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/serialization/SerializeSpec.scala index cb379ee8541..9538f5585aa 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/serialization/SerializeSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/serialization/SerializeSpec.scala @@ -129,7 +129,7 @@ object SerializationTests { def mostlyReferenceSystem: ActorSystem = { val referenceConf = ConfigFactory.defaultReference() val mostlyReferenceConf = PekkoSpec.testConf.withFallback(referenceConf) - ActorSystem("SerializationSystem", mostlyReferenceConf) + pekko.actor.scaladsl.ActorSystem("SerializationSystem", mostlyReferenceConf) } def allowJavaSerializationSystem: ActorSystem = { @@ -141,7 +141,7 @@ object SerializationTests { """) .withFallback(ConfigFactory.parseString(serializeConf)) .withFallback(PekkoSpec.testConf.withFallback(referenceConf)) - ActorSystem("SerializationSystem", conf) + pekko.actor.scaladsl.ActorSystem("SerializationSystem", conf) } val systemMessageMultiSerializerConf = """ @@ -282,7 +282,7 @@ class SerializeSpec extends PekkoSpec(SerializationTests.serializeConf) { "detect duplicate serializer ids" in { (intercept[IllegalArgumentException] { - val sys = ActorSystem( + val sys = pekko.actor.scaladsl.ActorSystem( "SerializeSpec", ConfigFactory.parseString(s""" pekko { @@ -491,7 +491,7 @@ class AllowJavaSerializationSpec extends PekkoSpec(SerializationTests.allowJavaS "serialize DeadLetterActorRef" in { val outbuf = new ByteArrayOutputStream() val out = new ObjectOutputStream(outbuf) - val a = ActorSystem("SerializeDeadLeterActorRef", PekkoSpec.testConf) + val a = pekko.actor.scaladsl.ActorSystem("SerializeDeadLeterActorRef", PekkoSpec.testConf) try { out.writeObject(a.deadLetters) out.flush() @@ -568,7 +568,7 @@ class SerializerDeadlockSpec extends PekkoSpec { "not be accessed from constructor of serializer" in { intercept[IllegalStateException] { - val sys = ActorSystem( + val sys = pekko.actor.scaladsl.ActorSystem( "SerializerDeadlockSpec", ConfigFactory.parseString(""" pekko { diff --git a/actor-typed-tests/src/test/scala/docs/org/apache/pekko/typed/coexistence/ClassicWatchingTypedSpec.scala b/actor-typed-tests/src/test/scala/docs/org/apache/pekko/typed/coexistence/ClassicWatchingTypedSpec.scala index 32504583e02..3bda3ee1f4f 100644 --- a/actor-typed-tests/src/test/scala/docs/org/apache/pekko/typed/coexistence/ClassicWatchingTypedSpec.scala +++ b/actor-typed-tests/src/test/scala/docs/org/apache/pekko/typed/coexistence/ClassicWatchingTypedSpec.scala @@ -89,7 +89,7 @@ class ClassicWatchingTypedSpec extends AnyWordSpec with LogCapturing { "Classic -> Typed" must { "support creating, watching and messaging" in { - val system = classic.ActorSystem("Coexistence") + val system = classic.scaladsl.ActorSystem("Coexistence") // #create-classic val classicActor = system.actorOf(Classic.props()) // #create-classic @@ -102,7 +102,7 @@ class ClassicWatchingTypedSpec extends AnyWordSpec with LogCapturing { "support converting a classic actor system to an actor system" in { // #convert-classic - val system = pekko.actor.ActorSystem("ClassicToTypedSystem") + val system = pekko.actor.scaladsl.ActorSystem("ClassicToTypedSystem") val typedSystem: ActorSystem[Nothing] = system.toTyped // #convert-classic typedSystem.scheduler // remove compile warning diff --git a/actor-typed-tests/src/test/scala/docs/org/apache/pekko/typed/coexistence/TypedWatchingClassicSpec.scala b/actor-typed-tests/src/test/scala/docs/org/apache/pekko/typed/coexistence/TypedWatchingClassicSpec.scala index b48d7359336..8448ed2974e 100644 --- a/actor-typed-tests/src/test/scala/docs/org/apache/pekko/typed/coexistence/TypedWatchingClassicSpec.scala +++ b/actor-typed-tests/src/test/scala/docs/org/apache/pekko/typed/coexistence/TypedWatchingClassicSpec.scala @@ -84,7 +84,7 @@ class TypedWatchingClassicSpec extends AnyWordSpec with LogCapturing { "Typed -> Classic" must { "support creating, watching and messaging" in { // #create - val system = classic.ActorSystem("TypedWatchingClassic") + val system = classic.scaladsl.ActorSystem("TypedWatchingClassic") val typed = system.spawn(Typed.behavior, "Typed") // #create val probe = TestProbe()(system) diff --git a/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/AskSpec.scala b/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/AskSpec.scala index 0a6b650dd08..1c26f867c4b 100644 --- a/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/AskSpec.scala +++ b/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/AskSpec.scala @@ -192,7 +192,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 classicSystem = pekko.actor.ActorSystem("AskSpec-classic-1") + implicit val classicSystem = pekko.actor.scaladsl.ActorSystem("AskSpec-classic-1") try { val ex = new RuntimeException("not good!") diff --git a/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/ExtensionsSpec.scala b/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/ExtensionsSpec.scala index 17bfa5d4ce2..b439074fdf4 100644 --- a/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/ExtensionsSpec.scala +++ b/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/ExtensionsSpec.scala @@ -204,7 +204,7 @@ class ExtensionsSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with "load registered extensions eagerly even for classic system" in { import pekko.actor.typed.scaladsl.adapter._ val beforeCreation = InstanceCountingExtension.createCount.get() - val classicSystem = pekko.actor.ActorSystem("as", ExtensionsSpec.config) + val classicSystem = pekko.actor.scaladsl.ActorSystem("as", ExtensionsSpec.config) try { val before = InstanceCountingExtension.createCount.get() InstanceCountingExtension(classicSystem.toTyped) @@ -215,13 +215,13 @@ class ExtensionsSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with before shouldEqual beforeCreation + 1 after shouldEqual before } finally { - classicSystem.terminate().futureValue + classicSystem.close() } } "not create an extension multiple times when using the ActorSystemAdapter" in { import pekko.actor.typed.scaladsl.adapter._ - val classicSystem = pekko.actor.ActorSystem() + val classicSystem = pekko.actor.scaladsl.ActorSystem() try { val ext1 = DummyExtension1(classicSystem.toTyped) val ext2 = DummyExtension1(classicSystem.toTyped) @@ -229,7 +229,7 @@ class ExtensionsSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with (ext1 should be).theSameInstanceAs(ext2) } finally { - classicSystem.terminate().futureValue + classicSystem.close() } } diff --git a/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/coexistence/ClassicSupervisingTypedSpec.scala b/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/coexistence/ClassicSupervisingTypedSpec.scala index 3014042166d..98ec5c54e59 100644 --- a/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/coexistence/ClassicSupervisingTypedSpec.scala +++ b/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/coexistence/ClassicSupervisingTypedSpec.scala @@ -61,7 +61,7 @@ class ClassicSupervisingTypedSpec extends AnyWordSpecLike with LogCapturing with import ClassicSupervisingTypedSpec._ - val classicSystem = pekko.actor.ActorSystem( + val classicSystem = pekko.actor.scaladsl.ActorSystem( "ClassicSupervisingTypedSpec", ConfigFactory.parseString(""" pekko.actor.testkit.typed.expect-no-message-default = 50 ms diff --git a/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/scaladsl/ActorLoggingSpec.scala b/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/scaladsl/ActorLoggingSpec.scala index 769855a14c1..c7a947ba03c 100644 --- a/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/scaladsl/ActorLoggingSpec.scala +++ b/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/scaladsl/ActorLoggingSpec.scala @@ -305,7 +305,7 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit(""" } "by default be amended to use Slf4jLogger when starting classic ActorSystem" in { - val classicSys = pekko.actor.ActorSystem(system.name) + val classicSys = pekko.actor.scaladsl.ActorSystem(system.name) try { classicSys.settings.config.getStringList("pekko.loggers").size() should ===(1) classicSys.settings.config.getStringList("pekko.loggers").get(0) should ===(classOf[Slf4jLogger].getName) diff --git a/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/scaladsl/adapter/AdapterSpec.scala b/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/scaladsl/adapter/AdapterSpec.scala index 0118fd2aacd..76bc7ceb513 100644 --- a/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/scaladsl/adapter/AdapterSpec.scala +++ b/actor-typed-tests/src/test/scala/org/apache/pekko/actor/typed/scaladsl/adapter/AdapterSpec.scala @@ -196,7 +196,7 @@ object AdapterSpec { class AdapterSpec extends AnyWordSpec with Matchers with BeforeAndAfterAll with LogCapturing { import AdapterSpec._ - implicit val system: classic.ActorSystem = pekko.actor.ActorSystem("AdapterSpec") + implicit val system: classic.ActorSystem = pekko.actor.scaladsl.ActorSystem("AdapterSpec") def typedSystem: ActorSystem[Nothing] = system.toTyped "ActorSystem adaption" must { diff --git a/actor-typed/src/main/scala/org/apache/pekko/actor/typed/internal/adapter/ActorSystemAdapter.scala b/actor-typed/src/main/scala/org/apache/pekko/actor/typed/internal/adapter/ActorSystemAdapter.scala index 039da8f7be1..78d3cd79142 100644 --- a/actor-typed/src/main/scala/org/apache/pekko/actor/typed/internal/adapter/ActorSystemAdapter.scala +++ b/actor-typed/src/main/scala/org/apache/pekko/actor/typed/internal/adapter/ActorSystemAdapter.scala @@ -116,9 +116,9 @@ import pekko.util.FutureConverters._ import org.apache.pekko.dispatch.ExecutionContexts.parasitic - override def terminate(): Unit = system.terminate() + override def terminate(): Unit = system.terminateImpl() override lazy val whenTerminated: scala.concurrent.Future[pekko.Done] = - system.whenTerminated.map(_ => Done)(parasitic) + system.whenTerminatedImpl.map(_ => Done)(parasitic) override lazy val getWhenTerminated: CompletionStage[pekko.Done] = whenTerminated.asJava diff --git a/actor/src/main/resources/reference.conf b/actor/src/main/resources/reference.conf index 356d5a00845..539b12bf81f 100644 --- a/actor/src/main/resources/reference.conf +++ b/actor/src/main/resources/reference.conf @@ -1212,6 +1212,9 @@ pekko { # Terminate the ActorSystem in the last phase actor-system-terminate. terminate-actor-system = on + # The timeout that will be used when calling .close on an ActorSystem + close-actor-system-timeout = 60 s + # Exit the JVM (System.exit(0)) in the last phase actor-system-terminate # if this is set to 'on'. It is done after termination of the # ActorSystem if terminate-actor-system=on, otherwise it is done diff --git a/actor/src/main/scala/org/apache/pekko/actor/ActorSystem.scala b/actor/src/main/scala/org/apache/pekko/actor/ActorSystem.scala index d0505e309e6..a7258873ae0 100644 --- a/actor/src/main/scala/org/apache/pekko/actor/ActorSystem.scala +++ b/actor/src/main/scala/org/apache/pekko/actor/ActorSystem.scala @@ -17,17 +17,13 @@ import java.io.Closeable import java.util.Optional import java.util.concurrent._ import java.util.concurrent.atomic.AtomicReference - import scala.annotation.tailrec import scala.collection.immutable -import scala.concurrent.{ ExecutionContext, ExecutionContextExecutor, Future, Promise } -import scala.concurrent.blocking +import scala.concurrent.{ blocking, Await, ExecutionContext, ExecutionContextExecutor, Future, Promise } import scala.concurrent.duration.Duration import scala.util.{ Failure, Success, Try } import scala.util.control.{ ControlThrowable, NonFatal } - import com.typesafe.config.{ Config, ConfigFactory } - import org.apache.pekko import pekko.ConfigurationException import pekko.actor.dungeon.ChildrenContainer @@ -189,6 +185,7 @@ object ActorSystem { * associated with the ActorSystem class. * Then it loads the default reference configuration using the ClassLoader. */ + @deprecated("Use org.apache.pekko.actor.javadsl.ActorSystem.create instead", "1.3.0") def create(): ActorSystem = apply() /** @@ -198,18 +195,21 @@ object ActorSystem { * associated with the ActorSystem class. * Then it loads the default reference configuration using the ClassLoader. */ + @deprecated("Use org.apache.pekko.actor.javadsl.ActorSystem.create instead", "1.3.0") def create(name: String): ActorSystem = apply(name) /** * Java API: Creates a new actor system with the specified name and settings * The core actor system settings are defined in [[BootstrapSetup]] */ + @deprecated("Use org.apache.pekko.actor.javadsl.ActorSystem.create instead", "1.3.0") def create(name: String, setups: ActorSystemSetup): ActorSystem = apply(name, setups) /** * Java API: Shortcut for creating an actor system with custom bootstrap settings. * Same behavior as calling `ActorSystem.create(name, ActorSystemSetup.create(bootstrapSettings))` */ + @deprecated("Use org.apache.pekko.actor.javadsl.ActorSystem.create instead", "1.3.0") def create(name: String, bootstrapSetup: BootstrapSetup): ActorSystem = create(name, ActorSystemSetup.create(bootstrapSetup)) @@ -221,6 +221,7 @@ object ActorSystem { * * @see The Typesafe Config Library API Documentation */ + @deprecated("Use org.apache.pekko.actor.javadsl.ActorSystem.create instead", "1.3.0") def create(name: String, config: Config): ActorSystem = apply(name, config) /** @@ -228,6 +229,7 @@ object ActorSystem { * * @see The Typesafe Config Library API Documentation */ + @deprecated("Use org.apache.pekko.actor.javadsl.ActorSystem.create instead", "1.3.0") def create(name: String, config: Config, classLoader: ClassLoader): ActorSystem = apply(name, config, classLoader) /** @@ -245,6 +247,7 @@ object ActorSystem { * * @see The Typesafe Config Library API Documentation */ + @deprecated("Use org.apache.pekko.actor.javadsl.ActorSystem.create instead", "1.3.0") def create( name: String, config: Config, @@ -259,6 +262,7 @@ object ActorSystem { * associated with the ActorSystem class. * Then it loads the default reference configuration using the ClassLoader. */ + @deprecated("Use org.apache.pekko.actor.scaladsl.ActorSystem.apply instead", "1.3.0") def apply(): ActorSystem = apply("default") /** @@ -268,12 +272,14 @@ object ActorSystem { * associated with the ActorSystem class. * Then it loads the default reference configuration using the ClassLoader. */ + @deprecated("Use org.apache.pekko.actor.scaladsl.ActorSystem.apply instead", "1.3.0") def apply(name: String): ActorSystem = apply(name, None, None, None) /** * Scala API: Creates a new actor system with the specified name and settings * The core actor system settings are defined in [[BootstrapSetup]] */ + @deprecated("Use org.apache.pekko.actor.scaladsl.ActorSystem.apply instead", "1.3.0") def apply(name: String, setup: ActorSystemSetup): ActorSystem = { val bootstrapSettings = setup.get[BootstrapSetup] val cl = bootstrapSettings.flatMap(_.classLoader).getOrElse(findClassLoader()) @@ -287,6 +293,7 @@ object ActorSystem { * Scala API: Shortcut for creating an actor system with custom bootstrap settings. * Same behavior as calling `ActorSystem(name, ActorSystemSetup(bootstrapSetup))` */ + @deprecated("Use org.apache.pekko.actor.scaladsl.ActorSystem.apply instead", "1.3.0") def apply(name: String, bootstrapSetup: BootstrapSetup): ActorSystem = create(name, ActorSystemSetup.create(bootstrapSetup)) @@ -298,6 +305,7 @@ object ActorSystem { * * @see The Typesafe Config Library API Documentation */ + @deprecated("Use org.apache.pekko.actor.scaladsl.ActorSystem.apply instead", "1.3.0") def apply(name: String, config: Config): ActorSystem = apply(name, Option(config), None, None) /** @@ -305,6 +313,7 @@ object ActorSystem { * * @see The Typesafe Config Library API Documentation */ + @deprecated("Use org.apache.pekko.actor.scaladsl.ActorSystem.apply instead", "1.3.0") def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = apply(name, Option(config), Option(classLoader), None) @@ -319,6 +328,7 @@ object ActorSystem { * * @see The Typesafe Config Library API Documentation */ + @deprecated("Use org.apache.pekko.actor.scaladsl.ActorSystem.apply instead", "1.3.0") def apply( name: String, config: Option[Config] = None, @@ -526,7 +536,7 @@ object ActorSystem { * extending [[pekko.actor.ExtendedActorSystem]] instead, but beware that you * are completely on your own in that case! */ -abstract class ActorSystem extends ActorRefFactory with ClassicActorSystemProvider { +trait ActorSystem extends ActorRefFactory with ClassicActorSystemProvider with AutoCloseable { import ActorSystem._ /** @@ -677,7 +687,10 @@ abstract class ActorSystem extends ActorRefFactory with ClassicActorSystemProvid * using the dispatcher of this actor system as it will have been shut down before the * future completes. */ - def terminate(): Future[Terminated] + @deprecated( + "Use .close() or the .terminateAsync() function on an ActorSystem created by org.apache.pekko.actor.scaladsl.ActorSystem.apply", + "1.3.0") + def terminate(): Future[Terminated] = terminateImpl() /** * Returns a Future which will be completed after the ActorSystem has been terminated @@ -687,7 +700,7 @@ abstract class ActorSystem extends ActorRefFactory with ClassicActorSystemProvid * such as `onComplete`, on the dispatchers (`ExecutionContext`) of this actor system as they * will have been shut down before this future completes. */ - def whenTerminated: Future[Terminated] + def whenTerminated: Future[Terminated] = whenTerminatedImpl /** * Returns a CompletionStage which will be completed after the ActorSystem has been terminated @@ -697,7 +710,56 @@ abstract class ActorSystem extends ActorRefFactory with ClassicActorSystemProvid * such as `thenRunAsync`, on the dispatchers (`Executor`) of this actor system as they * will have been shut down before this CompletionStage completes. */ - def getWhenTerminated: CompletionStage[Terminated] + @deprecated( + "Use the .getWhenTerminated() function on an ActorSystem created by org.apache.pekko.actor.javadsl.ActorSystem.create", + "1.3.0") + def getWhenTerminated: CompletionStage[Terminated] = whenTerminatedImpl.asJava + + /** + * Terminates this actor system by running [[CoordinatedShutdown]] with reason + * [[CoordinatedShutdown.ActorSystemTerminateReason]]. This method will block + * until either the actor system is terminated or + * `pekko.coordinated-shutdown.close-actor-system-timeout` timeout duration is + * passed, in which case a [[TimeoutException]] is thrown. + * + * If `pekko.coordinated-shutdown.run-by-actor-system-terminate` is configured to `off` + * it will not run `CoordinatedShutdown`, but the `ActorSystem` and its actors + * will still be terminated. + * + * This will stop the guardian actor, which in turn + * will recursively stop all its child actors, and finally the system guardian + * (below which the logging actors reside) and then execute all registered + * termination handlers (see [[ActorSystem#registerOnTermination]]). + * @since 1.3.0 + */ + @throws(classOf[TimeoutException]) + override def close(): Unit = { + terminateImpl() + Await.ready(whenTerminatedImpl, + Duration(settings.config.getDuration("coordinated-shutdown.close-actor-system-timeout").toMillis, + TimeUnit.MILLISECONDS)) + } + + /** + * Asynchronously terminates this actor system by running [[CoordinatedShutdown]] with reason + * [[CoordinatedShutdown.ActorSystemTerminateReason]]. This method will block + * until either the actor system is terminated or + * `pekko.coordinated-shutdown.close-actor-system-timeout` timeout duration is + * passed, in which case a [[TimeoutException]] is thrown. + * + * If `pekko.coordinated-shutdown.run-by-actor-system-terminate` is configured to `off` + * it will not run `CoordinatedShutdown`, but the `ActorSystem` and its actors + * will still be terminated. + * + * This will stop the guardian actor, which in turn + * will recursively stop all its child actors, and finally the system guardian + * (below which the logging actors reside) and then execute all registered + * termination handlers (see [[ActorSystem#registerOnTermination]]). + * @since 1.3.0 + */ + def closeAsync(): Unit = { + terminateImpl() + } /** * Registers the provided extension and creates its payload, if this extension isn't already registered @@ -719,6 +781,10 @@ abstract class ActorSystem extends ActorRefFactory with ClassicActorSystemProvid * of the payload, if is in the process of registration from another Thread of execution */ def hasExtension(ext: ExtensionId[_ <: Extension]): Boolean + + @InternalApi private[pekko] def terminateImpl(): Future[Terminated] + + @InternalApi private[pekko] def whenTerminatedImpl: Future[Terminated] } /** @@ -855,7 +921,7 @@ private[pekko] class ActorSystemImpl( finally System.exit(-1) else try logFatalError("shutting down", cause, thread) - finally terminate() + finally terminateImpl() } } @@ -978,8 +1044,7 @@ private[pekko] class ActorSystemImpl( private[this] final val terminationCallbacks = new TerminationCallbacks(provider.terminationFuture)(dispatcher) - override def whenTerminated: Future[Terminated] = terminationCallbacks.terminationFuture - override def getWhenTerminated: CompletionStage[Terminated] = whenTerminated.asJava + override def whenTerminatedImpl: Future[Terminated] = terminationCallbacks.terminationFuture def lookupRoot: InternalActorRef = provider.rootGuardian def guardian: LocalActorRef = provider.guardian def systemGuardian: LocalActorRef = provider.systemGuardian @@ -1054,7 +1119,7 @@ private[pekko] class ActorSystemImpl( this } catch { case NonFatal(e) => - try terminate() + try terminateImpl() catch { case NonFatal(_) => Try(stopScheduler()) } throw e } @@ -1065,7 +1130,7 @@ private[pekko] class ActorSystemImpl( @volatile private var terminating = false - override def terminate(): Future[Terminated] = { + override def terminateImpl(): Future[Terminated] = { terminating = true if (settings.CoordinatedShutdownRunByActorSystemTerminate && !aborting) { // Note that the combination CoordinatedShutdownRunByActorSystemTerminate==true && @@ -1079,7 +1144,7 @@ private[pekko] class ActorSystemImpl( } else { finalTerminate() } - whenTerminated + whenTerminatedImpl } override private[pekko] def finalTerminate(): Unit = { @@ -1103,7 +1168,7 @@ private[pekko] class ActorSystemImpl( */ def abort(): Unit = { aborting = true - terminate() + terminateImpl() } // #create-scheduler diff --git a/actor/src/main/scala/org/apache/pekko/actor/CoordinatedShutdown.scala b/actor/src/main/scala/org/apache/pekko/actor/CoordinatedShutdown.scala index b2d09e5cf5f..d96df3513ab 100644 --- a/actor/src/main/scala/org/apache/pekko/actor/CoordinatedShutdown.scala +++ b/actor/src/main/scala/org/apache/pekko/actor/CoordinatedShutdown.scala @@ -254,7 +254,7 @@ object CoordinatedShutdown extends ExtensionId[CoordinatedShutdown] with Extensi val timeout = coord.timeout(PhaseActorSystemTerminate) val t = new Thread { override def run(): Unit = { - if (Try(Await.ready(system.whenTerminated, timeout)).isFailure && !runningJvmHook) + if (Try(Await.ready(system.whenTerminatedImpl, timeout)).isFailure && !runningJvmHook) System.exit(exitCode) } } @@ -264,7 +264,7 @@ object CoordinatedShutdown extends ExtensionId[CoordinatedShutdown] with Extensi if (terminateActorSystem) { system.finalTerminate() - system.whenTerminated.map { _ => + system.whenTerminatedImpl.map { _ => if (exitJvm && !runningJvmHook) System.exit(exitCode) Done }(ExecutionContexts.parasitic) @@ -281,7 +281,7 @@ object CoordinatedShutdown extends ExtensionId[CoordinatedShutdown] with Extensi if (runByJvmShutdownHook) { coord.actorSystemJvmHook = OptionVal.Some(coord.addCancellableJvmShutdownHook { runningJvmHook = true // avoid System.exit from PhaseActorSystemTerminate task - if (!system.whenTerminated.isCompleted) { + if (!system.whenTerminatedImpl.isCompleted) { coord.log.debug("Starting coordinated shutdown from JVM shutdown hook") try { // totalTimeout will be 0 when no tasks registered, so at least 3.seconds diff --git a/actor/src/main/scala/org/apache/pekko/actor/javadsl/ActorSystem.scala b/actor/src/main/scala/org/apache/pekko/actor/javadsl/ActorSystem.scala new file mode 100644 index 00000000000..63108d74f7a --- /dev/null +++ b/actor/src/main/scala/org/apache/pekko/actor/javadsl/ActorSystem.scala @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * license agreements; and to You under the Apache License, version 2.0: + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * This file is part of the Apache Pekko project, which was derived from Akka. + */ + +/* + * Copyright (C) 2009-2022 Lightbend Inc. + */ + +package org.apache.pekko.actor.javadsl + +import java.util.concurrent.CompletionStage +import java.util.Optional + +import scala.concurrent.ExecutionContext + +import com.typesafe.config.{ Config, ConfigFactory } + +import org.apache.pekko +import pekko.actor.ActorSystem.findClassLoader +import pekko.actor._ +import pekko.actor.setup.ActorSystemSetup +import pekko.util.FutureConverters._ +import pekko.util.OptionConverters._ + +trait ActorSystem extends org.apache.pekko.actor.ActorSystem { + + /** + * Asynchronously terminates this actor system by running [[CoordinatedShutdown]] with reason + * [[CoordinatedShutdown.ActorSystemTerminateReason]]. + * + * If `pekko.coordinated-shutdown.run-by-actor-system-terminate` is configured to `off` + * it will not run `CoordinatedShutdown`, but the `ActorSystem` and its actors + * will still be terminated. + * + * This will stop the guardian actor, which in turn + * will recursively stop all its child actors, and finally the system guardian + * (below which the logging actors reside) and then execute all registered + * termination handlers (see [[ActorSystem.registerOnTermination]]). + * Be careful to not schedule any operations on completion of the returned future + * using the dispatcher of this actor system as it will have been shut down before the + * future completes. + */ + def terminateAsync(): CompletionStage[Terminated] = terminateImpl().asJava + + /** + * Returns a [[CompletionStage]] which will be completed after the [[ActorSystem]] has been terminated + * and termination hooks have been executed. If you registered any callback with + * [[ActorSystem.registerOnTermination]], the returned Future from this method will not complete + * until all the registered callbacks are finished. Be careful to not schedule any operations, + * such as `onComplete`, on the dispatchers (`ExecutionContext`) of this actor system as they + * will have been shut down before this future completes. + */ + override def getWhenTerminated: CompletionStage[Terminated] = whenTerminatedImpl.asJava +} + +object ActorSystem { + + /** + * Creates a new ActorSystem with the name "default", + * obtains the current ClassLoader by first inspecting the current threads' getContextClassLoader, + * then tries to walk the stack to find the callers class loader, then falls back to the ClassLoader + * associated with the ActorSystem class. + * Then it loads the default reference configuration using the ClassLoader. + */ + def create(): ActorSystem = create("default") + + def create(name: String): ActorSystem = create(name, Optional.empty(), Optional.empty(), Optional.empty()) + + def create(name: String, setup: ActorSystemSetup): ActorSystem = { + val bootstrapSettings = setup.get[BootstrapSetup] + val cl = bootstrapSettings.flatMap(_.classLoader).getOrElse(findClassLoader()) + val appConfig = bootstrapSettings.flatMap(_.config).getOrElse(ConfigFactory.load(cl)) + val defaultEC = bootstrapSettings.flatMap(_.defaultExecutionContext) + + val impl = new ActorSystemImpl(name, appConfig, cl, defaultEC, None, setup) with ActorSystem { + // TODO: Remove in Pekko 2.0.0, not needed anymore + override def getWhenTerminated: CompletionStage[Terminated] = super[ActorSystem].getWhenTerminated + } + + impl.start() + } + + def create(name: String, bootstrapSetup: BootstrapSetup): ActorSystem = + create(name, ActorSystemSetup.create(bootstrapSetup)) + + /** + * Creates a new ActorSystem with the specified name, and the specified Config, then + * obtains the current ClassLoader by first inspecting the current threads' getContextClassLoader, + * then tries to walk the stack to find the callers class loader, then falls back to the ClassLoader + * associated with the ActorSystem class. + * + * @see The Typesafe Config Library API Documentation + */ + def create(name: String, config: Config): ActorSystem = + create(name, Optional.of(config), Optional.empty(), Optional.empty()) + + /** + * Creates a new ActorSystem with the specified name, the specified Config, and specified ClassLoader + * + * @see The Typesafe Config Library API Documentation + */ + def create(name: String, config: Config, classLoader: ClassLoader): ActorSystem = + create(name, Optional.of(config), Optional.of(classLoader), Optional.empty()) + + /** + * Creates a new ActorSystem with the specified name, + * the specified ClassLoader if given, otherwise obtains the current ClassLoader by first inspecting the current + * threads' getContextClassLoader, then tries to walk the stack to find the callers class loader, then + * falls back to the ClassLoader associated with the ActorSystem class. + * If an ExecutionContext is given, it will be used as the default executor inside this ActorSystem. + * If no ExecutionContext is given, the system will fallback to the executor configured under "pekko.actor.default-dispatcher.default-executor.fallback". + * The system will use the passed in config, or falls back to the default reference configuration using the ClassLoader. + * + * @see The Typesafe Config Library API Documentation + */ + def create( + name: String, + config: Optional[Config], + classLoader: Optional[ClassLoader], + defaultExecutionContext: Optional[ExecutionContext]): ActorSystem = + create(name, ActorSystemSetup(BootstrapSetup(classLoader.toScala, config.toScala, defaultExecutionContext.toScala))) +} diff --git a/actor/src/main/scala/org/apache/pekko/actor/scaladsl/ActorSystem.scala b/actor/src/main/scala/org/apache/pekko/actor/scaladsl/ActorSystem.scala new file mode 100644 index 00000000000..a3b2bba326e --- /dev/null +++ b/actor/src/main/scala/org/apache/pekko/actor/scaladsl/ActorSystem.scala @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * license agreements; and to You under the Apache License, version 2.0: + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * This file is part of the Apache Pekko project, which was derived from Akka. + */ + +/* + * Copyright (C) 2009-2022 Lightbend Inc. + */ + +package org.apache.pekko.actor.scaladsl + +import scala.concurrent.{ ExecutionContext, Future } + +import com.typesafe.config.{ Config, ConfigFactory } + +import org.apache.pekko +import pekko.actor.ActorSystem.findClassLoader +import pekko.actor._ +import pekko.actor.setup.ActorSystemSetup + +trait ActorSystem extends org.apache.pekko.actor.ActorSystem { + + /** + * Asynchronously terminates this actor system by running [[CoordinatedShutdown]] with reason + * [[CoordinatedShutdown.ActorSystemTerminateReason]]. + * + * If `pekko.coordinated-shutdown.run-by-actor-system-terminate` is configured to `off` + * it will not run `CoordinatedShutdown`, but the `ActorSystem` and its actors + * will still be terminated. + * + * This will stop the guardian actor, which in turn + * will recursively stop all its child actors, and finally the system guardian + * (below which the logging actors reside) and then execute all registered + * termination handlers (see [[ActorSystem.registerOnTermination]]). + * Be careful to not schedule any operations on completion of the returned future + * using the dispatcher of this actor system as it will have been shut down before the + * future completes. + */ + def terminateAsync(): Future[Terminated] = terminateImpl() + + /** + * Returns a [[Future]] which will be completed after the [[ActorSystem]] has been terminated + * and termination hooks have been executed. If you registered any callback with + * [[ActorSystem.registerOnTermination]], the returned Future from this method will not complete + * until all the registered callbacks are finished. Be careful to not schedule any operations, + * such as `onComplete`, on the dispatchers (`ExecutionContext`) of this actor system as they + * will have been shut down before this future completes. + */ + override def whenTerminated: Future[Terminated] = whenTerminatedImpl +} + +object ActorSystem { + + /** + * Creates a new ActorSystem with the name "default", + * obtains the current ClassLoader by first inspecting the current threads' getContextClassLoader, + * then tries to walk the stack to find the callers class loader, then falls back to the ClassLoader + * associated with the ActorSystem class. + * Then it loads the default reference configuration using the ClassLoader. + */ + def apply(): ActorSystem = apply("default") + + /** + * Creates a new ActorSystem with the specified name, + * obtains the current ClassLoader by first inspecting the current threads' getContextClassLoader, + * then tries to walk the stack to find the callers class loader, then falls back to the ClassLoader + * associated with the ActorSystem class. + * Then it loads the default reference configuration using the ClassLoader. + */ + def apply(name: String): ActorSystem = apply(name, None, None, None) + + def apply(name: String, setup: ActorSystemSetup): ActorSystem = { + val bootstrapSettings = setup.get[BootstrapSetup] + val cl = bootstrapSettings.flatMap(_.classLoader).getOrElse(findClassLoader()) + val appConfig = bootstrapSettings.flatMap(_.config).getOrElse(ConfigFactory.load(cl)) + val defaultEC = bootstrapSettings.flatMap(_.defaultExecutionContext) + + val impl = new ActorSystemImpl(name, appConfig, cl, defaultEC, None, setup) with ActorSystem { + // TODO: Remove in Pekko 2.0.0, not needed anymore + override def whenTerminated: Future[Terminated] = super[ActorSystem].whenTerminated + } + + impl.start() + } + + def apply(name: String, bootstrapSetup: BootstrapSetup): ActorSystem = + apply(name, ActorSystemSetup.create(bootstrapSetup)) + + /** + * Creates a new ActorSystem with the specified name, and the specified Config, then + * obtains the current ClassLoader by first inspecting the current threads' getContextClassLoader, + * then tries to walk the stack to find the callers class loader, then falls back to the ClassLoader + * associated with the ActorSystem class. + * + * @see The Typesafe Config Library API Documentation + */ + def apply(name: String, config: Config): ActorSystem = apply(name, Option(config), None, None) + + /** + * Creates a new ActorSystem with the specified name, the specified Config, and specified ClassLoader + * + * @see The Typesafe Config Library API Documentation + */ + def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = + apply(name, Option(config), Option(classLoader), None) + + /** + * Creates a new ActorSystem with the specified name, + * the specified ClassLoader if given, otherwise obtains the current ClassLoader by first inspecting the current + * threads' getContextClassLoader, then tries to walk the stack to find the callers class loader, then + * falls back to the ClassLoader associated with the ActorSystem class. + * If an ExecutionContext is given, it will be used as the default executor inside this ActorSystem. + * If no ExecutionContext is given, the system will fallback to the executor configured under "pekko.actor.default-dispatcher.default-executor.fallback". + * The system will use the passed in config, or falls back to the default reference configuration using the ClassLoader. + * + * @see The Typesafe Config Library API Documentation + */ + def apply( + name: String, + config: Option[Config] = None, + classLoader: Option[ClassLoader] = None, + defaultExecutionContext: Option[ExecutionContext] = None): ActorSystem = + apply(name, ActorSystemSetup(BootstrapSetup(classLoader, config, defaultExecutionContext))) +} diff --git a/bench-jmh/src/main/scala/org/apache/pekko/actor/ActorBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/actor/ActorBenchmark.scala index d6348d83b22..0a55c523aaf 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/actor/ActorBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/actor/ActorBenchmark.scala @@ -15,9 +15,6 @@ package org.apache.pekko.actor import java.util.concurrent.TimeUnit -import scala.concurrent.Await -import scala.concurrent.duration._ - import BenchmarkActors._ import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations._ @@ -64,7 +61,7 @@ class ActorBenchmark { requireRightNumberOfCores(threads) - system = ActorSystem( + system = org.apache.pekko.actor.scaladsl.ActorSystem( "ActorBenchmark", ConfigFactory.parseString(s""" pekko.actor { @@ -100,8 +97,7 @@ class ActorBenchmark { @TearDown(Level.Trial) def shutdown(): Unit = { - system.terminate() - Await.ready(system.whenTerminated, 15.seconds) + system.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/actor/ActorCreationBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/actor/ActorCreationBenchmark.scala index 05317241e97..5f10cb7e3a7 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/actor/ActorCreationBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/actor/ActorCreationBenchmark.scala @@ -15,9 +15,6 @@ package org.apache.pekko.actor import java.util.concurrent.TimeUnit -import scala.concurrent.Await -import scala.concurrent.duration._ - import org.openjdk.jmh.annotations._ /* @@ -35,7 +32,7 @@ hand checking: @Warmup(iterations = 1000) @Measurement(iterations = 4000) class ActorCreationBenchmark { - implicit val system: ActorSystem = ActorSystem() + implicit val system: ActorSystem = org.apache.pekko.actor.scaladsl.ActorSystem() final val props = Props[MyActor]() @@ -47,8 +44,7 @@ class ActorCreationBenchmark { @TearDown(Level.Trial) def shutdown(): Unit = { - system.terminate() - Await.ready(system.whenTerminated, 15.seconds) + system.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/actor/AffinityPoolComparativeBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/actor/AffinityPoolComparativeBenchmark.scala index b69f7ac504d..3361c90d9e2 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/actor/AffinityPoolComparativeBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/actor/AffinityPoolComparativeBenchmark.scala @@ -56,7 +56,7 @@ class AffinityPoolComparativeBenchmark { s"""default-mailbox.mailbox-type = "${classOf[pekko.dispatch.SingleConsumerOnlyUnboundedMailbox].getName}"""" } - system = ActorSystem( + system = pekko.actor.scaladsl.ActorSystem( "AffinityPoolComparativeBenchmark", ConfigFactory.parseString(s"""| pekko { | log-dead-letters = off diff --git a/bench-jmh/src/main/scala/org/apache/pekko/actor/AffinityPoolIdleCPULevelBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/actor/AffinityPoolIdleCPULevelBenchmark.scala index c0b2d94d50f..e61c7bac43c 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/actor/AffinityPoolIdleCPULevelBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/actor/AffinityPoolIdleCPULevelBenchmark.scala @@ -18,7 +18,8 @@ import java.util.concurrent.TimeUnit import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations._ -import org.apache.pekko.actor.BenchmarkActors._ +import org.apache.pekko +import pekko.actor.BenchmarkActors._ @State(Scope.Benchmark) @BenchmarkMode(Array(Mode.Throughput)) @@ -45,7 +46,7 @@ class AffinityPoolIdleCPULevelBenchmark { requireRightNumberOfCores(numThreads) - system = ActorSystem( + system = pekko.actor.scaladsl.ActorSystem( "AffinityPoolWaitingStrategyBenchmark", ConfigFactory.parseString(s""" | pekko { | log-dead-letters = off diff --git a/bench-jmh/src/main/scala/org/apache/pekko/actor/AffinityPoolRequestResponseBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/actor/AffinityPoolRequestResponseBenchmark.scala index d5c51a3e6c6..6dd56c02428 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/actor/AffinityPoolRequestResponseBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/actor/AffinityPoolRequestResponseBenchmark.scala @@ -60,7 +60,7 @@ class AffinityPoolRequestResponseBenchmark { s"""default-mailbox.mailbox-type = "${classOf[pekko.dispatch.SingleConsumerOnlyUnboundedMailbox].getName}"""" } - system = ActorSystem( + system = pekko.actor.scaladsl.ActorSystem( "AffinityPoolComparativeBenchmark", ConfigFactory.parseString(s"""| pekko { | log-dead-letters = off diff --git a/bench-jmh/src/main/scala/org/apache/pekko/actor/BenchmarkActors.scala b/bench-jmh/src/main/scala/org/apache/pekko/actor/BenchmarkActors.scala index 340319ef4ba..e405adf745a 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/actor/BenchmarkActors.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/actor/BenchmarkActors.scala @@ -183,8 +183,8 @@ object BenchmarkActors { } def tearDownSystem()(implicit system: ActorSystem): Unit = { - system.terminate() - Await.ready(system.whenTerminated, timeout) + system.terminateImpl() + Await.ready(system.whenTerminatedImpl, timeout) } } diff --git a/bench-jmh/src/main/scala/org/apache/pekko/actor/ForkJoinActorBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/actor/ForkJoinActorBenchmark.scala index 16ec75c1fe0..7a622411d45 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/actor/ForkJoinActorBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/actor/ForkJoinActorBenchmark.scala @@ -16,14 +16,13 @@ package org.apache.pekko.actor import java.util.concurrent.TimeUnit import scala.annotation.tailrec -import scala.concurrent.Await -import scala.concurrent.duration._ import BenchmarkActors._ import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations._ -import org.apache.pekko.testkit.TestProbe +import org.apache.pekko +import pekko.testkit.TestProbe @State(Scope.Benchmark) @BenchmarkMode(Array(Mode.Throughput)) @@ -54,7 +53,7 @@ class ForkJoinActorBenchmark { requireRightNumberOfCores(cores) - system = ActorSystem( + system = pekko.actor.scaladsl.ActorSystem( "ForkJoinActorBenchmark", ConfigFactory.parseString(s""" pekko { @@ -78,8 +77,7 @@ class ForkJoinActorBenchmark { @TearDown(Level.Trial) def shutdown(): Unit = { - system.terminate() - Await.ready(system.whenTerminated, 15.seconds) + system.close() } // @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/actor/RouterPoolCreationBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/actor/RouterPoolCreationBenchmark.scala index 097f333b1c3..714064d53f3 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/actor/RouterPoolCreationBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/actor/RouterPoolCreationBenchmark.scala @@ -15,7 +15,6 @@ package org.apache.pekko.actor import java.util.concurrent.TimeUnit -import scala.concurrent.Await import scala.concurrent.duration._ import org.openjdk.jmh.annotations._ @@ -31,7 +30,7 @@ import pekko.testkit.TestProbe @Warmup(iterations = 20) @Measurement(iterations = 100) class RouterPoolCreationBenchmark { - implicit val system: ActorSystem = ActorSystem() + implicit val system: ActorSystem = pekko.actor.scaladsl.ActorSystem() val probe = TestProbe() Props[TestActors.EchoActor]() @@ -41,8 +40,7 @@ class RouterPoolCreationBenchmark { @TearDown(Level.Trial) def shutdown(): Unit = { - system.terminate() - Await.ready(system.whenTerminated, 15.seconds) + system.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/actor/ScheduleBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/actor/ScheduleBenchmark.scala index 3cdba681937..69c9b53736c 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/actor/ScheduleBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/actor/ScheduleBenchmark.scala @@ -22,7 +22,8 @@ import scala.concurrent.duration._ import org.openjdk.jmh.annotations._ -import org.apache.pekko.util.Timeout +import org.apache.pekko +import pekko.util.Timeout @State(Scope.Benchmark) @BenchmarkMode(Array(Mode.Throughput)) @@ -30,7 +31,7 @@ import org.apache.pekko.util.Timeout @Warmup(iterations = 10, time = 1700, timeUnit = TimeUnit.MILLISECONDS) @Measurement(iterations = 20, time = 1700, timeUnit = TimeUnit.MILLISECONDS) class ScheduleBenchmark { - implicit val system: ActorSystem = ActorSystem() + implicit val system: ActorSystem = pekko.actor.scaladsl.ActorSystem() val scheduler: Scheduler = system.scheduler val interval: FiniteDuration = 25.millis val within: FiniteDuration = 2.seconds @@ -53,8 +54,7 @@ class ScheduleBenchmark { @TearDown def shutdown(): Unit = { - system.terminate() - Await.ready(system.whenTerminated, 15.seconds) + system.close() } def op(idx: Int) = if (idx == winner) promise.trySuccess(idx) else idx diff --git a/bench-jmh/src/main/scala/org/apache/pekko/actor/StashCreationBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/actor/StashCreationBenchmark.scala index 683c6b90ede..9aeb9a41a3f 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/actor/StashCreationBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/actor/StashCreationBenchmark.scala @@ -15,13 +15,11 @@ package org.apache.pekko.actor import java.util.concurrent.TimeUnit -import scala.concurrent.Await -import scala.concurrent.duration._ - import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations._ -import org.apache.pekko.testkit.TestProbe +import org.apache.pekko +import pekko.testkit.TestProbe object StashCreationBenchmark { class StashingActor extends Actor with Stash { @@ -44,13 +42,12 @@ class StashCreationBenchmark { stash-capacity = 1000 } """) - implicit val system: ActorSystem = ActorSystem("StashCreationBenchmark", conf) + implicit val system: ActorSystem = pekko.actor.scaladsl.ActorSystem("StashCreationBenchmark", conf) val probe = TestProbe() @TearDown(Level.Trial) def shutdown(): Unit = { - system.terminate() - Await.ready(system.whenTerminated, 15.seconds) + system.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/actor/TellOnlyBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/actor/TellOnlyBenchmark.scala index 80052e1e8f3..7752ea37006 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/actor/TellOnlyBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/actor/TellOnlyBenchmark.scala @@ -15,7 +15,6 @@ package org.apache.pekko.actor import java.util.concurrent.TimeUnit -import scala.concurrent.Await import scala.concurrent.duration._ import com.typesafe.config.{ Config, ConfigFactory } @@ -39,7 +38,7 @@ class TellOnlyBenchmark { @Setup(Level.Trial) def setup(): Unit = { - system = ActorSystem( + system = pekko.actor.scaladsl.ActorSystem( "TellOnlyBenchmark", ConfigFactory.parseString(s"""| pekko { | log-dead-letters = off @@ -65,8 +64,7 @@ class TellOnlyBenchmark { @TearDown(Level.Trial) def shutdown(): Unit = { - system.terminate() - Await.ready(system.whenTerminated, 15.seconds) + system.close() } var actor: ActorRef = _ diff --git a/bench-jmh/src/main/scala/org/apache/pekko/cluster/ddata/ORSetSerializationBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/cluster/ddata/ORSetSerializationBenchmark.scala index 9d8c898f6a2..01fc4b74ff1 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/cluster/ddata/ORSetSerializationBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/cluster/ddata/ORSetSerializationBenchmark.scala @@ -15,9 +15,6 @@ package org.apache.pekko.cluster.ddata import java.util.concurrent.TimeUnit -import scala.concurrent.Await -import scala.concurrent.duration._ - import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations.{ Scope => JmhScope } import org.openjdk.jmh.annotations.Benchmark @@ -32,8 +29,8 @@ import org.openjdk.jmh.annotations.Warmup import org.apache.pekko import pekko.actor.ActorRef -import pekko.actor.ActorSystem import pekko.actor.Props +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.Cluster import pekko.serialization.SerializationExtension import pekko.serialization.Serializers @@ -72,8 +69,8 @@ class ORSetSerializationBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system1.terminate(), 5.seconds) - Await.result(system2.terminate(), 5.seconds) + system1.close() + system2.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/dispatch/NodeQueueBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/dispatch/NodeQueueBenchmark.scala index b08ddbe34b5..473024d3188 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/dispatch/NodeQueueBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/dispatch/NodeQueueBenchmark.scala @@ -15,14 +15,12 @@ package org.apache.pekko.dispatch import java.util.concurrent.TimeUnit -import scala.concurrent.Await -import scala.concurrent.duration._ - import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations._ import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.testkit.TestProbe object NodeQueueBenchmark { @@ -61,7 +59,7 @@ mailbox { }).withDispatcher("dispatcher").withMailbox("mailbox"), "receiver") @TearDown - def teardown(): Unit = Await.result(sys.terminate(), 5.seconds) + def teardown(): Unit = sys.close() @TearDown(Level.Invocation) def waitInBetween(): Unit = { diff --git a/bench-jmh/src/main/scala/org/apache/pekko/persistence/LevelDbBatchingBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/persistence/LevelDbBatchingBenchmark.scala index 551d9ea6607..78e4cd5a283 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/persistence/LevelDbBatchingBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/persistence/LevelDbBatchingBenchmark.scala @@ -15,8 +15,6 @@ package org.apache.pekko.persistence import java.io.File import java.util.concurrent.TimeUnit -import scala.concurrent.Await -import scala.concurrent.duration._ import org.apache.commons.io.FileUtils import org.openjdk.jmh.annotations._ import org.apache.pekko @@ -73,8 +71,7 @@ class LevelDbBatchingBenchmark { store ! PoisonPill Thread.sleep(500) - sys.terminate() - Await.ready(sys.whenTerminated, 10.seconds) + sys.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/persistence/PersistenceActorDeferBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/persistence/PersistenceActorDeferBenchmark.scala index 31ee2d4fc53..8dff453c1bf 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/persistence/PersistenceActorDeferBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/persistence/PersistenceActorDeferBenchmark.scala @@ -15,15 +15,13 @@ package org.apache.pekko.persistence import java.io.File -import scala.concurrent.Await -import scala.concurrent.duration._ - import org.apache.commons.io.FileUtils import org.openjdk.jmh.annotations._ import org.openjdk.jmh.annotations.Scope import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.testkit.TestProbe /* @@ -71,8 +69,7 @@ class PersistentActorDeferBenchmark { @TearDown def shutdown(): Unit = { - system.terminate() - Await.ready(system.whenTerminated, 15.seconds) + system.close() storageLocations.foreach(FileUtils.deleteDirectory) } diff --git a/bench-jmh/src/main/scala/org/apache/pekko/persistence/PersistentActorBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/persistence/PersistentActorBenchmark.scala index b69696fc050..2b672dd7ff3 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/persistence/PersistentActorBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/persistence/PersistentActorBenchmark.scala @@ -15,9 +15,6 @@ package org.apache.pekko.persistence import java.io.File -import scala.concurrent.Await -import scala.concurrent.duration._ - import org.apache.commons.io.FileUtils import org.openjdk.jmh.annotations._ import org.openjdk.jmh.annotations.Scope @@ -51,7 +48,7 @@ class PersistentActorThroughputBenchmark { @Setup def setup(): Unit = { - system = ActorSystem("test", config) + system = pekko.actor.scaladsl.ActorSystem("test", config) probe = TestProbe()(system) @@ -69,8 +66,7 @@ class PersistentActorThroughputBenchmark { @TearDown def shutdown(): Unit = { - system.terminate() - Await.ready(system.whenTerminated, 15.seconds) + system.close() storageLocations.foreach(FileUtils.deleteDirectory) } diff --git a/bench-jmh/src/main/scala/org/apache/pekko/persistence/PersistentActorWithAtLeastOnceDeliveryBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/persistence/PersistentActorWithAtLeastOnceDeliveryBenchmark.scala index 4a8860c5386..069c6645a59 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/persistence/PersistentActorWithAtLeastOnceDeliveryBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/persistence/PersistentActorWithAtLeastOnceDeliveryBenchmark.scala @@ -15,7 +15,6 @@ package org.apache.pekko.persistence import java.io.File -import scala.concurrent.Await import scala.concurrent.duration._ import org.apache.commons.io.FileUtils @@ -24,6 +23,7 @@ import org.openjdk.jmh.annotations.Scope import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.testkit.TestProbe @State(Scope.Benchmark) @@ -72,8 +72,7 @@ class PersistentActorWithAtLeastOnceDeliveryBenchmark { @TearDown def shutdown(): Unit = { - system.terminate() - Await.ready(system.whenTerminated, 15.seconds) + system.close() storageLocations.foreach(FileUtils.deleteDirectory) } diff --git a/bench-jmh/src/main/scala/org/apache/pekko/remote/artery/CodecBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/remote/artery/CodecBenchmark.scala index a1cb3a85619..ae8a58c575e 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/remote/artery/CodecBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/remote/artery/CodecBenchmark.scala @@ -32,6 +32,7 @@ import org.apache.pekko import pekko.Done import pekko.NotUsed import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.remote._ import pekko.remote.artery.Decoder.InboundCompressionAccess import pekko.remote.artery.compress._ @@ -197,8 +198,8 @@ class CodecBenchmark { @TearDown(Level.Trial) def tearDownTrial(): Unit = { - Await.result(system.terminate(), 5.seconds) - Await.result(systemB.terminate(), 5.seconds) + system.close() + systemB.close() } @Setup(Level.Iteration) diff --git a/bench-jmh/src/main/scala/org/apache/pekko/remote/artery/SendQueueBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/remote/artery/SendQueueBenchmark.scala index 9bf95b8350a..a4ae06a23f4 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/remote/artery/SendQueueBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/remote/artery/SendQueueBenchmark.scala @@ -17,15 +17,12 @@ import java.util.concurrent.CountDownLatch import java.util.concurrent.CyclicBarrier import java.util.concurrent.TimeUnit -import scala.concurrent.Await -import scala.concurrent.duration._ - import com.typesafe.config.ConfigFactory import org.agrona.concurrent.ManyToOneConcurrentArrayQueue import org.openjdk.jmh.annotations._ import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.stream.KillSwitches import pekko.stream.OverflowStrategy import pekko.stream.SystemMaterializer @@ -52,7 +49,7 @@ class SendQueueBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), 5.seconds) + system.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/serialization/jackson/JacksonSerializationBench.scala b/bench-jmh/src/main/scala/org/apache/pekko/serialization/jackson/JacksonSerializationBench.scala index 939cbad12c3..2812a030afe 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/serialization/jackson/JacksonSerializationBench.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/serialization/jackson/JacksonSerializationBench.scala @@ -19,7 +19,6 @@ import java.time.LocalDateTime import java.util import java.util.concurrent.TimeUnit -import scala.concurrent.Await import scala.concurrent.duration._ import scala.annotation.nowarn @@ -27,7 +26,7 @@ import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations._ import org.apache.pekko -import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.serialization.Serialization import pekko.serialization.SerializationExtension import pekko.serialization.SerializerWithStringManifest @@ -233,7 +232,7 @@ class JacksonSerializationBench { @TearDown(Level.Trial) def tearDownTrial(): Unit = { - Await.result(system.terminate(), 5.seconds) + system.close() } private var size = 0L diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/AskBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/AskBenchmark.scala index 0b007953238..5c341864f4f 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/AskBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/AskBenchmark.scala @@ -16,7 +16,6 @@ package org.apache.pekko.stream import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit -import scala.concurrent.Await import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory @@ -26,8 +25,8 @@ import org.apache.pekko import pekko.NotUsed import pekko.actor.Actor import pekko.actor.ActorRef -import pekko.actor.ActorSystem import pekko.actor.Props +import pekko.actor.scaladsl.ActorSystem import pekko.remote.artery.BenchTestSource import pekko.remote.artery.LatchSink import pekko.stream.scaladsl._ @@ -82,7 +81,7 @@ class AskBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), 5.seconds) + system.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/EmptySourceBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/EmptySourceBenchmark.scala index 4de17fbf734..19259047d22 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/EmptySourceBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/EmptySourceBenchmark.scala @@ -21,7 +21,7 @@ import scala.concurrent.duration._ import org.openjdk.jmh.annotations._ import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.stream.scaladsl._ @State(Scope.Benchmark) @@ -32,7 +32,7 @@ class EmptySourceBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), 5.seconds) + system.close() } val setup = Source.empty[String].toMat(Sink.ignore)(Keep.right) diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/FlatMapConcatBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/FlatMapConcatBenchmark.scala index f8c0d49d5d6..14ba6dfd2f3 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/FlatMapConcatBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/FlatMapConcatBenchmark.scala @@ -16,15 +16,14 @@ package org.apache.pekko.stream import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit -import scala.concurrent.{ Await, Future } -import scala.concurrent.duration._ +import scala.concurrent.Future import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations._ import org.apache.pekko import pekko.NotUsed -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.remote.artery.BenchTestSource import pekko.remote.artery.LatchSink import pekko.stream.impl.fusing.GraphStages @@ -63,7 +62,7 @@ class FlatMapConcatBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), 5.seconds) + system.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/FlatMapMergeBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/FlatMapMergeBenchmark.scala index 074bad5a02b..bf6bdfd6e32 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/FlatMapMergeBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/FlatMapMergeBenchmark.scala @@ -22,7 +22,7 @@ import org.openjdk.jmh.annotations._ import org.apache.pekko import pekko.{ Done, NotUsed } -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.remote.artery.BenchTestSourceSameElement import pekko.stream.scaladsl._ @@ -59,7 +59,7 @@ class FlatMapMergeBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), 5.seconds) + system.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/FlowMapBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/FlowMapBenchmark.scala index 5ab9a14c9f8..75d5e3b3988 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/FlowMapBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/FlowMapBenchmark.scala @@ -16,8 +16,6 @@ package org.apache.pekko.stream import java.util.concurrent.Semaphore import java.util.concurrent.TimeUnit -import scala.concurrent.Await -import scala.concurrent.duration._ import scala.util.Success import com.typesafe.config.ConfigFactory @@ -25,7 +23,7 @@ import org.openjdk.jmh.annotations._ import org.apache.pekko import pekko.NotUsed -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.remote.artery.BenchTestSource import pekko.stream.impl.fusing.GraphStages import pekko.stream.scaladsl._ @@ -92,7 +90,7 @@ class FlowMapBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), 5.seconds) + system.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/FramingBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/FramingBenchmark.scala index 269b120dbec..e5473c301ab 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/FramingBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/FramingBenchmark.scala @@ -16,8 +16,6 @@ package org.apache.pekko.stream import java.util.concurrent.Semaphore import java.util.concurrent.TimeUnit -import scala.concurrent.Await -import scala.concurrent.duration._ import scala.util.Random import com.typesafe.config.Config @@ -26,7 +24,7 @@ import org.openjdk.jmh.annotations._ import org.apache.pekko import pekko.NotUsed -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.remote.artery.BenchTestSourceSameElement import pekko.stream.scaladsl.Framing import pekko.stream.scaladsl.Sink @@ -87,7 +85,7 @@ class FramingBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), 5.seconds) + system.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/FusedGraphsBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/FusedGraphsBenchmark.scala index 1b0dd9d0bd7..ace7c7d815a 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/FusedGraphsBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/FusedGraphsBenchmark.scala @@ -16,16 +16,13 @@ package org.apache.pekko.stream import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit -import scala.concurrent.Await -import scala.concurrent.duration._ - import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations._ import org.openjdk.jmh.annotations.OperationsPerInvocation import org.apache.pekko import pekko.NotUsed -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.stream.scaladsl._ import pekko.stream.stage._ @@ -303,7 +300,7 @@ class FusedGraphsBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), 5.seconds) + system.close() } } diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/InvokeWithFeedbackBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/InvokeWithFeedbackBenchmark.scala index 17c30e5f948..4706c3b0957 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/InvokeWithFeedbackBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/InvokeWithFeedbackBenchmark.scala @@ -21,7 +21,7 @@ import scala.concurrent.duration._ import org.openjdk.jmh.annotations._ import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.stream.scaladsl._ @State(Scope.Benchmark) @@ -63,7 +63,7 @@ class InvokeWithFeedbackBenchmark { def tearDown(): Unit = { sourceQueue.complete() // no way to observe sink completion from the outside - Await.result(system.terminate(), 5.seconds) + system.close() } } diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/LazyFutureSourceBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/LazyFutureSourceBenchmark.scala index 390dfa3bca5..5cbd10f7975 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/LazyFutureSourceBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/LazyFutureSourceBenchmark.scala @@ -27,7 +27,7 @@ import org.openjdk.jmh.annotations._ import org.apache.pekko import pekko.NotUsed -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.stream.scaladsl._ object LazyFutureSourceBenchmark { @@ -54,7 +54,7 @@ class LazyFutureSourceBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), 5.seconds) + system.close() } private val newLazyFutureSource = Source.lazyFuture(() => Future.successful("")).toMat(Sink.ignore)(Keep.right) diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/MapAsyncBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/MapAsyncBenchmark.scala index 88f59d606d6..66e285fec7f 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/MapAsyncBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/MapAsyncBenchmark.scala @@ -16,16 +16,14 @@ package org.apache.pekko.stream import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit -import scala.concurrent.Await import scala.concurrent.Future -import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations._ import org.apache.pekko import pekko.NotUsed -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.remote.artery.BenchTestSource import pekko.remote.artery.LatchSink import pekko.stream.scaladsl._ @@ -70,7 +68,7 @@ class MapAsyncBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), 5.seconds) + system.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/MaterializationBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/MaterializationBenchmark.scala index c1fd4307104..70618aea95d 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/MaterializationBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/MaterializationBenchmark.scala @@ -24,7 +24,7 @@ import org.openjdk.jmh.annotations._ import org.apache.pekko import pekko.Done import pekko.NotUsed -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.stream.scaladsl._ object MaterializationBenchmark { @@ -129,7 +129,7 @@ class MaterializationBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), 5.seconds) + system.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/PartitionHubBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/PartitionHubBenchmark.scala index 40d25b51cc4..ece72a9db3b 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/PartitionHubBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/PartitionHubBenchmark.scala @@ -16,15 +16,12 @@ package org.apache.pekko.stream import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit -import scala.concurrent.Await -import scala.concurrent.duration._ - import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations._ import org.apache.pekko import pekko.NotUsed -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.remote.artery.BenchTestSource import pekko.remote.artery.FixedSizePartitionHub import pekko.remote.artery.LatchSink @@ -70,7 +67,7 @@ class PartitionHubBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), 5.seconds) + system.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/SourceRefBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/SourceRefBenchmark.scala index 87d69bce58c..dcd76980e3c 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/SourceRefBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/SourceRefBenchmark.scala @@ -16,15 +16,13 @@ package org.apache.pekko.stream import java.util.concurrent.Semaphore import java.util.concurrent.TimeUnit -import scala.concurrent.Await -import scala.concurrent.duration._ import scala.util.Success import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations._ import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.remote.artery.BenchTestSource import pekko.stream.scaladsl._ @@ -62,7 +60,7 @@ class SourceRefBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), 5.seconds) + system.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/impl/OutputStreamSourceStageBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/impl/OutputStreamSourceStageBenchmark.scala index 4b06c731d79..d5a7ff3a5bc 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/impl/OutputStreamSourceStageBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/impl/OutputStreamSourceStageBenchmark.scala @@ -22,7 +22,7 @@ import org.openjdk.jmh.annotations._ import org.openjdk.jmh.annotations.TearDown import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.stream.scaladsl.Keep import pekko.stream.scaladsl.Sink import pekko.stream.scaladsl.StreamConverters @@ -58,7 +58,7 @@ class OutputStreamSourceStageBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), 5.seconds) + system.close() } } diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/io/FileSourcesBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/io/FileSourcesBenchmark.scala index 3f558f044c7..8293e723c6a 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/io/FileSourcesBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/io/FileSourcesBenchmark.scala @@ -27,7 +27,7 @@ import org.openjdk.jmh.annotations._ import org.apache.pekko import pekko.Done import pekko.NotUsed -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.stream.Attributes import pekko.stream.IOResult import pekko.stream.scaladsl._ @@ -80,7 +80,7 @@ class FileSourcesBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), Duration.Inf) + system.close() } @Benchmark diff --git a/bench-jmh/src/main/scala/org/apache/pekko/stream/io/FileSourcesScaleBenchmark.scala b/bench-jmh/src/main/scala/org/apache/pekko/stream/io/FileSourcesScaleBenchmark.scala index e007ed22b49..7d0e170eefa 100644 --- a/bench-jmh/src/main/scala/org/apache/pekko/stream/io/FileSourcesScaleBenchmark.scala +++ b/bench-jmh/src/main/scala/org/apache/pekko/stream/io/FileSourcesScaleBenchmark.scala @@ -27,7 +27,7 @@ import org.openjdk.jmh.annotations.Scope import org.openjdk.jmh.annotations.State import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.stream.IOResult import pekko.stream.scaladsl._ import pekko.util.ByteString @@ -79,7 +79,7 @@ class FileSourcesScaleBenchmark { @TearDown def shutdown(): Unit = { - Await.result(system.terminate(), Duration.Inf) + system.close() } @Benchmark diff --git a/cluster-sharding/src/main/scala/org/apache/pekko/cluster/sharding/RemoveInternalClusterShardingData.scala b/cluster-sharding/src/main/scala/org/apache/pekko/cluster/sharding/RemoveInternalClusterShardingData.scala index 9f52a161f70..80bb762b168 100644 --- a/cluster-sharding/src/main/scala/org/apache/pekko/cluster/sharding/RemoveInternalClusterShardingData.scala +++ b/cluster-sharding/src/main/scala/org/apache/pekko/cluster/sharding/RemoveInternalClusterShardingData.scala @@ -73,7 +73,7 @@ object RemoveInternalClusterShardingData { if (args.isEmpty) println("Specify the Cluster Sharding type names to remove in program arguments") else { - val system = ActorSystem("RemoveInternalClusterShardingData") + val system = pekko.actor.scaladsl.ActorSystem("RemoveInternalClusterShardingData") val remove2dot3Data = args(0) == "-2.3" val typeNames = if (remove2dot3Data) args.tail.toSet else args.toSet if (typeNames.isEmpty) @@ -82,7 +82,7 @@ object RemoveInternalClusterShardingData { val journalPluginId = system.settings.config.getString("pekko.cluster.sharding.journal-plugin-id") import system.dispatcher remove(system, journalPluginId, typeNames, remove2dot3Data).onComplete { _ => - system.terminate() + system.terminateAsync() } } } diff --git a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sbr/RandomizedBrainResolverIntegrationSpec.scala b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sbr/RandomizedBrainResolverIntegrationSpec.scala index 39ca53bb23c..db358d0d213 100644 --- a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sbr/RandomizedBrainResolverIntegrationSpec.scala +++ b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sbr/RandomizedBrainResolverIntegrationSpec.scala @@ -23,6 +23,7 @@ import org.scalatest.BeforeAndAfterEach import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.Cluster import pekko.cluster.MemberStatus import pekko.cluster.MultiNodeClusterSpec diff --git a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sbr/SplitBrainResolverIntegrationSpec.scala b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sbr/SplitBrainResolverIntegrationSpec.scala index 9bd9bba49fc..ad528e2eedf 100644 --- a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sbr/SplitBrainResolverIntegrationSpec.scala +++ b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sbr/SplitBrainResolverIntegrationSpec.scala @@ -23,6 +23,7 @@ import org.scalatest.BeforeAndAfterEach import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.Cluster import pekko.cluster.ClusterSettings.DataCenter import pekko.cluster.ClusterSettings.DefaultDataCenter diff --git a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesNewExtractorSpec.scala b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesNewExtractorSpec.scala index 1614d401458..519c50fc25c 100644 --- a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesNewExtractorSpec.scala +++ b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesNewExtractorSpec.scala @@ -210,7 +210,7 @@ abstract class ClusterShardingRememberEntitiesNewExtractorSpec( // no sharding nodes left of the original cluster, start a new nodes runOn(second, third) { - val sys2 = ActorSystem(system.name, system.settings.config) + val sys2 = pekko.actor.scaladsl.ActorSystem(system.name, system.settings.config) val probe2 = TestProbe()(sys2) if (persistenceIsNeeded) { diff --git a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesSpec.scala b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesSpec.scala index a24c33d3061..07fe1d01a4a 100644 --- a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesSpec.scala +++ b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesSpec.scala @@ -212,7 +212,8 @@ abstract class ClusterShardingRememberEntitiesSpec(multiNodeConfig: ClusterShard } // no nodes left of the original cluster, start a new cluster - val sys2 = ActorSystem(system.name, MultiNodeSpec.configureNextPortIfFixed(system.settings.config)) + val sys2 = + pekko.actor.scaladsl.ActorSystem(system.name, MultiNodeSpec.configureNextPortIfFixed(system.settings.config)) val entityProbe2 = TestProbe()(sys2) if (persistenceIsNeeded) setStore(sys2, storeOn = first) diff --git a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingSettingsSpec.scala b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingSettingsSpec.scala index 43055355ff7..cc29aeb6327 100644 --- a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingSettingsSpec.scala +++ b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingSettingsSpec.scala @@ -14,7 +14,7 @@ package org.apache.pekko.cluster.sharding import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.testkit.{ PekkoSpec, TestKit } import com.typesafe.config.ConfigFactory import org.scalatest.matchers.should.Matchers diff --git a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/CoordinatedShutdownShardingSpec.scala b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/CoordinatedShutdownShardingSpec.scala index c232989b8e9..4c7f5817a43 100644 --- a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/CoordinatedShutdownShardingSpec.scala +++ b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/CoordinatedShutdownShardingSpec.scala @@ -18,7 +18,7 @@ import scala.concurrent.duration._ import org.apache.pekko import pekko.Done -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.actor.CoordinatedShutdown import pekko.actor.Props import pekko.cluster.Cluster @@ -177,7 +177,7 @@ class CoordinatedShutdownShardingSpec extends PekkoSpec(CoordinatedShutdownShard within(10.seconds) { awaitAssert { Cluster(sys3).isTerminated should ===(true) - sys3.whenTerminated.isCompleted should ===(true) + sys3.whenTerminatedImpl.isCompleted should ===(true) } } diff --git a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/JoinConfigCompatCheckShardingSpec.scala b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/JoinConfigCompatCheckShardingSpec.scala index 7010278bff3..34ecdefe6b4 100644 --- a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/JoinConfigCompatCheckShardingSpec.scala +++ b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/JoinConfigCompatCheckShardingSpec.scala @@ -19,7 +19,7 @@ import scala.concurrent.duration._ import com.typesafe.config.{ Config, ConfigFactory } import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.{ Cluster, ClusterReadView } import pekko.testkit.WithLogCapturing import pekko.testkit.{ LongRunningTest, PekkoSpec } diff --git a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/PersistentShardingMigrationSpec.scala b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/PersistentShardingMigrationSpec.scala index 2c20e1cf3c7..ed864fb7c40 100644 --- a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/PersistentShardingMigrationSpec.scala +++ b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/PersistentShardingMigrationSpec.scala @@ -15,14 +15,14 @@ package org.apache.pekko.cluster.sharding import java.util.UUID import org.apache.pekko -import pekko.actor.{ ActorRef, ActorSystem, Props } +import pekko.actor.{ ActorRef, Props } +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.sharding.ShardRegion.CurrentRegions import pekko.cluster.{ Cluster, MemberStatus } import pekko.persistence.PersistentActor import pekko.testkit.{ ImplicitSender, PekkoSpec, TestProbe } import com.typesafe.config.{ Config, ConfigFactory } -import scala.concurrent.Await import scala.concurrent.duration._ /** @@ -183,7 +183,7 @@ class PersistentShardingMigrationSpec extends PekkoSpec(PersistentShardingMigrat extractShardId(rememberedEntitiesProbe.ref)) f(system, region, rememberedEntitiesProbe) } finally { - Await.ready(system.terminate(), 20.seconds) + system.close() } } diff --git a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesShardIdExtractorChangeSpec.scala b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesShardIdExtractorChangeSpec.scala index 45dbacf1511..8d0e5b91d0d 100644 --- a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesShardIdExtractorChangeSpec.scala +++ b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesShardIdExtractorChangeSpec.scala @@ -17,8 +17,8 @@ import java.util.UUID import org.apache.pekko import pekko.actor.ActorRef -import pekko.actor.ActorSystem import pekko.actor.Props +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.Cluster import pekko.cluster.sharding.ShardRegion.CurrentRegions import pekko.persistence.PersistentActor @@ -27,9 +27,6 @@ import pekko.testkit.ImplicitSender import pekko.testkit.TestProbe import com.typesafe.config.ConfigFactory -import scala.concurrent.Await -import scala.concurrent.duration._ - /** * Covers that remembered entities is correctly migrated when used and the shard id extractor * is changed so that entities should live on other shards after a full restart of the cluster. @@ -144,7 +141,7 @@ class RememberEntitiesShardIdExtractorChangeSpec val region = ClusterSharding(system).start(TypeName, Props(new PA()), extractEntityId, extractShardId) f(system, region) } finally { - Await.ready(system.terminate(), 20.seconds) + system.close() } } diff --git a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardRegionSpec.scala b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardRegionSpec.scala index 95a7df4296b..0c314e4f39d 100644 --- a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardRegionSpec.scala +++ b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardRegionSpec.scala @@ -118,7 +118,7 @@ class ShardRegionSpec extends PekkoSpec(ShardRegionSpec.config) with WithLogCapt system.eventStream.publish(Mute(DeadLettersFilter[Any])) private val sysA = system - private val sysB = ActorSystem(system.name, system.settings.config) + private val sysB = pekko.actor.scaladsl.ActorSystem(system.name, system.settings.config) private val p1 = TestProbe()(sysA) private val p2 = TestProbe()(sysB) diff --git a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/passivation/simulator/Simulator.scala b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/passivation/simulator/Simulator.scala index 82a7122ef38..94dba984f53 100644 --- a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/passivation/simulator/Simulator.scala +++ b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/passivation/simulator/Simulator.scala @@ -15,7 +15,7 @@ package org.apache.pekko.cluster.sharding.passivation.simulator import org.apache.pekko import pekko.NotUsed -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.sharding.internal.ActiveEntities import pekko.cluster.sharding.internal.AdmissionFilter import pekko.cluster.sharding.internal.AdmissionOptimizer @@ -81,10 +81,10 @@ object Simulator { results.name +: DataTable.row(results.stats.totals) } PrintData(DataTable(DataTable.Headers.RunStats, summary)) - system.terminate() + system.terminateAsync() case Failure(exception) => println(s"Failed to run simulations: $exception") - system.terminate() + system.terminateAsync() } } diff --git a/cluster-tools/src/multi-jvm/scala/org/apache/pekko/cluster/client/ClusterClientSpec.scala b/cluster-tools/src/multi-jvm/scala/org/apache/pekko/cluster/client/ClusterClientSpec.scala index d5b1be40d84..e29ce2eb67d 100644 --- a/cluster-tools/src/multi-jvm/scala/org/apache/pekko/cluster/client/ClusterClientSpec.scala +++ b/cluster-tools/src/multi-jvm/scala/org/apache/pekko/cluster/client/ClusterClientSpec.scala @@ -70,7 +70,7 @@ object ClusterClientSpec extends MultiNodeConfig { class TestService(testActor: ActorRef) extends Actor { def receive = { case "shutdown" => - context.system.terminate() + context.system.closeAsync() case msg => testActor.forward(msg) sender() ! Reply(s"$msg-ack", Cluster(context.system).selfAddress) diff --git a/cluster-tools/src/multi-jvm/scala/org/apache/pekko/cluster/pubsub/DistributedPubSubRestartSpec.scala b/cluster-tools/src/multi-jvm/scala/org/apache/pekko/cluster/pubsub/DistributedPubSubRestartSpec.scala index fa1b0846244..605c0725ce2 100644 --- a/cluster-tools/src/multi-jvm/scala/org/apache/pekko/cluster/pubsub/DistributedPubSubRestartSpec.scala +++ b/cluster-tools/src/multi-jvm/scala/org/apache/pekko/cluster/pubsub/DistributedPubSubRestartSpec.scala @@ -22,10 +22,10 @@ import org.apache.pekko import pekko.actor.Actor import pekko.actor.ActorIdentity import pekko.actor.ActorRef -import pekko.actor.ActorSystem import pekko.actor.Identify import pekko.actor.Props import pekko.actor.RootActorPath +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.Cluster import pekko.remote.testconductor.RoleName import pekko.remote.testkit.MultiNodeConfig @@ -51,7 +51,7 @@ object DistributedPubSubRestartSpec extends MultiNodeConfig { class Shutdown extends Actor { def receive = { - case "shutdown" => context.system.terminate() + case "shutdown" => context.system.closeAsync() } } @@ -146,7 +146,7 @@ class DistributedPubSubRestartSpec } runOn(third) { - Await.result(system.whenTerminated, 10.seconds) + Await.result(system.whenTerminatedImpl, 10.seconds) val newSystem = { val port = Cluster(system).selfAddress.port.get val config = ConfigFactory.parseString(s""" @@ -171,8 +171,8 @@ class DistributedPubSubRestartSpec probe.expectMsg(0L) newSystem.actorOf(Props[Shutdown](), "shutdown") - Await.ready(newSystem.whenTerminated, 20.seconds) - } finally newSystem.terminate() + newSystem.close() + } finally newSystem.terminateAsync() } } diff --git a/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala b/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala index c7546db1782..3bda0ae1081 100644 --- a/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala +++ b/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala @@ -77,7 +77,7 @@ class ClusterSingletonLeavingSpeedSpec private val systems = (1 to 3).map { n => val roleConfig = ConfigFactory.parseString(s"""pekko.cluster.roles=[role-${n % 3}]""") - ActorSystem(system.name, roleConfig.withFallback(system.settings.config)) + pekko.actor.scaladsl.ActorSystem(system.name, roleConfig.withFallback(system.settings.config)) } private val probes = systems.map(TestProbe()(_)) diff --git a/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonProxySpec.scala b/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonProxySpec.scala index 8ddbe58759a..2087623c3eb 100644 --- a/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonProxySpec.scala +++ b/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonProxySpec.scala @@ -22,6 +22,7 @@ import org.scalatest.wordspec.AnyWordSpecLike import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.Cluster import pekko.testkit.{ TestKit, TestProbe } diff --git a/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestart2Spec.scala b/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestart2Spec.scala index 94846fc3cd4..c85fee1fd6b 100644 --- a/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestart2Spec.scala +++ b/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestart2Spec.scala @@ -19,9 +19,9 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.Actor -import pekko.actor.ActorSystem import pekko.actor.PoisonPill import pekko.actor.Props +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.Cluster import pekko.cluster.MemberStatus import pekko.cluster.UniqueAddress diff --git a/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestartSpec.scala b/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestartSpec.scala index b9acf76dc2d..a93ccb60bdc 100644 --- a/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestartSpec.scala +++ b/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestartSpec.scala @@ -18,7 +18,7 @@ import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.actor.PoisonPill import pekko.cluster.Cluster import pekko.cluster.MemberStatus diff --git a/cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/ClusterApiSpec.scala b/cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/ClusterApiSpec.scala index 619fbfaebe4..6ebec420999 100644 --- a/cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/ClusterApiSpec.scala +++ b/cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/ClusterApiSpec.scala @@ -60,7 +60,7 @@ class ClusterApiSpec extends ScalaTestWithActorTestKit(ClusterApiSpec.config) wi "join a cluster and observe events from both sides" in { - val system2 = pekko.actor.ActorSystem(system.name, system.settings.config) + val system2 = pekko.actor.scaladsl.ActorSystem(system.name, system.settings.config) val adaptedSystem2 = system2.toTyped try { diff --git a/cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/ClusterSingletonApiSpec.scala b/cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/ClusterSingletonApiSpec.scala index 11e60b17d0f..6e42f0ad946 100644 --- a/cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/ClusterSingletonApiSpec.scala +++ b/cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/ClusterSingletonApiSpec.scala @@ -13,7 +13,6 @@ package org.apache.pekko.cluster.typed -import scala.concurrent.Await import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory @@ -67,7 +66,7 @@ class ClusterSingletonApiSpec val clusterNode1 = Cluster(system) val classicSystem1 = system.toClassic - val system2 = pekko.actor.ActorSystem( + val system2 = pekko.actor.scaladsl.ActorSystem( system.name, ConfigFactory.parseString(""" pekko.cluster.roles = ["singleton"] @@ -121,7 +120,7 @@ class ClusterSingletonApiSpec override def afterAll(): Unit = { super.afterAll() - Await.result(system2.terminate(), 3.seconds) + system2.close() } } diff --git a/cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/RemoteMessageSpec.scala b/cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/RemoteMessageSpec.scala index fb802f2b015..24ef20c546a 100644 --- a/cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/RemoteMessageSpec.scala +++ b/cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/RemoteMessageSpec.scala @@ -19,7 +19,7 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.Done -import pekko.actor.{ ActorSystem => ClassicActorSystem } +import pekko.actor.scaladsl.{ ActorSystem => ClassicActorSystem } import pekko.actor.typed.ActorRef import pekko.actor.typed.ActorRefResolver import pekko.actor.typed.scaladsl.Behaviors @@ -87,7 +87,7 @@ class RemoteMessageSpec extends PekkoSpec(RemoteMessageSpec.config) { pongPromise.future.futureValue should ===(Done) } finally { - system2.terminate() + system2.closeAsync() } } diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/ClusterDeathWatchSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/ClusterDeathWatchSpec.scala index f72df20774a..376e66a1def 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/ClusterDeathWatchSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/ClusterDeathWatchSpec.scala @@ -23,6 +23,7 @@ import org.scalatest.concurrent.ScalaFutures import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.MultiNodeClusterSpec.EndActor import pekko.remote.RemoteActorRef import pekko.remote.RemoteWatcher @@ -280,7 +281,7 @@ abstract class ClusterDeathWatchSpec enterBarrier("first-unavailable") val timeout = remainingOrDefault - try Await.ready(system.whenTerminated, timeout) + try Await.ready(system.whenTerminatedImpl, timeout) catch { case _: TimeoutException => fail( diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MultiDcSplitBrainSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MultiDcSplitBrainSpec.scala index 95c38aa013d..12bc8a0c4df 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MultiDcSplitBrainSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MultiDcSplitBrainSpec.scala @@ -19,7 +19,7 @@ import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.ClusterEvent._ import pekko.remote.testconductor.RoleName import pekko.remote.testkit.MultiNodeConfig @@ -261,7 +261,7 @@ abstract class MultiDcSplitBrainSpec extends MultiNodeClusterSpec(MultiDcSplitBr // actor system to be able to start new with same port val thirdAddress = address(third) enterBarrier("fifth-waiting-for-termination") - Await.ready(system.whenTerminated, remaining) + Await.ready(system.whenTerminatedImpl, remaining) val port = Cluster(system).selfAddress.port.get val restartedSystem = ActorSystem( @@ -272,7 +272,7 @@ abstract class MultiDcSplitBrainSpec extends MultiNodeClusterSpec(MultiDcSplitBr pekko.coordinated-shutdown.terminate-actor-system = on """).withFallback(system.settings.config)) Cluster(restartedSystem).join(thirdAddress) - Await.ready(restartedSystem.whenTerminated, remaining) + Await.ready(restartedSystem.whenTerminatedImpl, remaining) } // no multi-jvm test facilities on fifth after this diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/NodeChurnSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/NodeChurnSpec.scala index be2416303ab..576d3d6bca0 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/NodeChurnSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/NodeChurnSpec.scala @@ -20,6 +20,7 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.event.Logging.Info import pekko.remote.RARP import pekko.remote.testkit.MultiNodeConfig diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/QuickRestartSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/QuickRestartSpec.scala index 0ee7e4bbb0e..a1f0fc8a6f6 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/QuickRestartSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/QuickRestartSpec.scala @@ -21,7 +21,8 @@ import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory import org.apache.pekko -import pekko.actor.{ ActorSystem, Address } +import pekko.actor.Address +import pekko.actor.scaladsl.ActorSystem import pekko.remote.testkit.{ MultiNodeConfig, MultiNodeSpec } import pekko.testkit._ @@ -116,7 +117,7 @@ abstract class QuickRestartSpec extends MultiNodeClusterSpec(QuickRestartMultiJv enterBarrier("before-terminate-" + n) runOn(second) { - restartingSystem.terminate().await + restartingSystem.terminateAsync().await } // don't wait for it to be removed, new incarnation will join in next round enterBarrier("terminated-" + n) diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartFirstSeedNodeSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartFirstSeedNodeSpec.scala index 6b43c7a37d8..1cdf417db6a 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartFirstSeedNodeSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartFirstSeedNodeSpec.scala @@ -20,11 +20,11 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.Actor -import pekko.actor.ActorSystem import pekko.actor.Address import pekko.actor.Deploy import pekko.actor.Props import pekko.actor.RootActorPath +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.MemberStatus._ import pekko.remote.testkit.MultiNodeConfig import pekko.remote.testkit.MultiNodeSpec diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode2Spec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode2Spec.scala index 1c12f00663e..60f5eb72148 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode2Spec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode2Spec.scala @@ -20,11 +20,11 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.Actor -import pekko.actor.ActorSystem import pekko.actor.Address import pekko.actor.Deploy import pekko.actor.Props import pekko.actor.RootActorPath +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.MemberStatus._ import pekko.remote.testkit.MultiNodeConfig import pekko.remote.testkit.MultiNodeSpec diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode3Spec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode3Spec.scala index 55bcf7ccc56..6529ce58d69 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode3Spec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode3Spec.scala @@ -20,11 +20,11 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.Actor -import pekko.actor.ActorSystem import pekko.actor.Address import pekko.actor.Deploy import pekko.actor.Props import pekko.actor.RootActorPath +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.MemberStatus._ import pekko.remote.testkit.MultiNodeConfig import pekko.remote.testkit.MultiNodeSpec diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNodeSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNodeSpec.scala index ffb76c21cb4..bae94e71a2f 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNodeSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNodeSpec.scala @@ -23,13 +23,13 @@ import pekko.Done import pekko.actor.Actor import pekko.actor.ActorIdentity import pekko.actor.ActorRef -import pekko.actor.ActorSystem import pekko.actor.Address import pekko.actor.Deploy import pekko.actor.Identify import pekko.actor.Props import pekko.actor.RootActorPath import pekko.actor.Terminated +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.MemberStatus._ import pekko.remote.testkit.MultiNodeConfig import pekko.remote.testkit.MultiNodeSpec diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/StressSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/StressSpec.scala index 0e9ad19534b..adcf005dfad 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/StressSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/StressSpec.scala @@ -787,7 +787,8 @@ abstract class StressSpec extends MultiNodeClusterSpec(StressMultiJvmSpec) with previousAS.foreach { as => TestKit.shutdownActorSystem(as) } - val sys = ActorSystem(system.name, MultiNodeSpec.configureNextPortIfFixed(system.settings.config)) + val sys = pekko.actor.scaladsl.ActorSystem(system.name, + MultiNodeSpec.configureNextPortIfFixed(system.settings.config)) muteLog(sys) Cluster(sys).joinSeedNodes(seedNodes.toIndexedSeq.map(address)) Some(sys) diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/UnreachableNodeJoinsAgainSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/UnreachableNodeJoinsAgainSpec.scala index 5eadcd4e41f..0457f43b09a 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/UnreachableNodeJoinsAgainSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/UnreachableNodeJoinsAgainSpec.scala @@ -20,9 +20,9 @@ import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory import org.apache.pekko -import pekko.actor.ActorSystem import pekko.actor.ExtendedActorSystem import pekko.actor.Props +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.MultiNodeClusterSpec.EndActor import pekko.remote.RARP import pekko.remote.testconductor.RoleName diff --git a/cluster/src/test/scala/org/apache/pekko/cluster/ClusterDeathWatchNotificationSpec.scala b/cluster/src/test/scala/org/apache/pekko/cluster/ClusterDeathWatchNotificationSpec.scala index 397ab77a149..1d41c4b7d09 100644 --- a/cluster/src/test/scala/org/apache/pekko/cluster/ClusterDeathWatchNotificationSpec.scala +++ b/cluster/src/test/scala/org/apache/pekko/cluster/ClusterDeathWatchNotificationSpec.scala @@ -132,7 +132,7 @@ class ClusterDeathWatchNotificationSpec receiverProbe3.expectMsg("echo3") system2.log.debug("terminating") - system2.terminate() + system2.terminateAsync() receiverProbe1.receiveN(messages.size, 5.seconds).toVector shouldBe messages receiverProbe1.expectTerminated(sender1) receiverProbe2.receiveN(messages.size).toVector shouldBe messages diff --git a/cluster/src/test/scala/org/apache/pekko/cluster/ClusterSpec.scala b/cluster/src/test/scala/org/apache/pekko/cluster/ClusterSpec.scala index 8dd0cdf01e9..be1ca561386 100644 --- a/cluster/src/test/scala/org/apache/pekko/cluster/ClusterSpec.scala +++ b/cluster/src/test/scala/org/apache/pekko/cluster/ClusterSpec.scala @@ -22,11 +22,11 @@ import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory import org.apache.pekko -import pekko.actor.ActorSystem import pekko.actor.Address import pekko.actor.CoordinatedShutdown import pekko.actor.ExtendedActorSystem import pekko.actor.Props +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.ClusterEvent._ import pekko.cluster.ClusterEvent.MemberEvent import pekko.cluster.InternalClusterAction._ diff --git a/cluster/src/test/scala/org/apache/pekko/cluster/ClusterTestKit.scala b/cluster/src/test/scala/org/apache/pekko/cluster/ClusterTestKit.scala index 2c501712297..1456c394413 100644 --- a/cluster/src/test/scala/org/apache/pekko/cluster/ClusterTestKit.scala +++ b/cluster/src/test/scala/org/apache/pekko/cluster/ClusterTestKit.scala @@ -19,7 +19,7 @@ import scala.util.Random import com.typesafe.config.{ Config, ConfigFactory } import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.testkit.{ PekkoSpec, TestKitBase } /** diff --git a/cluster/src/test/scala/org/apache/pekko/cluster/DowningProviderSpec.scala b/cluster/src/test/scala/org/apache/pekko/cluster/DowningProviderSpec.scala index 3619fb2488f..d150f05edf7 100644 --- a/cluster/src/test/scala/org/apache/pekko/cluster/DowningProviderSpec.scala +++ b/cluster/src/test/scala/org/apache/pekko/cluster/DowningProviderSpec.scala @@ -20,7 +20,7 @@ import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec import org.apache.pekko import pekko.ConfigurationException -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.actor.Props import pekko.testkit.TestKit.awaitCond import pekko.testkit.TestKit.shutdownActorSystem diff --git a/cluster/src/test/scala/org/apache/pekko/cluster/ShutdownAfterJoinSeedNodesSpec.scala b/cluster/src/test/scala/org/apache/pekko/cluster/ShutdownAfterJoinSeedNodesSpec.scala index e81b9ed452b..c485d6d6602 100644 --- a/cluster/src/test/scala/org/apache/pekko/cluster/ShutdownAfterJoinSeedNodesSpec.scala +++ b/cluster/src/test/scala/org/apache/pekko/cluster/ShutdownAfterJoinSeedNodesSpec.scala @@ -18,8 +18,8 @@ import scala.concurrent.Await import scala.concurrent.duration._ import org.apache.pekko -import pekko.actor.ActorSystem import pekko.actor.Address +import pekko.actor.scaladsl.ActorSystem import pekko.testkit._ object ShutdownAfterJoinSeedNodesSpec { diff --git a/discovery/src/test/java/jdoc/org/apache/pekko/discovery/CompileOnlyTest.java b/discovery/src/test/java/jdoc/org/apache/pekko/discovery/CompileOnlyTest.java index c2c8c7d19f2..e7157f67200 100644 --- a/discovery/src/test/java/jdoc/org/apache/pekko/discovery/CompileOnlyTest.java +++ b/discovery/src/test/java/jdoc/org/apache/pekko/discovery/CompileOnlyTest.java @@ -13,7 +13,7 @@ package jdoc.org.apache.pekko.discovery; -import org.apache.pekko.actor.ActorSystem; +import org.apache.pekko.actor.javadsl.ActorSystem; import org.apache.pekko.discovery.Lookup; import org.apache.pekko.discovery.Discovery; import org.apache.pekko.discovery.ServiceDiscovery; diff --git a/discovery/src/test/scala/doc/org/apache/pekko/discovery/CompileOnlySpec.scala b/discovery/src/test/scala/doc/org/apache/pekko/discovery/CompileOnlySpec.scala index 43d07af6875..fc53bcca496 100644 --- a/discovery/src/test/scala/doc/org/apache/pekko/discovery/CompileOnlySpec.scala +++ b/discovery/src/test/scala/doc/org/apache/pekko/discovery/CompileOnlySpec.scala @@ -13,7 +13,7 @@ package doc.org.apache.pekko.discovery -import org.apache.pekko.actor.ActorSystem +import org.apache.pekko.actor.scaladsl.ActorSystem import scala.concurrent.Future import scala.concurrent.duration._ diff --git a/discovery/src/test/scala/org/apache/pekko/discovery/DiscoveryConfigurationSpec.scala b/discovery/src/test/scala/org/apache/pekko/discovery/DiscoveryConfigurationSpec.scala index 1e0224248bb..cb55aa120dc 100644 --- a/discovery/src/test/scala/org/apache/pekko/discovery/DiscoveryConfigurationSpec.scala +++ b/discovery/src/test/scala/org/apache/pekko/discovery/DiscoveryConfigurationSpec.scala @@ -21,7 +21,7 @@ import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.discovery.ServiceDiscovery.Resolved import pekko.testkit.TestKit diff --git a/discovery/src/test/scala/org/apache/pekko/discovery/aggregate/AggregateServiceDiscoverySpec.scala b/discovery/src/test/scala/org/apache/pekko/discovery/aggregate/AggregateServiceDiscoverySpec.scala index c89928d8852..205ab3e8538 100644 --- a/discovery/src/test/scala/org/apache/pekko/discovery/aggregate/AggregateServiceDiscoverySpec.scala +++ b/discovery/src/test/scala/org/apache/pekko/discovery/aggregate/AggregateServiceDiscoverySpec.scala @@ -24,7 +24,8 @@ import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpecLike import org.apache.pekko -import pekko.actor.{ ActorSystem, ExtendedActorSystem } +import pekko.actor.ExtendedActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.discovery.{ Discovery, Lookup, ServiceDiscovery } import pekko.discovery.ServiceDiscovery.{ Resolved, ResolvedTarget } import pekko.testkit.TestKit diff --git a/discovery/src/test/scala/org/apache/pekko/discovery/config/ConfigServiceDiscoverySpec.scala b/discovery/src/test/scala/org/apache/pekko/discovery/config/ConfigServiceDiscoverySpec.scala index 5b2a4184445..1b24d89522b 100644 --- a/discovery/src/test/scala/org/apache/pekko/discovery/config/ConfigServiceDiscoverySpec.scala +++ b/discovery/src/test/scala/org/apache/pekko/discovery/config/ConfigServiceDiscoverySpec.scala @@ -22,7 +22,7 @@ import org.scalatest.concurrent.ScalaFutures import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpecLike -import org.apache.pekko.actor.ActorSystem +import org.apache.pekko.actor.scaladsl.ActorSystem import org.apache.pekko.discovery.Discovery import org.apache.pekko.discovery.ServiceDiscovery.ResolvedTarget import org.apache.pekko.testkit.TestKit diff --git a/discovery/src/test/scala/org/apache/pekko/discovery/dns/DnsDiscoverySpec.scala b/discovery/src/test/scala/org/apache/pekko/discovery/dns/DnsDiscoverySpec.scala index 01e82576584..24f31418d4e 100644 --- a/discovery/src/test/scala/org/apache/pekko/discovery/dns/DnsDiscoverySpec.scala +++ b/discovery/src/test/scala/org/apache/pekko/discovery/dns/DnsDiscoverySpec.scala @@ -18,7 +18,7 @@ import java.net.InetAddress import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.discovery.{ Discovery, Lookup } import pekko.discovery.ServiceDiscovery import pekko.discovery.ServiceDiscovery.ResolvedTarget diff --git a/distributed-data/src/multi-jvm/scala/org/apache/pekko/cluster/ddata/DurableDataSpec.scala b/distributed-data/src/multi-jvm/scala/org/apache/pekko/cluster/ddata/DurableDataSpec.scala index 1bde6f2d8d6..105e77ccc5d 100644 --- a/distributed-data/src/multi-jvm/scala/org/apache/pekko/cluster/ddata/DurableDataSpec.scala +++ b/distributed-data/src/multi-jvm/scala/org/apache/pekko/cluster/ddata/DurableDataSpec.scala @@ -257,7 +257,8 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig) "handle Update before load" in { runOn(first) { - val sys1 = ActorSystem("AdditionalSys", MultiNodeSpec.configureNextPortIfFixed(system.settings.config)) + val sys1 = pekko.actor.scaladsl.ActorSystem("AdditionalSys", + MultiNodeSpec.configureNextPortIfFixed(system.settings.config)) val address = Cluster(sys1).selfAddress try { Cluster(sys1).join(address) @@ -289,10 +290,10 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig) expectTerminated(r) } } finally { - Await.ready(sys1.terminate(), 10.seconds) + sys1.close() } - val sys2 = ActorSystem( + val sys2 = pekko.actor.scaladsl.ActorSystem( "AdditionalSys", // use the same port ConfigFactory.parseString(s""" @@ -320,7 +321,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig) expectMsgType[GetSuccess[GCounter]].dataValue.value.toInt should be(2) } } finally { - Await.ready(sys1.terminate(), 10.seconds) + sys1.close() } } diff --git a/distributed-data/src/multi-jvm/scala/org/apache/pekko/cluster/ddata/DurablePruningSpec.scala b/distributed-data/src/multi-jvm/scala/org/apache/pekko/cluster/ddata/DurablePruningSpec.scala index 36edb54f961..b687a6dd0dc 100644 --- a/distributed-data/src/multi-jvm/scala/org/apache/pekko/cluster/ddata/DurablePruningSpec.scala +++ b/distributed-data/src/multi-jvm/scala/org/apache/pekko/cluster/ddata/DurablePruningSpec.scala @@ -13,7 +13,6 @@ package org.apache.pekko.cluster.ddata -import scala.concurrent.Await import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory @@ -85,7 +84,8 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN join(first, first) join(second, first) - val sys2 = ActorSystem(system.name, MultiNodeSpec.configureNextPortIfFixed(system.settings.config)) + val sys2 = + pekko.actor.scaladsl.ActorSystem(system.name, MultiNodeSpec.configureNextPortIfFixed(system.settings.config)) val cluster2 = Cluster(sys2) val distributedData2 = DistributedData(sys2) val replicator2 = startReplicator(sys2) @@ -149,7 +149,7 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN } enterBarrier("removed") runOn(first) { - Await.ready(sys2.terminate(), 5.seconds) + sys2.close() } within(15.seconds) { @@ -168,7 +168,7 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN runOn(first) { val address = cluster2.selfAddress - val sys3 = ActorSystem( + val sys3 = pekko.actor.scaladsl.ActorSystem( system.name, ConfigFactory.parseString(s""" pekko.remote.artery.canonical.port = ${address.port.get} diff --git a/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/LocalConcurrencySpec.scala b/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/LocalConcurrencySpec.scala index ceb966f5455..2188250cb6b 100644 --- a/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/LocalConcurrencySpec.scala +++ b/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/LocalConcurrencySpec.scala @@ -20,9 +20,9 @@ import org.scalatest.wordspec.AnyWordSpecLike import org.apache.pekko import pekko.actor.Actor -import pekko.actor.ActorSystem import pekko.actor.Props import pekko.actor.Stash +import pekko.actor.scaladsl.ActorSystem import pekko.testkit.ImplicitSender import pekko.testkit.TestKit diff --git a/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/LotsOfDataBot.scala b/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/LotsOfDataBot.scala index 723e5ed9143..e5a5dabf24c 100644 --- a/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/LotsOfDataBot.scala +++ b/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/LotsOfDataBot.scala @@ -22,8 +22,8 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.Actor import pekko.actor.ActorLogging -import pekko.actor.ActorSystem import pekko.actor.Props +import pekko.actor.scaladsl.ActorSystem /** * This "sample" simulates lots of data entries, and can be used for diff --git a/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/VersionVectorSpec.scala b/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/VersionVectorSpec.scala index ad1e871d45e..26930717a0c 100644 --- a/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/VersionVectorSpec.scala +++ b/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/VersionVectorSpec.scala @@ -18,8 +18,8 @@ import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpecLike import org.apache.pekko -import pekko.actor.ActorSystem import pekko.actor.Address +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.UniqueAddress import pekko.testkit.TestKit diff --git a/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatedDataSerializerSpec.scala b/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatedDataSerializerSpec.scala index 66116ed1939..1ba8dc69e46 100644 --- a/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatedDataSerializerSpec.scala +++ b/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatedDataSerializerSpec.scala @@ -21,12 +21,12 @@ import org.scalatest.wordspec.AnyWordSpecLike import org.apache.pekko import pekko.actor.ActorIdentity import pekko.actor.ActorRef -import pekko.actor.ActorSystem import pekko.actor.Address import pekko.actor.ExtendedActorSystem import pekko.actor.Identify import pekko.actor.Props import pekko.actor.RootActorPath +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.Cluster import pekko.cluster.UniqueAddress import pekko.cluster.ddata._ diff --git a/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatorMessageSerializerSpec.scala b/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatorMessageSerializerSpec.scala index c1a0212e0dc..d37824fc869 100644 --- a/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatorMessageSerializerSpec.scala +++ b/distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatorMessageSerializerSpec.scala @@ -21,10 +21,10 @@ import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpecLike import org.apache.pekko -import pekko.actor.ActorSystem import pekko.actor.Address import pekko.actor.ExtendedActorSystem import pekko.actor.Props +import pekko.actor.scaladsl.ActorSystem import pekko.cluster.UniqueAddress import pekko.cluster.ddata.DurableStore.DurableDataEnvelope import pekko.cluster.ddata.GCounter diff --git a/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testconductor/Player.scala b/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testconductor/Player.scala index 3c0a452a3bd..5a59ae72ea1 100644 --- a/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testconductor/Player.scala +++ b/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testconductor/Player.scala @@ -80,7 +80,7 @@ trait Player { this: TestConductorExt => private def client = _client match { case null => throw new IllegalStateException("TestConductor client not yet started") - case _ if system.whenTerminated.isCompleted => + case _ if system.whenTerminatedImpl.isCompleted => throw new IllegalStateException( "TestConductor unavailable because system is terminated; you need to startNewSystem() before this point") case x => x @@ -288,7 +288,7 @@ private[pekko] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress // FIXME: Currently ignoring, needs support from Remoting stay() case TerminateMsg(Left(false)) => - context.system.terminate() + context.system.closeAsync() stop() case TerminateMsg(Left(true)) => context.system.asInstanceOf[ActorSystemImpl].abort() diff --git a/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testkit/MultiNodeSpec.scala b/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testkit/MultiNodeSpec.scala index 3b46c29822f..4ba64e69497 100644 --- a/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testkit/MultiNodeSpec.scala +++ b/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testkit/MultiNodeSpec.scala @@ -332,12 +332,12 @@ abstract class MultiNodeSpec( val name = TestKitUtils.testNameFromCallStack(classOf[MultiNodeSpec], "".r) config => try { - ActorSystem(name, config) + pekko.actor.scaladsl.ActorSystem(name, config) } catch { // Retry creating the system once as when using port = 0 two systems may try and use the same one. // RTE is for aeron, CE for netty - case _: RemoteTransportException => ActorSystem(name, config) - case _: ChannelException => ActorSystem(name, config) + case _: RemoteTransportException => pekko.actor.scaladsl.ActorSystem(name, config) + case _: ChannelException => pekko.actor.scaladsl.ActorSystem(name, config) } }) @@ -554,7 +554,7 @@ abstract class MultiNodeSpec( val config = ConfigFactory .parseString(s"pekko.remote.classic.netty.tcp{port=${myAddress.port.get}\nhostname=${myAddress.host.get}}") .withFallback(system.settings.config) - val sys = ActorSystem(system.name, config) + val sys = pekko.actor.scaladsl.ActorSystem(system.name, config) injectDeployments(sys, myself) attachConductor(TestConductor(sys)) sys diff --git a/osgi/src/main/scala/org/apache/pekko/osgi/ActorSystemActivator.scala b/osgi/src/main/scala/org/apache/pekko/osgi/ActorSystemActivator.scala index 12536c0326d..5e422bcb793 100644 --- a/osgi/src/main/scala/org/apache/pekko/osgi/ActorSystemActivator.scala +++ b/osgi/src/main/scala/org/apache/pekko/osgi/ActorSystemActivator.scala @@ -98,7 +98,7 @@ abstract class ActorSystemActivator extends BundleActivator { */ def stop(context: BundleContext): Unit = { registration.foreach(_.unregister()) - system.foreach(_.terminate()) + system.foreach(_.terminateImpl()) } /** diff --git a/osgi/src/main/scala/org/apache/pekko/osgi/OsgiActorSystemFactory.scala b/osgi/src/main/scala/org/apache/pekko/osgi/OsgiActorSystemFactory.scala index 5b5c319e876..acfd44d045e 100644 --- a/osgi/src/main/scala/org/apache/pekko/osgi/OsgiActorSystemFactory.scala +++ b/osgi/src/main/scala/org/apache/pekko/osgi/OsgiActorSystemFactory.scala @@ -17,7 +17,7 @@ import com.typesafe.config.{ Config, ConfigFactory } import org.osgi.framework.BundleContext import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.util.unused /** diff --git a/osgi/src/test/scala/org/apache/pekko/osgi/ActorSystemActivatorTest.scala b/osgi/src/test/scala/org/apache/pekko/osgi/ActorSystemActivatorTest.scala index 2b73242f16f..bf73f6f6383 100644 --- a/osgi/src/test/scala/org/apache/pekko/osgi/ActorSystemActivatorTest.scala +++ b/osgi/src/test/scala/org/apache/pekko/osgi/ActorSystemActivatorTest.scala @@ -62,11 +62,11 @@ class PingPongActorSystemActivatorTest extends AnyWordSpec with Matchers with Po "stop the ActorSystem when bundle stops" in { filterErrors() { val system = serviceForType[ActorSystem] - system.whenTerminated.isCompleted should be(false) + system.whenTerminatedImpl.isCompleted should be(false) bundleForName(TEST_BUNDLE_NAME).stop() - Await.ready(system.whenTerminated, Duration.Inf) - system.whenTerminated.isCompleted should be(true) + Await.ready(system.whenTerminatedImpl, Duration.Inf) + system.whenTerminatedImpl.isCompleted should be(true) } } } diff --git a/persistence-query/src/main/scala/org/apache/pekko/persistence/query/PersistenceQuery.scala b/persistence-query/src/main/scala/org/apache/pekko/persistence/query/PersistenceQuery.scala index 02325bdd8a5..cb6f9988819 100644 --- a/persistence-query/src/main/scala/org/apache/pekko/persistence/query/PersistenceQuery.scala +++ b/persistence-query/src/main/scala/org/apache/pekko/persistence/query/PersistenceQuery.scala @@ -37,16 +37,19 @@ object PersistenceQuery extends ExtensionId[PersistenceQuery] with ExtensionIdPr def lookup: PersistenceQuery.type = PersistenceQuery @InternalApi - private[pekko] val pluginProvider: PluginProvider[ReadJournalProvider, ReadJournal, javadsl.ReadJournal] = - new PluginProvider[ReadJournalProvider, scaladsl.ReadJournal, javadsl.ReadJournal] { + private[pekko] val pluginProvider + : PluginProvider[ReadJournalProvider, ReadJournal, pekko.persistence.query.javadsl.ReadJournal] = + new PluginProvider[ReadJournalProvider, pekko.persistence.query.scaladsl.ReadJournal, + pekko.persistence.query.javadsl.ReadJournal] { override def scalaDsl(t: ReadJournalProvider): ReadJournal = t.scaladslReadJournal() - override def javaDsl(t: ReadJournalProvider): javadsl.ReadJournal = t.javadslReadJournal() + override def javaDsl(t: ReadJournalProvider): pekko.persistence.query.javadsl.ReadJournal = t.javadslReadJournal() } } class PersistenceQuery(system: ExtendedActorSystem) - extends PersistencePlugin[scaladsl.ReadJournal, javadsl.ReadJournal, ReadJournalProvider](system)( + extends PersistencePlugin[pekko.persistence.query.scaladsl.ReadJournal, pekko.persistence.query.javadsl.ReadJournal, + ReadJournalProvider](system)( ClassTag(classOf[ReadJournalProvider]), PersistenceQuery.pluginProvider) with Extension { @@ -58,27 +61,29 @@ class PersistenceQuery(system: ExtendedActorSystem) * The provided readJournalPluginConfig will be used to configure the journal plugin instead of the actor system * config. */ - final def readJournalFor[T <: scaladsl.ReadJournal](readJournalPluginId: String, readJournalPluginConfig: Config): T = + final def readJournalFor[T <: pekko.persistence.query.scaladsl.ReadJournal](readJournalPluginId: String, + readJournalPluginConfig: Config): T = pluginFor(readJournalPluginId, readJournalPluginConfig).scaladslPlugin.asInstanceOf[T] /** * Scala API: Returns the [[pekko.persistence.query.scaladsl.ReadJournal]] specified by the given * read journal configuration entry. */ - final def readJournalFor[T <: scaladsl.ReadJournal](readJournalPluginId: String): T = + final def readJournalFor[T <: pekko.persistence.query.scaladsl.ReadJournal](readJournalPluginId: String): T = readJournalFor(readJournalPluginId, ConfigFactory.empty) /** * Java API: Returns the [[pekko.persistence.query.javadsl.ReadJournal]] specified by the given * read journal configuration entry. */ - final def getReadJournalFor[T <: javadsl.ReadJournal]( + final def getReadJournalFor[T <: pekko.persistence.query.javadsl.ReadJournal]( @unused clazz: Class[T], readJournalPluginId: String, readJournalPluginConfig: Config): T = pluginFor(readJournalPluginId, readJournalPluginConfig).javadslPlugin.asInstanceOf[T] - final def getReadJournalFor[T <: javadsl.ReadJournal](clazz: Class[T], readJournalPluginId: String): T = + final def getReadJournalFor[T <: pekko.persistence.query.javadsl.ReadJournal](clazz: Class[T], + readJournalPluginId: String): T = getReadJournalFor[T](clazz, readJournalPluginId, ConfigFactory.empty()) } diff --git a/persistence-query/src/test/scala/org/apache/pekko/persistence/query/PersistenceQuerySpec.scala b/persistence-query/src/test/scala/org/apache/pekko/persistence/query/PersistenceQuerySpec.scala index d4f3a3913a4..df70437b998 100644 --- a/persistence-query/src/test/scala/org/apache/pekko/persistence/query/PersistenceQuerySpec.scala +++ b/persistence-query/src/test/scala/org/apache/pekko/persistence/query/PersistenceQuerySpec.scala @@ -15,9 +15,6 @@ package org.apache.pekko.persistence.query import java.util.concurrent.atomic.AtomicInteger -import scala.concurrent.Await -import scala.concurrent.duration._ - import com.typesafe.config.{ Config, ConfigFactory } import org.scalatest.BeforeAndAfterAll import org.scalatest.matchers.should.Matchers @@ -99,9 +96,9 @@ class PersistenceQuerySpec extends AnyWordSpecLike with Matchers with BeforeAndA .withFallback(ConfigFactory.parseString(eventAdaptersConfig)) .withFallback(ConfigFactory.load()) - val sys = ActorSystem(s"sys-${systemCounter.incrementAndGet()}", config) + val sys = pekko.actor.scaladsl.ActorSystem(s"sys-${systemCounter.incrementAndGet()}", config) try block(sys) - finally Await.ready(sys.terminate(), 10.seconds) + finally sys.close() } } diff --git a/persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/PersistencePluginProxySpec.scala b/persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/PersistencePluginProxySpec.scala index c3fc0651e67..dc04a15f5cd 100644 --- a/persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/PersistencePluginProxySpec.scala +++ b/persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/PersistencePluginProxySpec.scala @@ -109,8 +109,9 @@ class PersistencePluginProxySpec with Cleanup { import PersistencePluginProxySpec._ - val systemA = ActorSystem("SysA", config) - val systemB = ActorSystem("SysB", targetAddressConfig(system).withFallback(PersistencePluginProxySpec.config)) + val systemA = pekko.actor.scaladsl.ActorSystem("SysA", config) + val systemB = pekko.actor.scaladsl.ActorSystem("SysB", + targetAddressConfig(system).withFallback(PersistencePluginProxySpec.config)) override protected def afterTermination(): Unit = { shutdown(systemA) diff --git a/persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala b/persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala index 149eaed5be4..9db9d3d445d 100644 --- a/persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala +++ b/persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala @@ -17,6 +17,7 @@ import scala.annotation.nowarn import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.persistence._ import pekko.testkit.{ PekkoSpec, TestProbe } diff --git a/persistence-shared/src/test/scala/org/apache/pekko/persistence/serialization/SerializerSpec.scala b/persistence-shared/src/test/scala/org/apache/pekko/persistence/serialization/SerializerSpec.scala index bbadb24f9d9..2ff94ccf770 100644 --- a/persistence-shared/src/test/scala/org/apache/pekko/persistence/serialization/SerializerSpec.scala +++ b/persistence-shared/src/test/scala/org/apache/pekko/persistence/serialization/SerializerSpec.scala @@ -16,14 +16,12 @@ package org.apache.pekko.persistence.serialization import java.io.NotSerializableException import java.util.UUID -import scala.concurrent.Await -import scala.concurrent.duration.Duration - import com.typesafe.config._ import org.apache.commons.codec.binary.Hex.{ decodeHex, encodeHex } import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.persistence._ import pekko.persistence.AtLeastOnceDelivery.{ AtLeastOnceDeliverySnapshot, UnconfirmedDelivery } import pekko.serialization._ @@ -348,7 +346,7 @@ class MessageSerializerRemotingSpec extends PekkoSpec(remote.withFallback(custom } override def afterTermination(): Unit = { - Await.ready(remoteSystem.terminate(), Duration.Inf) + remoteSystem.close() } "A message serializer" must { diff --git a/persistence-tck/src/main/scala/org/apache/pekko/persistence/journal/JournalSpec.scala b/persistence-tck/src/main/scala/org/apache/pekko/persistence/journal/JournalSpec.scala index dd6e216b317..11d736d8f8c 100644 --- a/persistence-tck/src/main/scala/org/apache/pekko/persistence/journal/JournalSpec.scala +++ b/persistence-tck/src/main/scala/org/apache/pekko/persistence/journal/JournalSpec.scala @@ -19,6 +19,7 @@ import com.typesafe.config._ import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.persistence._ import pekko.persistence.JournalProtocol._ import pekko.persistence.scalatest.{ MayVerb, OptionalTests } diff --git a/persistence-tck/src/main/scala/org/apache/pekko/persistence/snapshot/SnapshotStoreSpec.scala b/persistence-tck/src/main/scala/org/apache/pekko/persistence/snapshot/SnapshotStoreSpec.scala index 9f985ee505e..29cc07e93a4 100644 --- a/persistence-tck/src/main/scala/org/apache/pekko/persistence/snapshot/SnapshotStoreSpec.scala +++ b/persistence-tck/src/main/scala/org/apache/pekko/persistence/snapshot/SnapshotStoreSpec.scala @@ -20,6 +20,7 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.persistence._ import pekko.persistence.SnapshotProtocol._ import pekko.persistence.scalatest.{ MayVerb, OptionalTests } diff --git a/persistence-testkit/src/test/scala/org/apache/pekko/persistence/testkit/CommonUtils.scala b/persistence-testkit/src/test/scala/org/apache/pekko/persistence/testkit/CommonUtils.scala index 7be70a96f35..2c4b0bea192 100644 --- a/persistence-testkit/src/test/scala/org/apache/pekko/persistence/testkit/CommonUtils.scala +++ b/persistence-testkit/src/test/scala/org/apache/pekko/persistence/testkit/CommonUtils.scala @@ -19,7 +19,8 @@ import com.typesafe.config.ConfigFactory import org.scalatest.wordspec.AnyWordSpecLike import org.apache.pekko -import pekko.actor.{ ActorRef, ActorSystem } +import pekko.actor.ActorRef +import pekko.actor.scaladsl.ActorSystem import pekko.persistence._ import pekko.testkit.TestKitBase diff --git a/persistence-typed/src/test/scala/docs/org/apache/pekko/persistence/typed/PersistentFsmToTypedMigrationSpec.scala b/persistence-typed/src/test/scala/docs/org/apache/pekko/persistence/typed/PersistentFsmToTypedMigrationSpec.scala index 3a6e8443796..0d857396013 100644 --- a/persistence-typed/src/test/scala/docs/org/apache/pekko/persistence/typed/PersistentFsmToTypedMigrationSpec.scala +++ b/persistence-typed/src/test/scala/docs/org/apache/pekko/persistence/typed/PersistentFsmToTypedMigrationSpec.scala @@ -219,7 +219,8 @@ class PersistentFsmToTypedMigrationSpec extends AnyWordSpec with ScalaFutures wi "PersistentFSM migration to Persistence Typed" must { "work when snapshot is not current" in { - val classicActorSystem = pekko.actor.ActorSystem("ClassicSystem", PersistentFsmToTypedMigrationSpec.config) + val classicActorSystem = + pekko.actor.scaladsl.ActorSystem("ClassicSystem", PersistentFsmToTypedMigrationSpec.config) val shirt = Item("1", "Shirt", 59.99f) val shoes = Item("2", "Shoes", 89.99f) val coat = Item("3", "Coat", 119.99f) @@ -239,7 +240,7 @@ class PersistentFsmToTypedMigrationSpec extends AnyWordSpec with ScalaFutures wi fsmRef ! PoisonPill classicProbe.expectTerminated(fsmRef) } finally { - classicActorSystem.terminate().futureValue + classicActorSystem.close() } val typedTestKit = ActorTestKit("System", PersistentFsmToTypedMigrationSpec.config) @@ -260,7 +261,8 @@ class PersistentFsmToTypedMigrationSpec extends AnyWordSpec with ScalaFutures wi } "work if snapshot is current" in { - val classicActorSystem = pekko.actor.ActorSystem("CLassicSystem", PersistentFsmToTypedMigrationSpec.config) + val classicActorSystem = + pekko.actor.scaladsl.ActorSystem("CLassicSystem", PersistentFsmToTypedMigrationSpec.config) val shirt = Item("1", "Shirt", 59.99f) val pid = "current-shapshot" try { @@ -277,7 +279,7 @@ class PersistentFsmToTypedMigrationSpec extends AnyWordSpec with ScalaFutures wi fsmRef.tell(GetCurrentCart, classicProbe.ref) classicProbe.expectMsg(NonEmptyShoppingCart(Seq(shirt))) } finally { - classicActorSystem.terminate().futureValue + classicActorSystem.close() } val typedTestKit = ActorTestKit("TypedSystem", PersistentFsmToTypedMigrationSpec.config) diff --git a/persistence/src/test/scala/org/apache/pekko/persistence/EndToEndEventAdapterSpec.scala b/persistence/src/test/scala/org/apache/pekko/persistence/EndToEndEventAdapterSpec.scala index 0babafe0526..863217581f3 100644 --- a/persistence/src/test/scala/org/apache/pekko/persistence/EndToEndEventAdapterSpec.scala +++ b/persistence/src/test/scala/org/apache/pekko/persistence/EndToEndEventAdapterSpec.scala @@ -15,9 +15,6 @@ package org.apache.pekko.persistence import java.io.File -import scala.concurrent.Await -import scala.concurrent.duration._ - import com.typesafe.config.{ Config, ConfigFactory } import org.apache.commons.io.FileUtils import org.scalatest.BeforeAndAfterAll @@ -26,6 +23,7 @@ import org.scalatest.wordspec.AnyWordSpecLike import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.persistence.journal.{ EventAdapter, EventSeq } import pekko.testkit.TestProbe import pekko.util.unused @@ -184,7 +182,7 @@ class EndToEndEventAdapterSpec extends AnyWordSpecLike with Matchers with Before def withActorSystem[T](name: String, config: Config)(block: ActorSystem => T): T = { val system = ActorSystem(name, journalConfig.withFallback(config)) try block(system) - finally Await.ready(system.terminate(), 3.seconds) + finally system.close() } "EventAdapters in end-to-end scenarios" must { diff --git a/persistence/src/test/scala/org/apache/pekko/persistence/serialization/SnapshotSerializerMigrationAkkaSpec.scala b/persistence/src/test/scala/org/apache/pekko/persistence/serialization/SnapshotSerializerMigrationAkkaSpec.scala index 952df9ed9c0..6701fe0ac09 100644 --- a/persistence/src/test/scala/org/apache/pekko/persistence/serialization/SnapshotSerializerMigrationAkkaSpec.scala +++ b/persistence/src/test/scala/org/apache/pekko/persistence/serialization/SnapshotSerializerMigrationAkkaSpec.scala @@ -19,7 +19,7 @@ package org.apache.pekko.persistence.serialization import com.typesafe.config.ConfigFactory import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.persistence.fsm.PersistentFSM.PersistentFSMSnapshot import pekko.serialization.SerializationExtension import pekko.testkit.PekkoSpec @@ -64,7 +64,7 @@ class SnapshotSerializerMigrationAkkaSpec extends PekkoSpec( pekkoOnlySerialization.deserialize(bytes, classOf[Snapshot]).get } } finally { - pekkoOnlySystem.terminate() + pekkoOnlySystem.close() } } } diff --git a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteDeploymentDeathWatchSpec.scala b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteDeploymentDeathWatchSpec.scala index 5d1a4dcc4c7..3f6283226dc 100644 --- a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteDeploymentDeathWatchSpec.scala +++ b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteDeploymentDeathWatchSpec.scala @@ -104,7 +104,7 @@ abstract class RemoteDeploymentDeathWatchSpec(multiNodeConfig: RemoteDeploymentD // if the remote deployed actor is not removed the system will not shutdown val timeout = remainingOrDefault - try Await.ready(system.whenTerminated, timeout) + try Await.ready(system.whenTerminatedImpl, timeout) catch { case _: TimeoutException => fail( diff --git a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteNodeRestartDeathWatchSpec.scala b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteNodeRestartDeathWatchSpec.scala index c556ed8df62..fba832c421c 100644 --- a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteNodeRestartDeathWatchSpec.scala +++ b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteNodeRestartDeathWatchSpec.scala @@ -13,7 +13,6 @@ package org.apache.pekko.remote -import scala.concurrent.Await import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory @@ -22,11 +21,11 @@ import org.apache.pekko import pekko.actor.Actor import pekko.actor.ActorIdentity import pekko.actor.ActorRef -import pekko.actor.ActorSystem import pekko.actor.ExtendedActorSystem import pekko.actor.Identify import pekko.actor.Props import pekko.actor.RootActorPath +import pekko.actor.scaladsl.ActorSystem import pekko.remote.testconductor.RoleName import pekko.remote.testkit.MultiNodeConfig import pekko.remote.transport.ThrottlerTransportAdapter.Direction @@ -65,7 +64,7 @@ object RemoteNodeRestartDeathWatchSpec { def receive = { case "shutdown" => sender() ! "shutdown-ack" - context.system.terminate() + context.system.closeAsync() case msg => sender() ! msg } } @@ -118,7 +117,7 @@ abstract class RemoteNodeRestartDeathWatchSpec(multiNodeConfig: RemoteNodeRestar enterBarrier("watch-established") - Await.ready(system.whenTerminated, 30.seconds) + system.close() val freshSystem = ActorSystem( system.name, @@ -128,7 +127,7 @@ abstract class RemoteNodeRestartDeathWatchSpec(multiNodeConfig: RemoteNodeRestar """).withFallback(system.settings.config)) freshSystem.actorOf(Props[Subject](), "subject") - Await.ready(freshSystem.whenTerminated, 30.seconds) + system.close() } } diff --git a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteQuarantinePiercingSpec.scala b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteQuarantinePiercingSpec.scala index a33b8f9e1aa..b2389d5d42d 100644 --- a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteQuarantinePiercingSpec.scala +++ b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteQuarantinePiercingSpec.scala @@ -20,6 +20,7 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.remote.testconductor.RoleName import pekko.remote.testkit.MultiNodeConfig import pekko.testkit._ @@ -52,7 +53,7 @@ class ArteryRemoteQuarantinePiercingMultiJvmNode2 object RemoteQuarantinePiercingSpec { class Subject extends Actor { def receive = { - case "shutdown" => context.system.terminate() + case "shutdown" => context.system.closeAsync() case "identify" => sender() ! (AddressUidExtension(context.system).longAddressUid -> self) } } @@ -120,7 +121,7 @@ abstract class RemoteQuarantinePiercingSpec(multiNodeConfig: RemoteQuarantinePie enterBarrier("actor-identified") - Await.ready(system.whenTerminated, 30.seconds) + system.close() val freshSystem = ActorSystem( system.name, @@ -130,7 +131,7 @@ abstract class RemoteQuarantinePiercingSpec(multiNodeConfig: RemoteQuarantinePie """).withFallback(system.settings.config)) freshSystem.actorOf(Props[Subject](), "subject") - Await.ready(freshSystem.whenTerminated, 30.seconds) + freshSystem.close() } } diff --git a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteReDeploymentSpec.scala b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteReDeploymentSpec.scala index e08b6535dc7..9cbc4fd1844 100644 --- a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteReDeploymentSpec.scala +++ b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/RemoteReDeploymentSpec.scala @@ -13,7 +13,6 @@ package org.apache.pekko.remote -import scala.concurrent.Await import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory @@ -172,7 +171,7 @@ abstract class RemoteReDeploymentMultiJvmSpec(multiNodeConfig: RemoteReDeploymen // Start the second system again runOn(second) { - Await.ready(system.whenTerminated, 30.seconds) + system.close() expectNoMessage(sleepAfterKill) sys = startNewSystem() } @@ -224,7 +223,7 @@ abstract class RemoteReDeploymentMultiJvmSpec(multiNodeConfig: RemoteReDeploymen enterBarrier("stopping") runOn(second) { - Await.result(sys.terminate(), 10.seconds) + sys.close() } } diff --git a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/artery/HandshakeRestartReceiverSpec.scala b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/artery/HandshakeRestartReceiverSpec.scala index 747e7c2ad37..e113390ed28 100644 --- a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/artery/HandshakeRestartReceiverSpec.scala +++ b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/artery/HandshakeRestartReceiverSpec.scala @@ -49,7 +49,7 @@ object HandshakeRestartReceiverSpec extends MultiNodeConfig { class Subject extends Actor { def receive = { - case "shutdown" => context.system.terminate() + case "shutdown" => context.system.closeAsync() case "identify" => sender() ! (AddressUidExtension(context.system).longAddressUid -> self) } } @@ -130,14 +130,14 @@ abstract class HandshakeRestartReceiverSpec Await.result(system.whenTerminated, 10.seconds) - val freshSystem = ActorSystem( + val freshSystem = pekko.actor.scaladsl.ActorSystem( system.name, ConfigFactory.parseString(s""" pekko.remote.artery.canonical.port = ${address.port.get} """).withFallback(system.settings.config)) freshSystem.actorOf(Props[Subject](), "subject2") - Await.result(freshSystem.whenTerminated, 45.seconds) + freshSystem.close() } } diff --git a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/artery/RemoteRestartedQuarantinedSpec.scala b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/artery/RemoteRestartedQuarantinedSpec.scala index dffea04a02a..c7117564b81 100644 --- a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/artery/RemoteRestartedQuarantinedSpec.scala +++ b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/artery/RemoteRestartedQuarantinedSpec.scala @@ -20,6 +20,7 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.{ ActorIdentity, Identify, _ } +import pekko.actor.scaladsl.ActorSystem import pekko.remote.{ AddressUidExtension, RARP, RemotingMultiNodeSpec } import pekko.remote.testconductor.RoleName import pekko.remote.testkit.MultiNodeConfig @@ -43,7 +44,7 @@ object RemoteRestartedQuarantinedSpec extends MultiNodeConfig { class Subject extends Actor { def receive = { - case "shutdown" => context.system.terminate() + case "shutdown" => context.system.close() case "identify" => sender() ! (AddressUidExtension(context.system).longAddressUid -> self) } } @@ -122,7 +123,7 @@ abstract class RemoteRestartedQuarantinedSpec extends RemotingMultiNodeSpec(Remo enterBarrier("still-quarantined") - Await.result(system.whenTerminated, 10.seconds) + system.close() val freshSystem = ActorSystem( system.name, diff --git a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/classic/RemoteGatePiercingSpec.scala b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/classic/RemoteGatePiercingSpec.scala index c015ed19b6f..aff56893199 100644 --- a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/classic/RemoteGatePiercingSpec.scala +++ b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/classic/RemoteGatePiercingSpec.scala @@ -49,7 +49,7 @@ object RemoteGatePiercingSpec extends MultiNodeConfig { class Subject extends Actor { def receive = { - case "shutdown" => context.system.terminate() + case "shutdown" => context.system.closeAsync() } } diff --git a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/classic/RemoteNodeRestartGateSpec.scala b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/classic/RemoteNodeRestartGateSpec.scala index bbd4f747984..4d1ba95bc0f 100644 --- a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/classic/RemoteNodeRestartGateSpec.scala +++ b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/classic/RemoteNodeRestartGateSpec.scala @@ -106,7 +106,7 @@ abstract class RemoteNodeRestartGateSpec extends RemotingMultiNodeSpec(RemoteNod enterBarrier("gated") - Await.ready(system.whenTerminated, 10.seconds) + system.close() val freshSystem = ActorSystem( system.name, @@ -133,7 +133,7 @@ abstract class RemoteNodeRestartGateSpec extends RemotingMultiNodeSpec(RemoteNod // Now the other system will be able to pass, too freshSystem.actorOf(Props[Subject](), "subject") - Await.ready(freshSystem.whenTerminated, 30.seconds) + freshSystem.close() } } diff --git a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/classic/RemoteNodeShutdownAndComesBackSpec.scala b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/classic/RemoteNodeShutdownAndComesBackSpec.scala index 04a69451249..5d8777857d9 100644 --- a/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/classic/RemoteNodeShutdownAndComesBackSpec.scala +++ b/remote-tests/src/multi-jvm/scala/org/apache/pekko/remote/classic/RemoteNodeShutdownAndComesBackSpec.scala @@ -20,6 +20,7 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.{ ActorIdentity, Identify, _ } +import pekko.actor.scaladsl.ActorSystem import pekko.remote.{ RARP, RemotingMultiNodeSpec } import pekko.remote.testconductor.RoleName import pekko.remote.testkit.MultiNodeConfig @@ -47,7 +48,7 @@ object RemoteNodeShutdownAndComesBackSpec extends MultiNodeConfig { class Subject extends Actor { def receive = { - case "shutdown" => context.system.terminate() + case "shutdown" => context.system.closeAsync() case msg => sender() ! msg } } @@ -143,7 +144,7 @@ abstract class RemoteNodeShutdownAndComesBackSpec extends RemotingMultiNodeSpec( enterBarrier("watch-established") - Await.ready(system.whenTerminated, 30.seconds) + system.close() val freshSystem = ActorSystem( system.name, @@ -153,7 +154,7 @@ abstract class RemoteNodeShutdownAndComesBackSpec extends RemotingMultiNodeSpec( """).withFallback(system.settings.config)) freshSystem.actorOf(Props[Subject](), "subject") - Await.ready(freshSystem.whenTerminated, 30.seconds) + freshSystem.close() } } diff --git a/remote-tests/src/test/scala/org/apache/pekko/remote/artery/ArteryFailedToBindSpec.scala b/remote-tests/src/test/scala/org/apache/pekko/remote/artery/ArteryFailedToBindSpec.scala index 19fbe1bfd64..138b9e5061b 100644 --- a/remote-tests/src/test/scala/org/apache/pekko/remote/artery/ArteryFailedToBindSpec.scala +++ b/remote-tests/src/test/scala/org/apache/pekko/remote/artery/ArteryFailedToBindSpec.scala @@ -18,7 +18,7 @@ import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.remote.RARP import pekko.remote.RemoteTransportException import pekko.testkit.SocketUtil diff --git a/remote-tests/src/test/scala/org/apache/pekko/remote/classic/RemotingFailedToBindSpec.scala b/remote-tests/src/test/scala/org/apache/pekko/remote/classic/RemotingFailedToBindSpec.scala index b610c708e9a..d5b67b5eb19 100644 --- a/remote-tests/src/test/scala/org/apache/pekko/remote/classic/RemotingFailedToBindSpec.scala +++ b/remote-tests/src/test/scala/org/apache/pekko/remote/classic/RemotingFailedToBindSpec.scala @@ -17,7 +17,7 @@ import com.typesafe.config.ConfigFactory import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.testkit.SocketUtil import java.net.BindException @@ -48,7 +48,7 @@ class RemotingFailedToBindSpec extends AnyWordSpec with Matchers { } ex.getMessage should startWith("Address already in use") } finally { - as.terminate() + as.close() } } } diff --git a/remote/src/main/scala/org/apache/pekko/remote/artery/ArteryTransport.scala b/remote/src/main/scala/org/apache/pekko/remote/artery/ArteryTransport.scala index ae7b3a091e9..22536c79ff9 100644 --- a/remote/src/main/scala/org/apache/pekko/remote/artery/ArteryTransport.scala +++ b/remote/src/main/scala/org/apache/pekko/remote/artery/ArteryTransport.scala @@ -615,7 +615,7 @@ private[remote] abstract class ArteryTransport(_system: ExtendedActorSystem, _pr settings.Advanced.InboundMaxRestarts, settings.Advanced.InboundRestartTimeout.toSeconds, cause.getMessage) - system.terminate() + system.terminateImpl() } } } diff --git a/remote/src/main/scala/org/apache/pekko/remote/artery/Association.scala b/remote/src/main/scala/org/apache/pekko/remote/artery/Association.scala index 7da5370d0c5..2c0e997d3f1 100644 --- a/remote/src/main/scala/org/apache/pekko/remote/artery/Association.scala +++ b/remote/src/main/scala/org/apache/pekko/remote/artery/Association.scala @@ -1057,7 +1057,7 @@ private[remote] class Association( advancedSettings.OutboundMaxRestarts, advancedSettings.OutboundRestartTimeout.toSeconds) cancelAllTimers() - transport.system.terminate() + transport.system.terminateImpl() } } } diff --git a/remote/src/main/scala/org/apache/pekko/remote/artery/aeron/ArteryAeronUdpTransport.scala b/remote/src/main/scala/org/apache/pekko/remote/artery/aeron/ArteryAeronUdpTransport.scala index 4ebd910c401..54e603b127c 100644 --- a/remote/src/main/scala/org/apache/pekko/remote/artery/aeron/ArteryAeronUdpTransport.scala +++ b/remote/src/main/scala/org/apache/pekko/remote/artery/aeron/ArteryAeronUdpTransport.scala @@ -235,7 +235,7 @@ private[remote] class ArteryAeronUdpTransport(_system: ExtendedActorSystem, _pro taskRunner.stop() aeronErrorLogTask.cancel() if (settings.Advanced.Aeron.LogAeronCounters) aeronCounterTask.cancel() - system.terminate() + system.terminateImpl() throw new AeronTerminated(cause) } } else diff --git a/remote/src/test/scala/org/apache/pekko/remote/DaemonicSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/DaemonicSpec.scala index e69877cffa4..85d6451b00a 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/DaemonicSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/DaemonicSpec.scala @@ -18,7 +18,8 @@ import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory import org.apache.pekko -import pekko.actor.{ ActorSystem, Address } +import pekko.actor.Address +import pekko.actor.scaladsl.ActorSystem import pekko.testkit._ import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ diff --git a/remote/src/test/scala/org/apache/pekko/remote/MessageLoggingSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/MessageLoggingSpec.scala index 462d7aebc19..e167b0f63ae 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/MessageLoggingSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/MessageLoggingSpec.scala @@ -17,7 +17,8 @@ import MessageLoggingSpec._ import com.typesafe.config.{ Config, ConfigFactory } import org.apache.pekko -import pekko.actor.{ Actor, ActorIdentity, ActorSystem, ExtendedActorSystem, Identify, Props, RootActorPath } +import pekko.actor.{ Actor, ActorIdentity, ExtendedActorSystem, Identify, Props, RootActorPath } +import pekko.actor.scaladsl.ActorSystem import pekko.serialization.jackson.CborSerializable import pekko.testkit.EventFilter import pekko.testkit.TestActors diff --git a/remote/src/test/scala/org/apache/pekko/remote/RemoteRouterSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/RemoteRouterSpec.scala index a5e457e1d6b..1d5041cc052 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/RemoteRouterSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/RemoteRouterSpec.scala @@ -19,6 +19,7 @@ import com.typesafe.config._ import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.remote.routing._ import pekko.routing._ import pekko.testkit._ diff --git a/remote/src/test/scala/org/apache/pekko/remote/TransientSerializationErrorSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/TransientSerializationErrorSpec.scala index 001906e21f7..6707c55aad9 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/TransientSerializationErrorSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/TransientSerializationErrorSpec.scala @@ -18,7 +18,8 @@ import java.io.NotSerializableException import com.typesafe.config.{ Config, ConfigFactory } import org.apache.pekko -import pekko.actor.{ ActorSystem, ExtendedActorSystem, RootActorPath } +import pekko.actor.{ ExtendedActorSystem, RootActorPath } +import pekko.actor.scaladsl.ActorSystem import pekko.serialization.SerializerWithStringManifest import pekko.testkit.{ PekkoSpec, TestActors, TestKit } import pekko.util.unused diff --git a/remote/src/test/scala/org/apache/pekko/remote/TypedActorRemoteDeploySpec.scala b/remote/src/test/scala/org/apache/pekko/remote/TypedActorRemoteDeploySpec.scala index 6a27d632dce..47d9485e19e 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/TypedActorRemoteDeploySpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/TypedActorRemoteDeploySpec.scala @@ -21,7 +21,8 @@ import scala.annotation.nowarn import com.typesafe.config._ import org.apache.pekko -import pekko.actor.{ ActorSystem, Deploy, TypedActor, TypedProps } +import pekko.actor.{ Deploy, TypedActor, TypedProps } +import pekko.actor.scaladsl.ActorSystem import pekko.testkit.PekkoSpec object TypedActorRemoteDeploySpec { diff --git a/remote/src/test/scala/org/apache/pekko/remote/artery/ArteryMultiNodeSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/artery/ArteryMultiNodeSpec.scala index 1a732a80e69..6388674990a 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/artery/ArteryMultiNodeSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/artery/ArteryMultiNodeSpec.scala @@ -65,15 +65,15 @@ abstract class ArteryMultiNodeSpec(config: Config) def newRemoteSystem( extraConfig: Option[String] = None, name: Option[String] = None, - setup: Option[ActorSystemSetup] = None): ActorSystem = { + setup: Option[ActorSystemSetup] = None): pekko.actor.scaladsl.ActorSystem = { val config = extraConfig.fold(localSystem.settings.config)(str => ConfigFactory.parseString(str).withFallback(localSystem.settings.config)) val sysName = name.getOrElse(nextGeneratedSystemName) val remoteSystem = setup match { - case None => ActorSystem(sysName, config) - case Some(s) => ActorSystem(sysName, s.and(BootstrapSetup.apply(config))) + case None => pekko.actor.scaladsl.ActorSystem(sysName, config) + case Some(s) => pekko.actor.scaladsl.ActorSystem(sysName, s.and(BootstrapSetup.apply(config))) } remoteSystems = remoteSystems :+ remoteSystem diff --git a/remote/src/test/scala/org/apache/pekko/remote/artery/BindCanonicalAddressSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/artery/BindCanonicalAddressSpec.scala index 58a6c222b7d..85c0979a5d3 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/artery/BindCanonicalAddressSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/artery/BindCanonicalAddressSpec.scala @@ -15,15 +15,13 @@ package org.apache.pekko.remote.artery import java.net.InetAddress -import scala.concurrent.Await -import scala.concurrent.duration.Duration - import com.typesafe.config.ConfigFactory import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec import org.apache.pekko -import pekko.actor.{ ActorSystem, Address } +import pekko.actor.Address +import pekko.actor.scaladsl.ActorSystem import pekko.remote.transport.NettyTransportSpec._ import pekko.testkit.SocketUtil @@ -41,7 +39,7 @@ trait BindCanonicalAddressBehaviors { implicit val sys = ActorSystem("sys", config.withFallback(commonConfig)) getInternal() should contain(getExternal()) - Await.result(sys.terminate(), Duration.Inf) + sys.close() } "bind to a random port but remoting accepts from a specified port" in { @@ -60,13 +58,13 @@ trait BindCanonicalAddressBehaviors { if (getInternal().collect { case Address(_, _, _, Some(port)) => port }.toSeq.contains(address.getPort)) { val sys2 = ActorSystem("sys", config.withFallback(commonConfig)) val secondInternals = getInternal()(sys2) - Await.result(sys2.terminate(), Duration.Inf) + sys2.close() secondInternals } else { getInternal() } internals should not contain address.toAkkaAddress("pekko") - Await.result(sys.terminate(), Duration.Inf) + sys.close() } "bind to a specified bind hostname and remoting aspects from canonical hostname" in { @@ -106,7 +104,7 @@ trait BindCanonicalAddressBehaviors { getInternal().flatMap(_.port) should contain(getExternal().port.get) getInternal().map(x => (x.host.get should include).regex("0.0.0.0".r)) // regexp dot is intentional to match IPv4 and 6 addresses - Await.result(sys.terminate(), Duration.Inf) + sys.close() } } } diff --git a/remote/src/test/scala/org/apache/pekko/remote/artery/FlushOnShutdownSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/artery/FlushOnShutdownSpec.scala index 59ae026d48f..3a339fe3535 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/artery/FlushOnShutdownSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/artery/FlushOnShutdownSpec.scala @@ -46,7 +46,7 @@ class FlushOnShutdownSpec extends ArteryMultiNodeSpec(ArterySpecSupport.defaultC receiverRef ! "msg1" receiverRef ! "msg2" receiverRef ! "msg3" - context.system.terminate() + context.system.closeAsync() } }), "sender") diff --git a/remote/src/test/scala/org/apache/pekko/remote/artery/RemoteRouterSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/artery/RemoteRouterSpec.scala index f1fd7986a1e..4f60d42da5d 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/artery/RemoteRouterSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/artery/RemoteRouterSpec.scala @@ -19,6 +19,7 @@ import com.typesafe.config._ import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.remote.{ RARP, RemoteScope } import pekko.remote.routing._ import pekko.routing._ diff --git a/remote/src/test/scala/org/apache/pekko/remote/artery/SystemMessageDeliverySpec.scala b/remote/src/test/scala/org/apache/pekko/remote/artery/SystemMessageDeliverySpec.scala index 26c2f1709d1..6857f554d6a 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/artery/SystemMessageDeliverySpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/artery/SystemMessageDeliverySpec.scala @@ -24,9 +24,9 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.NotUsed import pekko.actor.ActorIdentity -import pekko.actor.ActorSystem import pekko.actor.Identify import pekko.actor.RootActorPath +import pekko.actor.scaladsl.ActorSystem import pekko.remote.AddressUidExtension import pekko.remote.RARP import pekko.remote.UniqueAddress @@ -184,7 +184,7 @@ class SystemMessageDeliverySpec extends AbstractSystemMessageDeliverySpec(System watch(remoteRef) remoteRef ! "hello" expectMsg("hello") - Await.ready(systemC.terminate(), 10.seconds) + systemC.close() system.log.debug("systemC terminated") // DeathWatchNotification is sent from systemC, failure detection takes longer than 3 seconds expectTerminated(remoteRef, 10.seconds) diff --git a/remote/src/test/scala/org/apache/pekko/remote/artery/tcp/TlsTcpSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/artery/tcp/TlsTcpSpec.scala index 8de36833414..12277eaa35d 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/artery/tcp/TlsTcpSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/artery/tcp/TlsTcpSpec.scala @@ -195,7 +195,7 @@ class TlsTcpWithHostnameVerificationSpec system.actorSelection(rootB / "user" / "echo") ! Identify("echo") } expectNoMessage(2.seconds) - systemB.terminate() + systemB.terminateAsync() } "succeed when the name in the server certificate matches" in { if (!arteryTcpTlsEnabled()) @@ -221,7 +221,7 @@ class TlsTcpWithHostnameVerificationSpec id.ref.get ! "42" expectMsg("42") - systemB.terminate() + systemB.terminateAsync() } } } diff --git a/remote/src/test/scala/org/apache/pekko/remote/artery/tcp/ssl/RotatingKeysSSLEngineProviderSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/artery/tcp/ssl/RotatingKeysSSLEngineProviderSpec.scala index 0c3b2680a98..60047c2e810 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/artery/tcp/ssl/RotatingKeysSSLEngineProviderSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/artery/tcp/ssl/RotatingKeysSSLEngineProviderSpec.scala @@ -280,7 +280,7 @@ abstract class RotatingKeysSSLEngineProviderSpec(extraConfig: String) override def beforeTermination(): Unit = { systemsToTerminate.foreach { systemToTerminate => system.log.info(s"Terminating $systemToTerminate...") - systemToTerminate.terminate() + systemToTerminate.terminateImpl() } systemsToTerminate.foreach(shutdown(_, verifySystemShutdown = true)) super.beforeTermination() diff --git a/remote/src/test/scala/org/apache/pekko/remote/classic/ActorsLeakSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/classic/ActorsLeakSpec.scala index 2a9dd698d05..aff86b43c51 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/classic/ActorsLeakSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/classic/ActorsLeakSpec.scala @@ -22,6 +22,7 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.actor.dungeon.ChildrenContainer import pekko.remote.{ AddressUidExtension, RARP } import pekko.remote.transport.ThrottlerTransportAdapter.ForceDisassociate @@ -111,10 +112,8 @@ class ActorsLeakSpec extends PekkoSpec(ActorsLeakSpec.config) with ImplicitSende probe.expectMsgType[ActorIdentity].ref.nonEmpty should be(true) } finally { - remoteSystem.terminate() + remoteSystem.close() } - - Await.result(remoteSystem.whenTerminated, 10.seconds) } // Quarantine an old incarnation case @@ -154,11 +153,9 @@ class ActorsLeakSpec extends PekkoSpec(ActorsLeakSpec.config) with ImplicitSende assertResult(beforeQuarantineActors)(afterQuarantineActors) } finally { - remoteSystem.terminate() + remoteSystem.close() } - Await.result(remoteSystem.whenTerminated, 10.seconds) - } // Missing SHUTDOWN case @@ -183,13 +180,13 @@ class ActorsLeakSpec extends PekkoSpec(ActorsLeakSpec.config) with ImplicitSende Await.result(RARP(system).provider.transport.managementCommand(ForceDisassociate(remoteAddress)), 3.seconds) } finally { - remoteSystem.terminate() + remoteSystem.terminateAsync() } EventFilter .warning(start = s"Association with remote system [$remoteAddress] has failed", occurrences = 1) .intercept { - Await.result(remoteSystem.whenTerminated, 10.seconds) + remoteSystem.close() } } @@ -218,10 +215,9 @@ class ActorsLeakSpec extends PekkoSpec(ActorsLeakSpec.config) with ImplicitSende Await.result(RARP(system).provider.transport.managementCommand(ForceDisassociate(remoteAddress)), 3.seconds) } finally { - remoteSystem.terminate() + remoteSystem.close() } - Await.result(remoteSystem.whenTerminated, 10.seconds) awaitAssert(assertResult(initialActors)(targets.flatMap(collectLiveActors).toSet), 10.seconds) } diff --git a/remote/src/test/scala/org/apache/pekko/remote/classic/RemoteInitErrorSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/classic/RemoteInitErrorSpec.scala index 09bb9e7cdf6..b6d27745f67 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/classic/RemoteInitErrorSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/classic/RemoteInitErrorSpec.scala @@ -23,7 +23,7 @@ import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.util.ccompat.JavaConverters._ /** diff --git a/remote/src/test/scala/org/apache/pekko/remote/classic/UntrustedSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/classic/UntrustedSpec.scala index c4c74213a79..a15547915ca 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/classic/UntrustedSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/classic/UntrustedSpec.scala @@ -22,7 +22,6 @@ import pekko.actor.Actor import pekko.actor.ActorIdentity import pekko.actor.ActorRef import pekko.actor.ActorSelection -import pekko.actor.ActorSystem import pekko.actor.Deploy import pekko.actor.ExtendedActorSystem import pekko.actor.Identify @@ -30,6 +29,7 @@ import pekko.actor.PoisonPill import pekko.actor.Props import pekko.actor.RootActorPath import pekko.actor.Terminated +import pekko.actor.scaladsl.ActorSystem import pekko.event.Logging import pekko.testkit.PekkoSpec import pekko.testkit.EventFilter diff --git a/remote/src/test/scala/org/apache/pekko/remote/classic/transport/PekkoProtocolStressTest.scala b/remote/src/test/scala/org/apache/pekko/remote/classic/transport/PekkoProtocolStressTest.scala index 9e0bf6bceaa..e54a9b4263e 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/classic/transport/PekkoProtocolStressTest.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/classic/transport/PekkoProtocolStressTest.scala @@ -20,6 +20,7 @@ import com.typesafe.config.{ Config, ConfigFactory } import org.apache.pekko import pekko.actor._ +import pekko.actor.scaladsl.ActorSystem import pekko.remote.{ EndpointException, RARP } import pekko.remote.classic.transport.PekkoProtocolStressTest._ import pekko.remote.transport.FailureInjectorTransportAdapter.{ Drop, One } diff --git a/remote/src/test/scala/org/apache/pekko/remote/serialization/AllowJavaSerializationOffSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/serialization/AllowJavaSerializationOffSpec.scala index 8b553c63808..718e9554a47 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/serialization/AllowJavaSerializationOffSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/serialization/AllowJavaSerializationOffSpec.scala @@ -56,7 +56,7 @@ object AllowJavaSerializationOffSpec { None) val actorSystemSettings = ActorSystemSetup(bootstrapSettings, serializationSettings) - val noJavaSerializationSystem = ActorSystem( + val noJavaSerializationSystem = pekko.actor.scaladsl.ActorSystem( "AllowJavaSerializationOffSpec" + "NoJavaSerialization", ConfigFactory.parseString(""" pekko { @@ -72,7 +72,8 @@ object AllowJavaSerializationOffSpec { } class AllowJavaSerializationOffSpec - extends PekkoSpec(ActorSystem("AllowJavaSerializationOffSpec", AllowJavaSerializationOffSpec.actorSystemSettings)) { + extends PekkoSpec(pekko.actor.scaladsl.ActorSystem("AllowJavaSerializationOffSpec", + AllowJavaSerializationOffSpec.actorSystemSettings)) { import AllowJavaSerializationOffSpec._ @@ -103,7 +104,7 @@ class AllowJavaSerializationOffSpec None) val dontAllowJavaSystem = - ActorSystem( + pekko.actor.scaladsl.ActorSystem( "addedJavaSerializationSystem", ActorSystemSetup(addedJavaSerializationProgramaticallyButDisabledSettings, addedJavaSerializationSettings)) diff --git a/remote/src/test/scala/org/apache/pekko/remote/serialization/SerializationTransportInformationSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/serialization/SerializationTransportInformationSpec.scala index 946f87abcea..6a43635a505 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/serialization/SerializationTransportInformationSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/serialization/SerializationTransportInformationSpec.scala @@ -19,7 +19,6 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.ActorIdentity import pekko.actor.ActorRef -import pekko.actor.ActorSystem import pekko.actor.ExtendedActorSystem import pekko.actor.Identify import pekko.actor.RootActorPath @@ -108,7 +107,7 @@ abstract class AbstractSerializationTransportInformationSpec(config: Config) if (RARP(system).provider.remoteSettings.Artery.Enabled) "pekko" else "pekko.tcp" - val system2 = ActorSystem(system.name, system.settings.config) + val system2 = pekko.actor.scaladsl.ActorSystem(system.name, system.settings.config) val system2Address = RARP(system2).provider.getDefaultAddress "Serialization of ActorRef in remote message" must { diff --git a/remote/src/test/scala/org/apache/pekko/remote/transport/NettyTransportSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/transport/NettyTransportSpec.scala index 436994de582..938326a60ff 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/transport/NettyTransportSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/transport/NettyTransportSpec.scala @@ -27,8 +27,6 @@ import org.scalatest.wordspec.AnyWordSpec import java.net.{ InetAddress, InetSocketAddress } import java.nio.channels.ServerSocketChannel -import scala.concurrent.Await -import scala.concurrent.duration.Duration object NettyTransportSpec { val commonConfig = ConfigFactory.parseString(""" @@ -65,11 +63,11 @@ class NettyTransportSpec extends AnyWordSpec with Matchers with BindBehavior { port = 0 } """) - implicit val sys = ActorSystem("sys", bindConfig.withFallback(commonConfig)) + implicit val sys = pekko.actor.scaladsl.ActorSystem("sys", bindConfig.withFallback(commonConfig)) getInternal() should contain(getExternal().withProtocol("tcp")) - Await.result(sys.terminate(), Duration.Inf) + sys.close() } "bind to a random port but remoting accepts from a specified port" in { @@ -84,12 +82,12 @@ class NettyTransportSpec extends AnyWordSpec with Matchers with BindBehavior { bind-port = 0 } """) - implicit val sys = ActorSystem("sys", bindConfig.withFallback(commonConfig)) + implicit val sys = pekko.actor.scaladsl.ActorSystem("sys", bindConfig.withFallback(commonConfig)) getExternal() should ===(address.toAkkaAddress("pekko.tcp")) getInternal() should not contain address.toAkkaAddress("tcp") - Await.result(sys.terminate(), Duration.Inf) + sys.close() } finally { openSS.close() } @@ -112,12 +110,12 @@ class NettyTransportSpec extends AnyWordSpec with Matchers with BindBehavior { bind-port = ${address.getPort} } """) - implicit val sys = ActorSystem("sys", bindConfig.withFallback(commonConfig)) + implicit val sys = pekko.actor.scaladsl.ActorSystem("sys", bindConfig.withFallback(commonConfig)) getExternal() should ===(address.toAkkaAddress("pekko.tcp")) getInternal() should contain(address.toAkkaAddress("tcp")) - Await.result(sys.terminate(), Duration.Inf) + sys.close() } "bind to all interfaces" in { @@ -127,12 +125,12 @@ class NettyTransportSpec extends AnyWordSpec with Matchers with BindBehavior { netty.tcp.bind-hostname = "0.0.0.0" } """) - implicit val sys = ActorSystem("sys", bindConfig.withFallback(commonConfig)) + implicit val sys = pekko.actor.scaladsl.ActorSystem("sys", bindConfig.withFallback(commonConfig)) getInternal().flatMap(_.port) should contain(getExternal().port.get) getInternal().map(x => (x.host.get should include).regex("0.0.0.0".r)) // regexp dot is intentional to match IPv4 and 6 addresses - Await.result(sys.terminate(), Duration.Inf) + sys.close() } "be able to specify byte buffer allocator" in { @@ -181,12 +179,12 @@ trait BindBehavior { enabled-transports = ["pekko.remote.classic.netty.tcp"] } """) - implicit val sys = ActorSystem("sys", bindConfig.withFallback(commonConfig)) + implicit val sys = pekko.actor.scaladsl.ActorSystem("sys", bindConfig.withFallback(commonConfig)) getExternal() should ===(address.toAkkaAddress(s"pekko.tcp")) getInternal() should contain(address.toAkkaAddress("tcp")) - Await.result(sys.terminate(), Duration.Inf) + sys.close() } s"bind to specified tcp address" in { @@ -213,12 +211,12 @@ trait BindBehavior { enabled-transports = ["pekko.remote.classic.netty.tcp"] } """) - implicit val sys = ActorSystem("sys", bindConfig.withFallback(commonConfig)) + implicit val sys = pekko.actor.scaladsl.ActorSystem("sys", bindConfig.withFallback(commonConfig)) getExternal() should ===(address.toAkkaAddress(s"pekko.tcp")) getInternal() should contain(bindAddress.toAkkaAddress("tcp")) - Await.result(sys.terminate(), Duration.Inf) + sys.close() } } } diff --git a/serialization-jackson/src/test/scala/doc/org/apache/pekko/serialization/jackson/SerializationDocSpec.scala b/serialization-jackson/src/test/scala/doc/org/apache/pekko/serialization/jackson/SerializationDocSpec.scala index 67e435aeec1..c5511ab2ee8 100644 --- a/serialization-jackson/src/test/scala/doc/org/apache/pekko/serialization/jackson/SerializationDocSpec.scala +++ b/serialization-jackson/src/test/scala/doc/org/apache/pekko/serialization/jackson/SerializationDocSpec.scala @@ -16,7 +16,7 @@ package doc.org.apache.pekko.serialization.jackson import java.util.Optional import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.serialization.Serialization import pekko.serialization.SerializationExtension import pekko.serialization.SerializerWithStringManifest diff --git a/serialization-jackson/src/test/scala/org/apache/pekko/serialization/jackson/JacksonFactorySpec.scala b/serialization-jackson/src/test/scala/org/apache/pekko/serialization/jackson/JacksonFactorySpec.scala index d1a853d24c3..c7479665c23 100644 --- a/serialization-jackson/src/test/scala/org/apache/pekko/serialization/jackson/JacksonFactorySpec.scala +++ b/serialization-jackson/src/test/scala/org/apache/pekko/serialization/jackson/JacksonFactorySpec.scala @@ -23,7 +23,8 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpecLike import org.apache.pekko -import pekko.actor.{ ActorSystem, ExtendedActorSystem } +import pekko.actor.ExtendedActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.testkit.TestKit class JacksonFactorySpec extends TestKit(ActorSystem("JacksonFactorySpec")) @@ -35,7 +36,7 @@ class JacksonFactorySpec extends TestKit(ActorSystem("JacksonFactorySpec")) override def afterAll(): Unit = { super.afterAll() - system.terminate() + system.close() } "Jackson Factory config" must { diff --git a/serialization-jackson/src/test/scala/org/apache/pekko/serialization/jackson/JacksonSerializerSpec.scala b/serialization-jackson/src/test/scala/org/apache/pekko/serialization/jackson/JacksonSerializerSpec.scala index 60fab76c931..65651a08e98 100644 --- a/serialization-jackson/src/test/scala/org/apache/pekko/serialization/jackson/JacksonSerializerSpec.scala +++ b/serialization-jackson/src/test/scala/org/apache/pekko/serialization/jackson/JacksonSerializerSpec.scala @@ -750,7 +750,7 @@ object JacksonSerializerSpec { abstract class JacksonSerializerSpec(serializerName: String) extends TestKit( - ActorSystem( + pekko.actor.scaladsl.ActorSystem( "JacksonJsonSerializerSpec", ConfigFactory.parseString(JacksonSerializerSpec.baseConfig(serializerName)))) with AnyWordSpecLike @@ -764,14 +764,15 @@ abstract class JacksonSerializerSpec(serializerName: String) } def withSystem[T](config: String)(block: ActorSystem => T): T = { - val sys = ActorSystem(system.name, ConfigFactory.parseString(config).withFallback(system.settings.config)) + val sys = pekko.actor.scaladsl.ActorSystem(system.name, + ConfigFactory.parseString(config).withFallback(system.settings.config)) try { block(sys) } finally shutdown(sys) } def withSystem[T](setup: ActorSystemSetup)(block: ActorSystem => T): T = { - val sys = ActorSystem(system.name, setup) + val sys = pekko.actor.scaladsl.ActorSystem(system.name, setup) try { block(sys) } finally shutdown(sys) @@ -1278,7 +1279,7 @@ abstract class JacksonSerializerSpec(serializerName: String) val className = clazz.getName withClue(className) { intercept[IllegalArgumentException] { - val sys = ActorSystem( + val sys = pekko.actor.scaladsl.ActorSystem( system.name, ConfigFactory.parseString(s""" pekko.actor.serialization-bindings { diff --git a/stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/StreamSpec.scala b/stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/StreamSpec.scala index 2fc883026aa..b8a0b6d3998 100644 --- a/stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/StreamSpec.scala +++ b/stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/StreamSpec.scala @@ -17,7 +17,8 @@ import scala.concurrent.Future import scala.concurrent.duration._ import org.apache.pekko -import pekko.actor.{ ActorRef, ActorSystem } +import pekko.actor.ActorRef +import pekko.actor.scaladsl.ActorSystem import pekko.stream.Materializer import pekko.stream.impl.PhasedFusingActorMaterializer import pekko.stream.impl.StreamSupervisor diff --git a/stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ActorSystemLifecycle.scala b/stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ActorSystemLifecycle.scala index 135bd376675..6a2bc337ffe 100644 --- a/stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ActorSystemLifecycle.scala +++ b/stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ActorSystemLifecycle.scala @@ -24,7 +24,7 @@ import org.testng.annotations.AfterClass import org.testng.annotations.BeforeClass import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.actor.ActorSystemImpl import pekko.event.Logging import pekko.testkit.PekkoSpec @@ -50,7 +50,7 @@ trait ActorSystemLifecycle { @AfterClass def shutdownActorSystem(): Unit = { try { - Await.ready(system.terminate(), shutdownTimeout) + Await.ready(system.terminateAsync(), shutdownTimeout) } catch { case _: TimeoutException => val msg = "Failed to stop [%s] within [%s] \n%s".format( diff --git a/stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FilePublisherTest.scala b/stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FilePublisherTest.scala index bdc64f881c3..e7557dd054d 100644 --- a/stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FilePublisherTest.scala +++ b/stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FilePublisherTest.scala @@ -19,7 +19,7 @@ import org.reactivestreams.Publisher import org.testng.annotations.{ AfterClass, BeforeClass } import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.event.Logging import pekko.stream.scaladsl.{ FileIO, Sink } import pekko.stream.testkit.Utils._ diff --git a/stream-tests/src/test/scala/org/apache/pekko/stream/io/TcpSpec.scala b/stream-tests/src/test/scala/org/apache/pekko/stream/io/TcpSpec.scala index 7ba6e0539fd..1707ff776b3 100644 --- a/stream-tests/src/test/scala/org/apache/pekko/stream/io/TcpSpec.scala +++ b/stream-tests/src/test/scala/org/apache/pekko/stream/io/TcpSpec.scala @@ -549,7 +549,7 @@ class TcpSpec extends StreamSpec(""" } "handle when connection actor terminates unexpectedly" in { - val system2 = ActorSystem( + val system2 = pekko.actor.scaladsl.ActorSystem( "TcpSpec-unexpected-system2", ConfigFactory.parseString(""" pekko.loglevel = DEBUG # issue #21660 @@ -599,7 +599,7 @@ class TcpSpec extends StreamSpec(""" } "provide full exceptions when connection attempt fails because name cannot be resolved" in { - val systemWithBrokenDns = ActorSystem( + val systemWithBrokenDns = pekko.actor.scaladsl.ActorSystem( "TcpSpec-resolution-failure", ConfigFactory.parseString(""" pekko.io.dns.inet-address.provider-object = org.apache.pekko.stream.io.FailingDnsResolver @@ -718,8 +718,8 @@ class TcpSpec extends StreamSpec(""" pekko.stream.materializer.subscription-timeout.mode = cancel pekko.stream.materializer.subscription-timeout.timeout = 42s """) - val serverSystem = ActorSystem("server", config) - val clientSystem = ActorSystem("client", config) + val serverSystem = pekko.actor.scaladsl.ActorSystem("server", config) + val clientSystem = pekko.actor.scaladsl.ActorSystem("client", config) try { @@ -853,7 +853,7 @@ class TcpSpec extends StreamSpec(""" } "not thrown on unbind after system has been shut down" in { - val sys2 = ActorSystem("shutdown-test-system") + val sys2 = pekko.actor.scaladsl.ActorSystem("shutdown-test-system") implicit val materializer = SystemMaterializer(sys2).materializer try { val address = temporaryServerAddress() @@ -865,11 +865,11 @@ class TcpSpec extends StreamSpec(""" // and is possible to communicate with Source.single(ByteString(0)).via(Tcp().outgoingConnection(address)).runWith(Sink.ignore).futureValue - sys2.terminate().futureValue + sys2.close() val binding = bindingFuture.futureValue binding.unbind().futureValue - } finally sys2.terminate() + } finally sys2.terminateAsync() } "show host and port in bind exception message" in EventFilter[BindException](occurrences = 1).intercept { diff --git a/stream-tests/src/test/scala/org/apache/pekko/stream/io/compression/CodecSpecSupport.scala b/stream-tests/src/test/scala/org/apache/pekko/stream/io/compression/CodecSpecSupport.scala index 54b9228e20b..18a4c8e01c8 100644 --- a/stream-tests/src/test/scala/org/apache/pekko/stream/io/compression/CodecSpecSupport.scala +++ b/stream-tests/src/test/scala/org/apache/pekko/stream/io/compression/CodecSpecSupport.scala @@ -14,7 +14,7 @@ package org.apache.pekko.stream.io.compression import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.testkit.TestKit import pekko.util.ByteString diff --git a/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/AggregateWithBoundarySpec.scala b/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/AggregateWithBoundarySpec.scala index 0cda1362b65..c667bbe1352 100644 --- a/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/AggregateWithBoundarySpec.scala +++ b/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/AggregateWithBoundarySpec.scala @@ -18,7 +18,7 @@ import scala.concurrent.Await import scala.concurrent.duration._ import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.stream.OverflowStrategy import pekko.stream.testkit.{ StreamSpec, TestPublisher, TestSubscriber } import pekko.testkit.{ ExplicitlyTriggeredScheduler, PekkoSpec } diff --git a/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/SourceFromPublisherSpec.scala b/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/SourceFromPublisherSpec.scala index 1d16dacb6ad..effa6456b34 100644 --- a/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/SourceFromPublisherSpec.scala +++ b/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/SourceFromPublisherSpec.scala @@ -14,7 +14,7 @@ package org.apache.pekko.stream.scaladsl import org.apache.pekko -import pekko.actor.ActorSystem +import pekko.actor.scaladsl.ActorSystem import pekko.stream.Attributes import pekko.stream.testkit.TestPublisher import pekko.testkit.TestKit diff --git a/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/StreamRefsSpec.scala b/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/StreamRefsSpec.scala index cefa6394692..8029085b429 100644 --- a/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/StreamRefsSpec.scala +++ b/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/StreamRefsSpec.scala @@ -21,7 +21,8 @@ import scala.util.control.NoStackTrace import org.apache.pekko import pekko.{ Done, NotUsed } -import pekko.actor.{ Actor, ActorIdentity, ActorLogging, ActorRef, ActorSystem, ActorSystemImpl, Identify, Props } +import pekko.actor.{ Actor, ActorIdentity, ActorLogging, ActorRef, ActorSystemImpl, Identify, Props } +import pekko.actor.scaladsl.ActorSystem import pekko.actor.Status.Failure import pekko.pattern._ import pekko.stream._ diff --git a/testkit/src/main/scala/org/apache/pekko/testkit/TestKit.scala b/testkit/src/main/scala/org/apache/pekko/testkit/TestKit.scala index 56c0d26f624..f16793f1f71 100644 --- a/testkit/src/main/scala/org/apache/pekko/testkit/TestKit.scala +++ b/testkit/src/main/scala/org/apache/pekko/testkit/TestKit.scala @@ -1035,8 +1035,8 @@ object TestKit { case f: FiniteDuration => f.dilated(actorSystem) case _ => 10.seconds.dilated(actorSystem).min(10.seconds) } - actorSystem.terminate() - try Await.ready(actorSystem.whenTerminated, d) + actorSystem.terminateImpl() + try Await.ready(actorSystem.whenTerminatedImpl, d) catch { case _: TimeoutException => val msg = "Failed to stop [%s] within [%s] \n%s".format( diff --git a/testkit/src/test/java/org/apache/pekko/testkit/PekkoJUnitActorSystemResource.java b/testkit/src/test/java/org/apache/pekko/testkit/PekkoJUnitActorSystemResource.java index 682cda8c080..0b7f68e740a 100644 --- a/testkit/src/test/java/org/apache/pekko/testkit/PekkoJUnitActorSystemResource.java +++ b/testkit/src/test/java/org/apache/pekko/testkit/PekkoJUnitActorSystemResource.java @@ -13,7 +13,7 @@ package org.apache.pekko.testkit; -import org.apache.pekko.actor.ActorSystem; +import org.apache.pekko.actor.javadsl.ActorSystem; import org.apache.pekko.testkit.javadsl.TestKit; import com.typesafe.config.Config; diff --git a/testkit/src/test/scala/org/apache/pekko/testkit/DefaultTimeoutSpec.scala b/testkit/src/test/scala/org/apache/pekko/testkit/DefaultTimeoutSpec.scala index fb6796ae8c4..13ce48c1bb6 100644 --- a/testkit/src/test/scala/org/apache/pekko/testkit/DefaultTimeoutSpec.scala +++ b/testkit/src/test/scala/org/apache/pekko/testkit/DefaultTimeoutSpec.scala @@ -17,13 +17,13 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec -import org.apache.pekko.actor.ActorSystem +import org.apache.pekko.actor.scaladsl.ActorSystem class DefaultTimeoutSpec extends AnyWordSpec with Matchers with BeforeAndAfterAll with TestKitBase with DefaultTimeout { implicit lazy val system: ActorSystem = ActorSystem("PekkoCustomSpec") - override def afterAll() = system.terminate() + override def afterAll() = system.close() "A spec with DefaultTimeout" should { "use timeout from settings" in { diff --git a/testkit/src/test/scala/org/apache/pekko/testkit/ImplicitSenderSpec.scala b/testkit/src/test/scala/org/apache/pekko/testkit/ImplicitSenderSpec.scala index dc0be66f265..cabfc028db7 100644 --- a/testkit/src/test/scala/org/apache/pekko/testkit/ImplicitSenderSpec.scala +++ b/testkit/src/test/scala/org/apache/pekko/testkit/ImplicitSenderSpec.scala @@ -17,13 +17,13 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec -import org.apache.pekko.actor.ActorSystem +import org.apache.pekko.actor.scaladsl.ActorSystem class ImplicitSenderSpec extends AnyWordSpec with Matchers with BeforeAndAfterAll with TestKitBase with ImplicitSender { implicit lazy val system: ActorSystem = ActorSystem("PekkoCustomSpec") - override def afterAll() = system.terminate() + override def afterAll() = system.close() "An ImplicitSender" should { "have testActor as its self" in { diff --git a/testkit/src/test/scala/org/apache/pekko/testkit/PekkoSpec.scala b/testkit/src/test/scala/org/apache/pekko/testkit/PekkoSpec.scala index 70263b213e0..5392997eddf 100644 --- a/testkit/src/test/scala/org/apache/pekko/testkit/PekkoSpec.scala +++ b/testkit/src/test/scala/org/apache/pekko/testkit/PekkoSpec.scala @@ -87,7 +87,7 @@ abstract class PekkoSpec(_system: ActorSystem) def this(config: Config) = this( - ActorSystem( + pekko.actor.scaladsl.ActorSystem( TestKitUtils.testNameFromCallStack(classOf[PekkoSpec], "".r), ConfigFactory.load(config.withFallback(PekkoSpec.testConf)))) @@ -95,7 +95,8 @@ abstract class PekkoSpec(_system: ActorSystem) def this(configMap: Map[String, _]) = this(PekkoSpec.mapToConfig(configMap)) - def this() = this(ActorSystem(TestKitUtils.testNameFromCallStack(classOf[PekkoSpec], "".r), PekkoSpec.testConf)) + def this() = this(pekko.actor.scaladsl.ActorSystem(TestKitUtils.testNameFromCallStack(classOf[PekkoSpec], "".r), + PekkoSpec.testConf)) implicit val patience: PatienceConfig = PatienceConfig(testKitSettings.SingleExpectDefaultTimeout.dilated, Span(100, Millis))