From 0cb5056ad5d733a920cce413381b986baa9317b3 Mon Sep 17 00:00:00 2001 From: Matthew de Detrich Date: Fri, 2 Dec 2022 14:49:16 +0100 Subject: [PATCH] Rename classes containing Akka to Pekko --- .../testkit/typed/internal/TestAppender.scala | 2 +- .../pekko/actor/AbstractFSMActorTest.java | 8 +- .../pekko/actor/ActorSelectionTest.java | 8 +- .../apache/pekko/actor/ActorSystemTest.java | 6 +- .../java/org/apache/pekko/actor/JavaAPI.java | 8 +- .../org/apache/pekko/actor/JavaExtension.java | 10 +- .../org/apache/pekko/actor/StashJavaAPI.java | 6 +- .../pekko/dispatch/JavaFutureTests.java | 8 +- .../pekko/event/LoggingAdapterTest.java | 6 +- .../pekko/pattern/CircuitBreakerTest.java | 8 +- .../apache/pekko/pattern/PatternsTest.java | 8 +- .../apache/pekko/pattern/StatusReplyTest.java | 8 +- .../apache/pekko/util/LineNumberSpec.scala | 4 +- .../apache/pekko/util/LineNumberSpec.scala | 4 +- .../apache/pekko/util/LineNumberSpec.scala | 4 +- .../org/apache/pekko/AkkaVersionSpec.scala | 86 ---- ...ionSpec.scala => PekkoExceptionSpec.scala} | 12 +- .../org/apache/pekko/PekkoVersionSpec.scala | 86 ++++ .../ActorConfigurationVerificationSpec.scala | 2 +- .../pekko/actor/ActorCreationPerfSpec.scala | 4 +- .../pekko/actor/ActorLifeCycleSpec.scala | 2 +- .../apache/pekko/actor/ActorMailboxSpec.scala | 2 +- .../org/apache/pekko/actor/ActorRefSpec.scala | 2 +- .../pekko/actor/ActorSelectionSpec.scala | 2 +- .../actor/ActorSystemDispatcherSpec.scala | 4 +- .../apache/pekko/actor/ActorSystemSpec.scala | 20 +- .../apache/pekko/actor/ActorTimeoutSpec.scala | 2 +- .../actor/ActorWithBoundedStashSpec.scala | 2 +- .../pekko/actor/ActorWithStashSpec.scala | 2 +- .../apache/pekko/actor/ConsistencySpec.scala | 6 +- .../pekko/actor/CoordinatedShutdownSpec.scala | 4 +- .../actor/DeadLetterSupressionSpec.scala | 4 +- .../actor/DeadLetterSuspensionSpec.scala | 4 +- .../apache/pekko/actor/DeathWatchSpec.scala | 4 +- .../org/apache/pekko/actor/DeployerSpec.scala | 8 +- .../org/apache/pekko/actor/FSMActorSpec.scala | 2 +- .../apache/pekko/actor/FSMTimingSpec.scala | 2 +- .../pekko/actor/FSMTransitionSpec.scala | 2 +- .../apache/pekko/actor/ForwardActorSpec.scala | 2 +- .../apache/pekko/actor/FunctionRefSpec.scala | 4 +- .../org/apache/pekko/actor/HotSwapSpec.scala | 2 +- .../actor/LocalActorRefProviderSpec.scala | 2 +- .../pekko/actor/PropsCreationSpec.scala | 4 +- .../pekko/actor/ReceiveTimeoutSpec.scala | 2 +- .../pekko/actor/RestartStrategySpec.scala | 4 +- .../apache/pekko/actor/SchedulerSpec.scala | 6 +- .../pekko/actor/SupervisorHierarchySpec.scala | 5 +- .../pekko/actor/SupervisorMiscSpec.scala | 4 +- .../apache/pekko/actor/SupervisorSpec.scala | 2 +- .../pekko/actor/SupervisorTreeSpec.scala | 4 +- .../apache/pekko/actor/Ticket669Spec.scala | 4 +- .../org/apache/pekko/actor/TimerSpec.scala | 4 +- .../apache/pekko/actor/TypedActorSpec.scala | 6 +- .../org/apache/pekko/actor/UidClashTest.scala | 4 +- .../pekko/actor/dispatch/ActorModelSpec.scala | 2 +- .../dispatch/BalancingDispatcherSpec.scala | 4 +- .../actor/dispatch/DispatcherActorSpec.scala | 4 +- .../actor/dispatch/DispatcherActorsSpec.scala | 4 +- .../actor/dispatch/DispatchersSpec.scala | 4 +- .../actor/dispatch/PinnedActorSpec.scala | 4 +- .../pekko/actor/dungeon/DispatchSpec.scala | 2 +- .../pekko/actor/routing/ListenerSpec.scala | 2 +- .../org/apache/pekko/config/ConfigSpec.scala | 4 +- .../apache/pekko/dataflow/Future2Actor.scala | 4 +- .../dispatch/ControlAwareDispatcherSpec.scala | 4 +- .../pekko/dispatch/ExecutionContextSpec.scala | 4 +- .../dispatch/ForkJoinPoolStarvationSpec.scala | 6 +- .../pekko/dispatch/MailboxConfigSpec.scala | 8 +- .../dispatch/PriorityDispatcherSpec.scala | 4 +- .../SameThreadExecutionContextSpec.scala | 4 +- .../StablePriorityDispatcherSpec.scala | 4 +- .../sysmsg/SystemMessageListSpec.scala | 4 +- .../AddressTerminatedTopicBenchSpec.scala | 2 +- .../org/apache/pekko/event/EventBusSpec.scala | 4 +- .../apache/pekko/event/EventStreamSpec.scala | 4 +- .../org/apache/pekko/event/LoggerSpec.scala | 12 +- .../pekko/event/LoggingReceiveSpec.scala | 2 +- .../pekko/event/MarkerLoggingSpec.scala | 2 +- .../pekko/event/jul/JavaLoggerSpec.scala | 4 +- .../apache/pekko/io/CapacityLimitSpec.scala | 4 +- .../pekko/io/InetAddressDnsResolverSpec.scala | 6 +- .../apache/pekko/io/TcpConnectionSpec.scala | 6 +- .../apache/pekko/io/TcpIntegrationSpec.scala | 4 +- .../pekko/io/TcpIntegrationSpecSupport.scala | 4 +- .../org/apache/pekko/io/TcpListenerSpec.scala | 4 +- .../io/UdpConnectedIntegrationSpec.scala | 4 +- .../apache/pekko/io/UdpIntegrationSpec.scala | 4 +- .../apache/pekko/io/dns/DnsSettingsSpec.scala | 4 +- .../pekko/io/dns/DockerBindDnsService.scala | 4 +- .../io/dns/internal/AsyncDnsManagerSpec.scala | 4 +- .../dns/internal/AsyncDnsResolverSpec.scala | 4 +- .../pekko/io/dns/internal/DnsClientSpec.scala | 4 +- .../io/dns/internal/TcpDnsClientSpec.scala | 4 +- .../org/apache/pekko/pattern/AskSpec.scala | 4 +- .../BackoffOnRestartSupervisorSpec.scala | 4 +- .../pekko/pattern/BackoffSupervisorSpec.scala | 2 +- .../pekko/pattern/CircuitBreakerMTSpec.scala | 2 +- .../pekko/pattern/CircuitBreakerSpec.scala | 2 +- .../pattern/CircuitBreakerStressSpec.scala | 4 +- .../apache/pekko/pattern/PatternSpec.scala | 4 +- .../org/apache/pekko/pattern/PipeToSpec.scala | 4 +- .../apache/pekko/pattern/PromiseRefSpec.scala | 4 +- .../org/apache/pekko/pattern/RetrySpec.scala | 4 +- .../pekko/pattern/StatusReplySpec.scala | 4 +- .../pattern/extended/ExplicitAskSpec.scala | 4 +- .../apache/pekko/routing/BalancingSpec.scala | 6 +- .../apache/pekko/routing/BroadcastSpec.scala | 4 +- .../routing/ConfiguredLocalRoutingSpec.scala | 4 +- .../routing/ConsistentHashingRouterSpec.scala | 4 +- .../routing/MetricsBasedResizerSpec.scala | 2 +- .../org/apache/pekko/routing/RandomSpec.scala | 4 +- .../apache/pekko/routing/ResizerSpec.scala | 2 +- .../apache/pekko/routing/RoundRobinSpec.scala | 2 +- .../pekko/routing/RouteeCreationSpec.scala | 4 +- .../apache/pekko/routing/RoutingSpec.scala | 2 +- .../ScatterGatherFirstCompletedSpec.scala | 4 +- .../pekko/routing/SmallestMailboxSpec.scala | 4 +- .../pekko/routing/TailChoppingSpec.scala | 2 +- .../serialization/AsyncSerializeSpec.scala | 4 +- .../DisabledJavaSerializerWarningSpec.scala | 2 +- .../PrimitivesSerializationSpec.scala | 6 +- .../SerializationSetupSpec.scala | 4 +- .../pekko/serialization/SerializeSpec.scala | 26 +- .../org/apache/pekko/util/DurationSpec.scala | 4 +- .../org/apache/pekko/util/IndexSpec.scala | 4 +- .../apache/pekko/util/ManifestInfoSpec.scala | 4 +- .../apache/pekko/util/TokenBucketSpec.scala | 4 +- ...la => SerializerOutsidePekkoPackage.scala} | 2 +- .../typed/javadsl/ActorContextAskTest.java | 4 +- .../actor/typed/javadsl/AdapterTest.java | 8 +- .../actor/typed/javadsl/InterceptTest.java | 4 +- .../actor/typed/javadsl/RoutersTest.java | 4 +- ...kaException.scala => PekkoException.scala} | 6 +- .../{AkkaVersion.scala => PekkoVersion.scala} | 14 +- .../scala/org/apache/pekko/actor/Actor.scala | 16 +- .../org/apache/pekko/actor/Scheduler.scala | 2 +- .../scala/org/apache/pekko/actor/Stash.scala | 4 +- .../apache/pekko/actor/dungeon/Dispatch.scala | 4 +- .../ForkJoinExecutorConfigurator.scala | 8 +- .../pekko/dispatch/ThreadPoolBuilder.scala | 4 +- .../org/apache/pekko/event/Logging.scala | 6 +- .../pekko/io/dns/internal/TcpDnsClient.scala | 6 +- .../apache/pekko/pattern/CircuitBreaker.scala | 4 +- .../org/apache/pekko/routing/Resizer.scala | 4 +- .../metrics/ClusterMetricsExtensionSpec.scala | 2 +- .../metrics/ClusterMetricsSettingsSpec.scala | 4 +- .../pekko/cluster/metrics/EWMASpec.scala | 4 +- .../pekko/cluster/metrics/MetricSpec.scala | 6 +- .../metrics/MetricsCollectorSpec.scala | 2 +- .../pekko/cluster/metrics/TestUtil.scala | 4 +- .../cluster/metrics/WeightedRouteesSpec.scala | 4 +- .../protobuf/MessageSerializerSpec.scala | 4 +- .../ClusterShardingHealthCheckSpec.scala | 4 +- .../ClusterShardingInternalsSpec.scala | 4 +- .../sharding/ClusterShardingLeaseSpec.scala | 4 +- .../ClusterShardingSettingsSpec.scala | 4 +- .../ConcurrentStartupShardingSpec.scala | 4 +- .../CoordinatedShutdownShardingSpec.scala | 4 +- ...atedLeastShardAllocationStrategySpec.scala | 4 +- .../sharding/EntityTerminationSpec.scala | 4 +- .../sharding/GetShardTypeNamesSpec.scala | 4 +- .../JoinConfigCompatCheckShardingSpec.scala | 4 +- ...hardAllocationStrategyRandomizedSpec.scala | 4 +- .../LeastShardAllocationStrategySpec.scala | 4 +- .../PersistentShardingMigrationSpec.scala | 4 +- .../cluster/sharding/ProxyShardingSpec.scala | 4 +- .../RememberEntitiesAndStartEntitySpec.scala | 4 +- .../RememberEntitiesBatchedUpdatesSpec.scala | 4 +- .../RememberEntitiesFailureSpec.scala | 4 +- ...erEntitiesShardIdExtractorChangeSpec.scala | 4 +- ...emoveInternalClusterShardingDataSpec.scala | 4 +- .../cluster/sharding/ShardRegionSpec.scala | 4 +- .../cluster/sharding/ShardWithLeaseSpec.scala | 4 +- .../sharding/ShardingQueriesSpec.scala | 4 +- .../cluster/sharding/StartEntitySpec.scala | 4 +- .../cluster/sharding/SupervisionSpec.scala | 6 +- .../ExternalShardAllocationStrategySpec.scala | 4 +- .../RememberEntitiesShardStoreSpec.scala | 4 +- .../RememberEntitiesStarterSpec.scala | 4 +- .../passivation/EntityPassivationSpec.scala | 4 +- ...ClusterShardingMessageSerializerSpec.scala | 4 +- .../singleton/ClusterSingletonManager.scala | 4 +- .../cluster/client/ClusterClientTest.java | 6 +- .../pubsub/DistributedPubSubMediatorTest.java | 6 +- .../ClusterClientMessageSerializerSpec.scala | 4 +- ...ributedPubSubMediatorDeadLettersSpec.scala | 4 +- .../DistributedPubSubMediatorRouterSpec.scala | 4 +- ...stributedPubSubMessageSerializerSpec.scala | 4 +- .../singleton/ClusterSingletonLeaseSpec.scala | 4 +- .../ClusterSingletonLeavingSpeedSpec.scala | 4 +- .../ClusterSingletonRestart2Spec.scala | 4 +- .../ClusterSingletonRestartSpec.scala | 4 +- ...lusterSingletonMessageSerializerSpec.scala | 4 +- .../src/main/resources/reference.conf | 4 +- ...cala => PekkoClusterTypedSerializer.scala} | 2 +- .../cluster/typed/RemoteMessageSpec.scala | 4 +- ... => PekkoClusterTypedSerializerSpec.scala} | 8 +- .../pekko/cluster/ClusterConfigSpec.scala | 4 +- .../pekko/cluster/ClusterDeployerSpec.scala | 2 +- .../ClusterDomainEventPublisherSpec.scala | 4 +- .../ClusterHeartbeatReceiverSpec.scala | 4 +- .../cluster/ClusterHeartbeatSenderSpec.scala | 4 +- .../apache/pekko/cluster/ClusterLogSpec.scala | 4 +- .../apache/pekko/cluster/ClusterSpec.scala | 4 +- .../apache/pekko/cluster/ClusterTestKit.scala | 6 +- .../cluster/CrossDcHeartbeatSenderSpec.scala | 4 +- .../JoinConfigCompatCheckClusterSpec.scala | 4 +- .../cluster/JoinConfigCompatCheckerSpec.scala | 6 +- .../ShutdownAfterJoinSeedNodesSpec.scala | 2 +- .../cluster/StartupWithOneThreadSpec.scala | 6 +- .../pekko/cluster/VectorClockSpec.scala | 4 +- .../ClusterMessageSerializerSpec.scala | 4 +- .../routing/ClusterRouterSupervisorSpec.scala | 2 +- .../pekko/cluster/sbr/LeaseMajoritySpec.scala | 4 +- .../cluster/sbr/SplitBrainResolverSpec.scala | 4 +- .../pekko/cluster/testkit/AutoDownSpec.scala | 4 +- .../lease/javadsl/LeaseProviderTest.java | 6 +- .../lease/scaladsl/LeaseProviderSpec.scala | 4 +- .../dns/DnsServiceDiscoverySpec.scala | 4 +- .../ReplicatorMessageSerializer.scala | 4 +- .../ddata/ReplicatorSettingsSpec.scala | 4 +- .../cluster/ddata/WriteAggregatorSpec.scala | 2 +- .../remote/testconductor/Conductor.scala | 4 +- .../query/PersistenceQueryTest.java | 6 +- .../query/internal/QuerySerializerSpec.scala | 4 +- .../leveldb/AllPersistenceIdsSpec.scala | 4 +- .../query/journal/leveldb/Cleanup.scala | 4 +- .../leveldb/EventsByPersistenceIdSpec.scala | 4 +- .../journal/leveldb/EventsByTagSpec.scala | 4 +- .../leveldb/PersistencePluginProxySpec.scala | 4 +- .../leveldb/SharedLeveldbJournalSpec.scala | 4 +- .../serialization/SerializerSpec.scala | 6 +- .../ReplicatedEventSourcingJacksonSpec.scala | 10 +- .../persistence/journal/AsyncWriteProxy.scala | 4 +- .../AtLeastOnceDeliveryCrashSpec.scala | 4 +- .../AtLeastOnceDeliveryFailureSpec.scala | 2 +- .../pekko/persistence/PersistenceSpec.scala | 8 +- .../PersistentActorJournalProtocolSpec.scala | 2 +- .../PersistentActorRecoveryTimeoutSpec.scala | 4 +- .../SnapshotDirectoryFailureSpec.scala | 4 +- .../journal/InmemEventAdaptersSpec.scala | 4 +- .../journal/ReplayFilterSpec.scala | 2 +- .../serialization/MessageSerializerSpec.scala | 4 +- .../remote/testconductor/BarrierSpec.scala | 4 +- .../remote/testconductor/ControllerSpec.scala | 4 +- .../org/apache/pekko/remote/WireFormats.java | 446 +++++++++--------- .../src/main/protobuf/WireFormats.proto | 12 +- .../apache/pekko/remote/AckedDelivery.scala | 6 +- .../org/apache/pekko/remote/Endpoint.scala | 52 +- .../apache/pekko/remote/RemoteTransport.scala | 4 +- .../org/apache/pekko/remote/Remoting.scala | 63 +-- .../FailureInjectorTransportAdapter.scala | 4 +- ...AkkaPduCodec.scala => PekkoPduCodec.scala} | 53 ++- ...ort.scala => PekkoProtocolTransport.scala} | 87 ++-- .../transport/ThrottlerTransportAdapter.scala | 4 +- .../pekko/remote/transport/Transport.scala | 4 +- .../remote/transport/netty/NettyHelpers.scala | 4 +- .../jfr/JFRRemotingFlightRecorderSpec.scala | 4 +- .../remote/AccrualFailureDetectorSpec.scala | 4 +- .../pekko/remote/AckedDeliverySpec.scala | 4 +- .../apache/pekko/remote/DaemonicSpec.scala | 2 +- .../remote/DeadlineFailureDetectorSpec.scala | 4 +- .../pekko/remote/EndpointRegistrySpec.scala | 4 +- .../remote/FailureDetectorRegistrySpec.scala | 4 +- .../apache/pekko/remote/LogSourceSpec.scala | 4 +- .../pekko/remote/MessageLoggingSpec.scala | 4 +- .../pekko/remote/NetworkFailureSpec.scala | 4 +- .../pekko/remote/RemoteConfigSpec.scala | 12 +- .../RemoteConsistentHashingRouterSpec.scala | 4 +- .../pekko/remote/RemoteDeployerSpec.scala | 2 +- .../pekko/remote/RemoteRouterSpec.scala | 2 +- .../remote/Ticket1978CommunicationSpec.scala | 8 +- .../pekko/remote/Ticket1978ConfigSpec.scala | 2 +- .../TransientSerializationErrorSpec.scala | 4 +- .../remote/TypedActorRemoteDeploySpec.scala | 4 +- .../remote/artery/ArteryMultiNodeSpec.scala | 6 +- .../remote/artery/DuplicateFlushSpec.scala | 4 +- .../artery/DuplicateHandshakeSpec.scala | 4 +- .../remote/artery/EnvelopeBufferSpec.scala | 4 +- .../artery/InboundControlJunctionSpec.scala | 4 +- .../remote/artery/InboundHandshakeSpec.scala | 4 +- .../remote/artery/LruBoundedCacheSpec.scala | 4 +- .../artery/OutboundControlJunctionSpec.scala | 4 +- .../remote/artery/OutboundHandshakeSpec.scala | 4 +- .../remote/artery/RemoteDeployerSpec.scala | 2 +- .../RemoteInstrumentsSerializationSpec.scala | 4 +- .../remote/artery/RemoteRouterSpec.scala | 2 +- .../artery/RemotingFlightRecorderSpec.scala | 4 +- .../RollingEventLogSimulationSpec.scala | 4 +- .../pekko/remote/artery/SendQueueSpec.scala | 4 +- .../artery/SystemMessageAckerSpec.scala | 4 +- .../remote/artery/aeron/AeronSinkSpec.scala | 4 +- .../compress/CompressionTableSpec.scala | 4 +- .../compress/OutboundCompressionSpec.scala | 4 +- .../artery/tcp/SecureRandomFactorySpec.scala | 4 +- .../remote/artery/tcp/TcpFramingSpec.scala | 4 +- .../pekko/remote/classic/ActorsLeakSpec.scala | 2 +- .../remote/classic/RemoteDeathWatchSpec.scala | 2 +- .../RemoteDeploymentAllowListSpec.scala | 2 +- .../remote/classic/RemoteWatcherSpec.scala | 2 +- .../pekko/remote/classic/RemotingSpec.scala | 8 +- .../pekko/remote/classic/UntrustedSpec.scala | 4 +- .../transport/GenericTransportSpec.scala | 14 +- ...ocolSpec.scala => PekkoProtocolSpec.scala} | 36 +- ...st.scala => PekkoProtocolStressTest.scala} | 10 +- .../SwitchableLoggedBehaviorSpec.scala | 8 +- .../SystemMessageDeliveryStressTest.scala | 6 +- .../classic/transport/TestTransportSpec.scala | 2 +- .../classic/transport/ThrottleModeSpec.scala | 4 +- .../ThrottlerTransportAdapterSpec.scala | 4 +- .../transport/netty/NettyTransportSpec.scala | 2 +- .../AllowJavaSerializationOffSpec.scala | 4 +- .../ArteryMessageSerializerSpec.scala | 4 +- ...SerializerAllowJavaSerializationSpec.scala | 8 +- .../MessageContainerSerializerSpec.scala | 4 +- .../MiscMessageSerializerSpec.scala | 6 +- .../PrimitivesSerializationSpec.scala | 6 +- .../ProtobufSerializerSpec.scala | 4 +- ...erializationTransportInformationSpec.scala | 4 +- .../SystemMessageSerializationSpec.scala | 6 +- .../src/main/resources/reference.conf | 10 +- .../jackson/AkkaJacksonModule.scala | 26 - .../jackson/AkkaSerializationModule.scala | 4 +- .../jackson/JacksonObjectMapperProvider.scala | 4 +- .../jackson/PekkoJacksonModule.scala | 26 + .../jackson/JacksonSerializerSpec.scala | 30 +- .../pekko/event/slf4j/Slf4jLoggerSpec.scala | 4 +- .../event/slf4j/Slf4jLoggingFilterSpec.scala | 4 +- .../stream/testkit/BaseTwoStreamsSetup.scala | 4 +- .../pekko/stream/testkit/StreamSpec.scala | 10 +- .../stream/testkit/StreamTestKitSpec.scala | 2 +- .../testkit/TestPublisherSubscriberSpec.scala | 4 +- ...blePublisherViaJavaFlowPublisherTest.scala | 2 +- .../stream/tck/ActorSystemLifecycle.scala | 4 +- .../tck/AkkaSubscriberVerification.scala | 10 +- .../tck/CancelledSinkSubscriberTest.scala | 2 +- .../apache/pekko/stream/tck/ConcatTest.scala | 2 +- .../pekko/stream/tck/EmptyPublisherTest.scala | 2 +- .../stream/tck/FanoutPublisherTest.scala | 2 +- .../pekko/stream/tck/FilePublisherTest.scala | 6 +- .../FlatMapConcatDoubleSubscriberTest.scala | 2 +- .../pekko/stream/tck/FlatMapPrefixTest.scala | 2 +- .../apache/pekko/stream/tck/FlattenTest.scala | 2 +- .../stream/tck/FoldSinkSubscriberTest.scala | 2 +- .../tck/ForeachSinkSubscriberTest.scala | 2 +- .../stream/tck/FusableProcessorTest.scala | 2 +- .../stream/tck/FuturePublisherTest.scala | 2 +- .../apache/pekko/stream/tck/GroupByTest.scala | 2 +- .../stream/tck/InputStreamSourceTest.scala | 2 +- .../stream/tck/IterablePublisherTest.scala | 2 +- .../org/apache/pekko/stream/tck/MapTest.scala | 2 +- .../pekko/stream/tck/MaybeSourceTest.scala | 2 +- ... PekkoIdentityProcessorVerification.scala} | 2 +- ...scala => PekkoPublisherVerification.scala} | 2 +- .../pekko/stream/tck/PrefixAndTailTest.scala | 2 +- .../stream/tck/SingleElementSourceTest.scala | 2 +- .../pekko/stream/tck/SplitWhenTest.scala | 2 +- .../stream/tck/TransformProcessorTest.scala | 2 +- .../stream/tck/VirtualPublisherTest.scala | 4 +- .../pekko/stream/StreamAttributeDocTest.java | 10 +- .../org/apache/pekko/stream/StreamTest.java | 4 +- .../pekko/stream/io/InputStreamSinkTest.java | 6 +- .../pekko/stream/io/OutputStreamSinkTest.java | 6 +- .../stream/io/OutputStreamSourceTest.java | 6 +- .../pekko/stream/io/SinkAsJavaSourceTest.java | 6 +- .../pekko/stream/javadsl/AttributesTest.java | 8 +- .../apache/pekko/stream/javadsl/FlowTest.java | 8 +- .../stream/javadsl/FlowThrottleTest.java | 8 +- .../stream/javadsl/FlowWithContextTest.java | 8 +- .../javadsl/FlowWithContextThrottleTest.java | 8 +- .../pekko/stream/javadsl/FramingTest.java | 8 +- .../pekko/stream/javadsl/GraphDslTest.java | 8 +- .../pekko/stream/javadsl/JsonFramingTest.java | 8 +- .../pekko/stream/javadsl/KillSwitchTest.java | 8 +- .../stream/javadsl/LazyAndFutureFlowTest.java | 8 +- .../javadsl/LazyAndFutureSourcesTest.java | 8 +- .../pekko/stream/javadsl/RetryFlowTest.java | 8 +- .../stream/javadsl/RunnableGraphTest.java | 8 +- .../pekko/stream/javadsl/SetupTest.java | 8 +- .../apache/pekko/stream/javadsl/SinkTest.java | 8 +- .../pekko/stream/javadsl/SourceTest.java | 8 +- .../SourceWithContextThrottleTest.java | 8 +- .../apache/pekko/stream/javadsl/TcpTest.java | 8 +- .../apache/pekko/stream/stage/StageTest.java | 8 +- .../impl/LinearTraversalBuilderSpec.scala | 4 +- .../stream/impl/TraversalBuilderSpec.scala | 4 +- .../impl/fusing/AsyncCallbackSpec.scala | 4 +- .../impl/fusing/ChasingEventsSpec.scala | 4 +- .../pekko/stream/io/DeprecatedTlsSpec.scala | 4 +- .../org/apache/pekko/stream/io/TcpSpec.scala | 4 +- .../scaladsl/AggregateWithBoundarySpec.scala | 4 +- .../stream/scaladsl/FlowOrElseSpec.scala | 4 +- .../stream/scaladsl/FlowPrependSpec.scala | 4 +- .../stream/scaladsl/JsonFramingSpec.scala | 4 +- .../stream/scaladsl/QueueSourceSpec.scala | 2 +- .../stream/scaladsl/StreamRefsSpec.scala | 4 +- .../snapshot/MaterializerStateSpec.scala | 4 +- akka-stream/src/main/resources/reference.conf | 2 +- ...kaSSLConfig.scala => PekkoSSLConfig.scala} | 42 +- ...erBridge.scala => PekkoLoggerBridge.scala} | 8 +- .../pekko/stream/impl/io/TLSActor.scala | 8 +- .../pekko/stream/impl/io/TlsModule.scala | 8 +- .../org/apache/pekko/stream/javadsl/TLS.scala | 6 +- .../apache/pekko/stream/scaladsl/TLS.scala | 8 +- ...ava => PekkoJUnitActorSystemResource.java} | 16 +- .../apache/pekko/testkit/AbstractSpec.scala | 2 +- .../pekko/testkit/DefaultTimeoutSpec.scala | 2 +- .../ExplicitlyTriggeredSchedulerSpec.scala | 2 +- .../pekko/testkit/ImplicitSenderSpec.scala | 2 +- .../{AkkaSpec.scala => PekkoSpec.scala} | 12 +- ...AkkaSpecSpec.scala => PekkoSpecSpec.scala} | 16 +- .../pekko/testkit/TestActorRefSpec.scala | 2 +- .../apache/pekko/testkit/TestActorsSpec.scala | 2 +- .../pekko/testkit/TestEventListenerSpec.scala | 2 +- .../apache/pekko/testkit/TestFSMRefSpec.scala | 2 +- .../apache/pekko/testkit/TestProbeSpec.scala | 2 +- .../apache/pekko/testkit/TestTimeSpec.scala | 2 +- .../pekko/testkit/metrics/MetricsKit.scala | 8 +- ...orter.scala => PekkoConsoleReporter.scala} | 2 +- build.sbt | 24 +- docs/src/main/paradox/actors.md | 8 +- docs/src/main/paradox/additional/osgi.md | 6 +- docs/src/main/paradox/cluster-client.md | 6 +- docs/src/main/paradox/cluster-metrics.md | 6 +- docs/src/main/paradox/cluster-routing.md | 6 +- docs/src/main/paradox/cluster-sharding.md | 6 +- docs/src/main/paradox/cluster-singleton.md | 6 +- docs/src/main/paradox/cluster-usage.md | 6 +- docs/src/main/paradox/coordination.md | 6 +- docs/src/main/paradox/discovery/index.md | 6 +- docs/src/main/paradox/dispatchers.md | 6 +- docs/src/main/paradox/distributed-data.md | 6 +- docs/src/main/paradox/distributed-pub-sub.md | 6 +- .../durable-state/persistence-query.md | 6 +- docs/src/main/paradox/fault-tolerance.md | 6 +- docs/src/main/paradox/fsm.md | 8 +- docs/src/main/paradox/futures.md | 6 +- docs/src/main/paradox/index-actors.md | 8 +- .../main/paradox/index-utilities-classic.md | 8 +- docs/src/main/paradox/io-tcp.md | 6 +- docs/src/main/paradox/io-udp.md | 6 +- docs/src/main/paradox/io.md | 6 +- docs/src/main/paradox/logging.md | 12 +- docs/src/main/paradox/mailboxes.md | 6 +- docs/src/main/paradox/multi-node-testing.md | 6 +- docs/src/main/paradox/persistence-fsm.md | 6 +- docs/src/main/paradox/persistence-journals.md | 6 +- .../main/paradox/persistence-query-leveldb.md | 6 +- docs/src/main/paradox/persistence-query.md | 6 +- .../paradox/persistence-schema-evolution.md | 8 +- docs/src/main/paradox/persistence.md | 8 +- docs/src/main/paradox/remoting-artery.md | 6 +- docs/src/main/paradox/remoting.md | 6 +- docs/src/main/paradox/routing.md | 6 +- docs/src/main/paradox/scheduler.md | 6 +- .../src/main/paradox/serialization-classic.md | 6 +- .../src/main/paradox/serialization-jackson.md | 10 +- docs/src/main/paradox/serialization.md | 6 +- docs/src/main/paradox/split-brain-resolver.md | 6 +- docs/src/main/paradox/stream/actor-interop.md | 6 +- .../main/paradox/stream/futures-interop.md | 6 +- docs/src/main/paradox/stream/index.md | 8 +- .../paradox/stream/operators/ActorFlow/ask.md | 6 +- .../operators/ActorFlow/askWithContext.md | 6 +- .../operators/ActorFlow/askWithStatus.md | 4 +- .../ActorFlow/askWithStatusAndContext.md | 6 +- .../stream/operators/ActorSink/actorRef.md | 6 +- .../ActorSink/actorRefWithBackpressure.md | 6 +- .../stream/operators/ActorSource/actorRef.md | 6 +- .../ActorSource/actorRefWithBackpressure.md | 6 +- .../paradox/stream/operators/PubSub/sink.md | 6 +- .../paradox/stream/operators/PubSub/source.md | 6 +- .../paradox/stream/operators/Source/range.md | 6 +- .../stream/reactive-streams-interop.md | 6 +- .../main/paradox/stream/stream-composition.md | 6 +- .../main/paradox/stream/stream-cookbook.md | 6 +- .../main/paradox/stream/stream-customize.md | 6 +- .../src/main/paradox/stream/stream-dynamic.md | 6 +- docs/src/main/paradox/stream/stream-error.md | 6 +- .../paradox/stream/stream-flows-and-basics.md | 6 +- docs/src/main/paradox/stream/stream-graphs.md | 6 +- docs/src/main/paradox/stream/stream-io.md | 6 +- .../main/paradox/stream/stream-parallelism.md | 6 +- .../main/paradox/stream/stream-quickstart.md | 6 +- docs/src/main/paradox/stream/stream-rate.md | 6 +- docs/src/main/paradox/stream/stream-refs.md | 6 +- .../main/paradox/stream/stream-substream.md | 6 +- .../src/main/paradox/stream/stream-testkit.md | 6 +- docs/src/main/paradox/testing.md | 6 +- .../src/main/paradox/typed/actor-discovery.md | 6 +- .../src/main/paradox/typed/actor-lifecycle.md | 6 +- docs/src/main/paradox/typed/actors.md | 8 +- docs/src/main/paradox/typed/cluster-dc.md | 6 +- .../typed/cluster-sharded-daemon-process.md | 6 +- .../main/paradox/typed/cluster-sharding.md | 6 +- .../main/paradox/typed/cluster-singleton.md | 6 +- docs/src/main/paradox/typed/cluster.md | 6 +- docs/src/main/paradox/typed/coexisting.md | 6 +- docs/src/main/paradox/typed/dispatchers.md | 6 +- .../main/paradox/typed/distributed-data.md | 6 +- .../main/paradox/typed/distributed-pub-sub.md | 6 +- .../typed/durable-state/persistence.md | 8 +- docs/src/main/paradox/typed/from-classic.md | 6 +- docs/src/main/paradox/typed/guide/modules.md | 54 +-- .../main/paradox/typed/guide/tutorial_1.md | 6 +- .../paradox/typed/interaction-patterns.md | 6 +- docs/src/main/paradox/typed/logging.md | 6 +- docs/src/main/paradox/typed/mailboxes.md | 6 +- .../main/paradox/typed/persistence-testing.md | 20 +- docs/src/main/paradox/typed/persistence.md | 8 +- .../main/paradox/typed/reliable-delivery.md | 18 +- docs/src/main/paradox/typed/routers.md | 6 +- docs/src/main/paradox/typed/stash.md | 6 +- docs/src/main/paradox/typed/testing.md | 6 +- .../java/jdocs/actor/SchedulerDocTest.java | 8 +- .../jdocs/dispatcher/DispatcherDocTest.java | 10 +- .../java/jdocs/event/EventBusDocTest.java | 6 +- .../test/java/jdocs/future/FutureDocTest.java | 9 +- .../test/java/jdocs/io/japi/IODocTest.java | 8 +- .../remoting/RemoteDeploymentDocTest.java | 10 +- .../ConsistentHashingRouterDocTest.java | 6 +- .../jdocs/routing/CustomRouterDocTest.java | 6 +- .../java/jdocs/routing/RouterDocTest.java | 6 +- .../java/jdocs/testkit/ParentChildTest.java | 6 +- .../java/jdocs/testkit/TestKitDocTest.java | 6 +- .../test/scala/docs/actor/ActorDocSpec.scala | 4 +- .../test/scala/docs/actor/FSMDocSpec.scala | 4 +- .../docs/actor/InitializationDocSpec.scala | 4 +- .../scala/docs/actor/SchedulerDocSpec.scala | 2 +- .../scala/docs/cluster/ClusterDocSpec.scala | 4 +- .../docs/coordination/LeaseDocSpec.scala | 4 +- .../docs/ddata/DistributedDataDocSpec.scala | 4 +- .../docs/discovery/DnsDiscoveryDocSpec.scala | 4 +- .../docs/dispatcher/DispatcherDocSpec.scala | 4 +- .../scala/docs/event/EventBusDocSpec.scala | 4 +- .../scala/docs/event/LoggingDocSpec.scala | 4 +- .../docs/extension/ExtensionDocSpec.scala | 4 +- .../extension/SettingsExtensionDocSpec.scala | 4 +- .../scala/docs/future/FutureDocSpec.scala | 2 +- docs/src/test/scala/docs/io/IODocSpec.scala | 4 +- .../test/scala/docs/io/ReadBackPressure.scala | 4 +- docs/src/test/scala/docs/io/UdpDocSpec.scala | 4 +- .../PersistenceEventAdapterDocSpec.scala | 4 +- .../LeveldbPersistenceQueryDocSpec.scala | 4 +- .../query/PersistenceQueryDocSpec.scala | 4 +- .../remoting/RemoteDeploymentDocSpec.scala | 4 +- .../ConsistentHashingRouterDocSpec.scala | 4 +- .../docs/routing/CustomRouterDocSpec.scala | 4 +- .../scala/docs/routing/RouterDocSpec.scala | 2 +- .../serialization/SerializationDocSpec.scala | 2 +- .../scala/docs/stream/BidiFlowDocSpec.scala | 4 +- .../docs/stream/CompositionDocSpec.scala | 4 +- .../test/scala/docs/stream/FlowDocSpec.scala | 4 +- .../scala/docs/stream/FlowErrorDocSpec.scala | 4 +- .../docs/stream/FlowParallelismDocSpec.scala | 4 +- .../docs/stream/FlowStreamRefsDocSpec.scala | 4 +- .../scala/docs/stream/GraphCyclesSpec.scala | 4 +- .../scala/docs/stream/GraphDSLDocSpec.scala | 4 +- .../scala/docs/stream/GraphStageDocSpec.scala | 4 +- .../stream/GraphStageLoggingDocSpec.scala | 4 +- .../test/scala/docs/stream/HubsDocSpec.scala | 4 +- .../docs/stream/IntegrationDocSpec.scala | 4 +- .../scala/docs/stream/KillSwitchDocSpec.scala | 4 +- .../scala/docs/stream/MigrationsScala.scala | 4 +- .../stream/RateTransformationDocSpec.scala | 4 +- .../docs/stream/ReactiveStreamsDocSpec.scala | 4 +- .../scala/docs/stream/RestartDocSpec.scala | 4 +- .../docs/stream/StreamBuffersRateSpec.scala | 4 +- .../stream/StreamPartialGraphDSLDocSpec.scala | 4 +- .../docs/stream/StreamTestKitDocSpec.scala | 4 +- .../scala/docs/stream/SubstreamDocSpec.scala | 4 +- .../TwitterStreamQuickstartDocSpec.scala | 4 +- .../docs/stream/cookbook/RecipeSpec.scala | 4 +- .../docs/stream/io/StreamFileDocSpec.scala | 4 +- .../docs/stream/io/StreamTcpDocSpec.scala | 4 +- .../stream/operators/WithContextSpec.scala | 4 +- .../converters/StreamConvertersToJava.scala | 4 +- .../converters/ToFromJavaIOStreams.scala | 4 +- .../scala/docs/testkit/TestkitDocSpec.scala | 2 +- project/CopyrightHeader.scala | 4 +- project/Jdk9.scala | 8 +- project/{AkkaBuild.scala => PekkoBuild.scala} | 2 +- ...ugin.scala => PekkoDisciplinePlugin.scala} | 2 +- project/ValidatePullRequest.scala | 16 +- 584 files changed, 1950 insertions(+), 1947 deletions(-) delete mode 100644 akka-actor-tests/src/test/scala/org/apache/pekko/AkkaVersionSpec.scala rename akka-actor-tests/src/test/scala/org/apache/pekko/{AkkaExceptionSpec.scala => PekkoExceptionSpec.scala} (60%) create mode 100644 akka-actor-tests/src/test/scala/org/apache/pekko/PekkoVersionSpec.scala rename akka-actor-tests/src/test/scala/other/{SerializerOutsideAkkaPackage.scala => SerializerOutsidePekkoPackage.scala} (87%) rename akka-actor/src/main/scala/org/apache/pekko/{AkkaException.scala => PekkoException.scala} (81%) rename akka-actor/src/main/scala/org/apache/pekko/{AkkaVersion.scala => PekkoVersion.scala} (76%) rename akka-cluster-typed/src/main/scala/org/apache/pekko/cluster/typed/internal/{AkkaClusterTypedSerializer.scala => PekkoClusterTypedSerializer.scala} (97%) rename akka-cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/internal/{AkkaClusterTypedSerializerSpec.scala => PekkoClusterTypedSerializerSpec.scala} (80%) rename akka-remote/src/main/scala/org/apache/pekko/remote/transport/{AkkaPduCodec.scala => PekkoPduCodec.scala} (84%) rename akka-remote/src/main/scala/org/apache/pekko/remote/transport/{AkkaProtocolTransport.scala => PekkoProtocolTransport.scala} (91%) rename akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/{AkkaProtocolSpec.scala => PekkoProtocolSpec.scala} (93%) rename akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/{AkkaProtocolStressTest.scala => PekkoProtocolStressTest.scala} (93%) delete mode 100644 akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/AkkaJacksonModule.scala create mode 100644 akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/PekkoJacksonModule.scala rename akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/{AkkaIdentityProcessorVerification.scala => PekkoIdentityProcessorVerification.scala} (94%) rename akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/{AkkaPublisherVerification.scala => PekkoPublisherVerification.scala} (93%) rename akka-stream/src/main/scala/com/typesafe/sslconfig/pekko/{AkkaSSLConfig.scala => PekkoSSLConfig.scala} (80%) rename akka-stream/src/main/scala/com/typesafe/sslconfig/pekko/util/{AkkaLoggerBridge.scala => PekkoLoggerBridge.scala} (72%) rename akka-testkit/src/test/java/org/apache/pekko/testkit/{AkkaJUnitActorSystemResource.java => PekkoJUnitActorSystemResource.java} (79%) rename akka-testkit/src/test/scala/org/apache/pekko/testkit/{AkkaSpec.scala => PekkoSpec.scala} (90%) rename akka-testkit/src/test/scala/org/apache/pekko/testkit/{AkkaSpecSpec.scala => PekkoSpecSpec.scala} (84%) rename akka-testkit/src/test/scala/org/apache/pekko/testkit/metrics/reporter/{AkkaConsoleReporter.scala => PekkoConsoleReporter.scala} (98%) rename project/{AkkaBuild.scala => PekkoBuild.scala} (99%) rename project/{AkkaDisciplinePlugin.scala => PekkoDisciplinePlugin.scala} (99%) diff --git a/akka-actor-testkit-typed/src/main/scala/org/apache/pekko/actor/testkit/typed/internal/TestAppender.scala b/akka-actor-testkit-typed/src/main/scala/org/apache/pekko/actor/testkit/typed/internal/TestAppender.scala index c1a00e97bc..ce117fca53 100644 --- a/akka-actor-testkit-typed/src/main/scala/org/apache/pekko/actor/testkit/typed/internal/TestAppender.scala +++ b/akka-actor-testkit-typed/src/main/scala/org/apache/pekko/actor/testkit/typed/internal/TestAppender.scala @@ -24,7 +24,7 @@ import pekko.annotation.InternalApi @InternalApi private[pekko] object TestAppender { import LogbackUtil._ - private val TestAppenderName = "AkkaTestAppender" + private val TestAppenderName = "PekkoTestAppender" def setupTestAppender(loggerName: String): Unit = { val logbackLogger = getLogbackLogger(loggerName) diff --git a/akka-actor-tests/src/test/java/org/apache/pekko/actor/AbstractFSMActorTest.java b/akka-actor-tests/src/test/java/org/apache/pekko/actor/AbstractFSMActorTest.java index 5d8d851728..69432081c0 100644 --- a/akka-actor-tests/src/test/java/org/apache/pekko/actor/AbstractFSMActorTest.java +++ b/akka-actor-tests/src/test/java/org/apache/pekko/actor/AbstractFSMActorTest.java @@ -4,8 +4,8 @@ package org.apache.pekko.actor; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoSpec; import org.apache.pekko.testkit.TestProbe; import org.junit.ClassRule; import org.junit.Test; @@ -34,8 +34,8 @@ public class AbstractFSMActorTest extends JUnitSuite { } @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("AbstractFSMActorTest", AkkaSpec.testConf()); + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("AbstractFSMActorTest", PekkoSpec.testConf()); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/akka-actor-tests/src/test/java/org/apache/pekko/actor/ActorSelectionTest.java b/akka-actor-tests/src/test/java/org/apache/pekko/actor/ActorSelectionTest.java index 73ba340ec7..1592b6d135 100644 --- a/akka-actor-tests/src/test/java/org/apache/pekko/actor/ActorSelectionTest.java +++ b/akka-actor-tests/src/test/java/org/apache/pekko/actor/ActorSelectionTest.java @@ -8,8 +8,8 @@ import java.time.Duration; import java.util.concurrent.CompletionStage; import java.util.concurrent.TimeUnit; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoSpec; import org.junit.ClassRule; import org.junit.Test; @@ -20,8 +20,8 @@ import static org.junit.Assert.assertEquals; public class ActorSelectionTest extends JUnitSuite { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("ActorSelectionTest", AkkaSpec.testConf()); + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("ActorSelectionTest", PekkoSpec.testConf()); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/akka-actor-tests/src/test/java/org/apache/pekko/actor/ActorSystemTest.java b/akka-actor-tests/src/test/java/org/apache/pekko/actor/ActorSystemTest.java index e095ecd2a0..03e5a4a2e3 100644 --- a/akka-actor-tests/src/test/java/org/apache/pekko/actor/ActorSystemTest.java +++ b/akka-actor-tests/src/test/java/org/apache/pekko/actor/ActorSystemTest.java @@ -4,7 +4,7 @@ package org.apache.pekko.actor; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -18,8 +18,8 @@ import static org.junit.Assert.assertFalse; public class ActorSystemTest extends JUnitSuite { @Rule - public final AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("ActorSystemTest"); + public final PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("ActorSystemTest"); private ActorSystem system = null; diff --git a/akka-actor-tests/src/test/java/org/apache/pekko/actor/JavaAPI.java b/akka-actor-tests/src/test/java/org/apache/pekko/actor/JavaAPI.java index 695ec63541..c2ab6c0ff8 100644 --- a/akka-actor-tests/src/test/java/org/apache/pekko/actor/JavaAPI.java +++ b/akka-actor-tests/src/test/java/org/apache/pekko/actor/JavaAPI.java @@ -14,8 +14,8 @@ import org.apache.pekko.japi.tuple.Tuple4; import org.apache.pekko.routing.GetRoutees; import org.apache.pekko.routing.FromConfig; import org.apache.pekko.routing.NoRouter; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoSpec; import org.apache.pekko.testkit.TestProbe; import org.junit.ClassRule; @@ -30,8 +30,8 @@ import static org.junit.Assert.*; public class JavaAPI extends JUnitSuite { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("JavaAPI", AkkaSpec.testConf()); + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("JavaAPI", PekkoSpec.testConf()); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/akka-actor-tests/src/test/java/org/apache/pekko/actor/JavaExtension.java b/akka-actor-tests/src/test/java/org/apache/pekko/actor/JavaExtension.java index 94a04a5017..d76b406cd7 100644 --- a/akka-actor-tests/src/test/java/org/apache/pekko/actor/JavaExtension.java +++ b/akka-actor-tests/src/test/java/org/apache/pekko/actor/JavaExtension.java @@ -4,9 +4,9 @@ package org.apache.pekko.actor; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import org.junit.*; -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoSpec; import com.typesafe.config.ConfigFactory; import org.scalatestplus.junit.JUnitSuite; @@ -62,12 +62,12 @@ public class JavaExtension extends JUnitSuite { } @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource( + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource( "JavaExtension", ConfigFactory.parseString( "pekko.extensions = [ \"org.apache.pekko.actor.JavaExtension$TestExtensionId\" ]") - .withFallback(AkkaSpec.testConf())); + .withFallback(PekkoSpec.testConf())); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/akka-actor-tests/src/test/java/org/apache/pekko/actor/StashJavaAPI.java b/akka-actor-tests/src/test/java/org/apache/pekko/actor/StashJavaAPI.java index 93465b82ab..51b2136d1e 100644 --- a/akka-actor-tests/src/test/java/org/apache/pekko/actor/StashJavaAPI.java +++ b/akka-actor-tests/src/test/java/org/apache/pekko/actor/StashJavaAPI.java @@ -4,7 +4,7 @@ package org.apache.pekko.actor; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import org.apache.pekko.testkit.TestProbe; import org.junit.ClassRule; @@ -14,8 +14,8 @@ import org.scalatestplus.junit.JUnitSuite; public class StashJavaAPI extends JUnitSuite { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("StashJavaAPI", ActorWithBoundedStashSpec.testConf()); + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("StashJavaAPI", ActorWithBoundedStashSpec.testConf()); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/akka-actor-tests/src/test/java/org/apache/pekko/dispatch/JavaFutureTests.java b/akka-actor-tests/src/test/java/org/apache/pekko/dispatch/JavaFutureTests.java index f407072e1e..f66f7f85cf 100644 --- a/akka-actor-tests/src/test/java/org/apache/pekko/dispatch/JavaFutureTests.java +++ b/akka-actor-tests/src/test/java/org/apache/pekko/dispatch/JavaFutureTests.java @@ -4,7 +4,7 @@ package org.apache.pekko.dispatch; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import org.apache.pekko.actor.ActorSystem; import org.apache.pekko.japi.*; @@ -25,13 +25,13 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import static org.apache.pekko.japi.Util.classTag; -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoSpec; public class JavaFutureTests extends JUnitSuite { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("JavaFutureTests", AkkaSpec.testConf()); + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("JavaFutureTests", PekkoSpec.testConf()); private final ActorSystem system = actorSystemResource.getSystem(); private final Duration timeout = Duration.create(5, TimeUnit.SECONDS); diff --git a/akka-actor-tests/src/test/java/org/apache/pekko/event/LoggingAdapterTest.java b/akka-actor-tests/src/test/java/org/apache/pekko/event/LoggingAdapterTest.java index 51a9897f88..adf629d70f 100644 --- a/akka-actor-tests/src/test/java/org/apache/pekko/event/LoggingAdapterTest.java +++ b/akka-actor-tests/src/test/java/org/apache/pekko/event/LoggingAdapterTest.java @@ -7,7 +7,7 @@ package org.apache.pekko.event; import org.apache.pekko.actor.ActorRef; import org.apache.pekko.actor.ActorSystem; import org.apache.pekko.actor.Props; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import org.apache.pekko.event.Logging.Error; import org.apache.pekko.event.ActorWithMDC.Log; import static org.apache.pekko.event.Logging.*; @@ -33,8 +33,8 @@ public class LoggingAdapterTest extends JUnitSuite { private static final Config config = ConfigFactory.parseString("pekko.loglevel = DEBUG\n"); @Rule - public AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("LoggingAdapterTest", config); + public PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("LoggingAdapterTest", config); private ActorSystem system = null; diff --git a/akka-actor-tests/src/test/java/org/apache/pekko/pattern/CircuitBreakerTest.java b/akka-actor-tests/src/test/java/org/apache/pekko/pattern/CircuitBreakerTest.java index e45f19f72d..24d375cb0d 100644 --- a/akka-actor-tests/src/test/java/org/apache/pekko/pattern/CircuitBreakerTest.java +++ b/akka-actor-tests/src/test/java/org/apache/pekko/pattern/CircuitBreakerTest.java @@ -5,8 +5,8 @@ package org.apache.pekko.pattern; import org.apache.pekko.actor.*; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoSpec; import org.apache.pekko.util.JavaDurationConverters; import org.junit.ClassRule; import org.junit.Test; @@ -25,8 +25,8 @@ import static org.junit.Assert.assertEquals; public class CircuitBreakerTest extends JUnitSuite { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("JavaAPI", AkkaSpec.testConf()); + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("JavaAPI", PekkoSpec.testConf()); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/akka-actor-tests/src/test/java/org/apache/pekko/pattern/PatternsTest.java b/akka-actor-tests/src/test/java/org/apache/pekko/pattern/PatternsTest.java index 9672ce5372..17c4f577e9 100644 --- a/akka-actor-tests/src/test/java/org/apache/pekko/pattern/PatternsTest.java +++ b/akka-actor-tests/src/test/java/org/apache/pekko/pattern/PatternsTest.java @@ -6,8 +6,8 @@ package org.apache.pekko.pattern; import org.apache.pekko.actor.*; import org.apache.pekko.dispatch.Futures; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoSpec; import org.apache.pekko.testkit.TestProbe; import org.apache.pekko.util.Timeout; import org.junit.ClassRule; @@ -62,8 +62,8 @@ public class PatternsTest extends JUnitSuite { } @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("JavaAPI", AkkaSpec.testConf()); + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("JavaAPI", PekkoSpec.testConf()); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/akka-actor-tests/src/test/java/org/apache/pekko/pattern/StatusReplyTest.java b/akka-actor-tests/src/test/java/org/apache/pekko/pattern/StatusReplyTest.java index cad3454dbb..eff7084f60 100644 --- a/akka-actor-tests/src/test/java/org/apache/pekko/pattern/StatusReplyTest.java +++ b/akka-actor-tests/src/test/java/org/apache/pekko/pattern/StatusReplyTest.java @@ -5,8 +5,8 @@ package org.apache.pekko.pattern; import org.apache.pekko.actor.ActorRef; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoSpec; import org.apache.pekko.testkit.TestException; import org.apache.pekko.testkit.TestProbe; import org.junit.Assert; @@ -26,8 +26,8 @@ import static org.junit.Assert.*; public class StatusReplyTest extends JUnitSuite { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("JavaAPI", AkkaSpec.testConf()); + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("JavaAPI", PekkoSpec.testConf()); @Test public void successReplyThrowsExceptionWhenGetErrorIsCalled() { diff --git a/akka-actor-tests/src/test/scala-2.12/org/apache/pekko/util/LineNumberSpec.scala b/akka-actor-tests/src/test/scala-2.12/org/apache/pekko/util/LineNumberSpec.scala index 54d1a9c326..440ce96b28 100644 --- a/akka-actor-tests/src/test/scala-2.12/org/apache/pekko/util/LineNumberSpec.scala +++ b/akka-actor-tests/src/test/scala-2.12/org/apache/pekko/util/LineNumberSpec.scala @@ -5,10 +5,10 @@ package org.apache.pekko.util import org.apache.pekko -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.util.LineNumbers._ -class LineNumberSpec extends AkkaSpec { +class LineNumberSpec extends PekkoSpec { "LineNumbers" when { diff --git a/akka-actor-tests/src/test/scala-2.13/org/apache/pekko/util/LineNumberSpec.scala b/akka-actor-tests/src/test/scala-2.13/org/apache/pekko/util/LineNumberSpec.scala index 54d1a9c326..440ce96b28 100644 --- a/akka-actor-tests/src/test/scala-2.13/org/apache/pekko/util/LineNumberSpec.scala +++ b/akka-actor-tests/src/test/scala-2.13/org/apache/pekko/util/LineNumberSpec.scala @@ -5,10 +5,10 @@ package org.apache.pekko.util import org.apache.pekko -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.util.LineNumbers._ -class LineNumberSpec extends AkkaSpec { +class LineNumberSpec extends PekkoSpec { "LineNumbers" when { diff --git a/akka-actor-tests/src/test/scala-3/org/apache/pekko/util/LineNumberSpec.scala b/akka-actor-tests/src/test/scala-3/org/apache/pekko/util/LineNumberSpec.scala index 9718ee1a30..03c3f4cd3c 100644 --- a/akka-actor-tests/src/test/scala-3/org/apache/pekko/util/LineNumberSpec.scala +++ b/akka-actor-tests/src/test/scala-3/org/apache/pekko/util/LineNumberSpec.scala @@ -5,10 +5,10 @@ package org.apache.pekko.util import org.apache.pekko -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.util.LineNumbers._ -class LineNumberSpec extends AkkaSpec { +class LineNumberSpec extends PekkoSpec { "LineNumbers" when { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/AkkaVersionSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/AkkaVersionSpec.scala deleted file mode 100644 index 52e9c22a4d..0000000000 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/AkkaVersionSpec.scala +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Copyright (C) 2009-2022 Lightbend Inc. - */ - -package org.apache.pekko - -import org.scalatest.matchers.should.Matchers -import org.scalatest.wordspec.AnyWordSpec - -class AkkaVersionSpec extends AnyWordSpec with Matchers { - - "The Akka version check" must { - - "succeed if version is ok" in { - AkkaVersion.require("AkkaVersionSpec", "2.5.6", "2.5.6") - AkkaVersion.require("AkkaVersionSpec", "2.5.6", "2.5.7") - AkkaVersion.require("AkkaVersionSpec", "2.5.6", "2.6.0") - } - - "succeed if version is RC and ok" in { - AkkaVersion.require("AkkaVersionSpec", "2.5.6", "2.5.7-RC10") - AkkaVersion.require("AkkaVersionSpec", "2.6.0-RC1", "2.6.0-RC1") - } - - "fail if version is RC and not ok" in { - intercept[UnsupportedAkkaVersion] { - AkkaVersion.require("AkkaVersionSpec", "2.5.6", "2.5.6-RC1") - } - } - - "succeed if version is milestone and ok" in { - AkkaVersion.require("AkkaVersionSpec", "2.5.6", "2.5.7-M10") - } - - "fail if version is milestone and not ok" in { - intercept[UnsupportedAkkaVersion] { - AkkaVersion.require("AkkaVersionSpec", "2.5.6", "2.5.6-M1") - } - } - - "fail if major version is different" in { - // because not bincomp - intercept[UnsupportedAkkaVersion] { - AkkaVersion.require("AkkaVersionSpec", "2.5.6", "3.0.0") - } - intercept[UnsupportedAkkaVersion] { - AkkaVersion.require("AkkaVersionSpec", "2.5.6", "1.0.0") - } - } - - "fail if minor version is too low" in { - intercept[UnsupportedAkkaVersion] { - AkkaVersion.require("AkkaVersionSpec", "2.5.6", "2.4.19") - } - } - - "fail if patch version is too low" in { - intercept[UnsupportedAkkaVersion] { - AkkaVersion.require("AkkaVersionSpec", "2.5.6", "2.5.5") - } - } - - "succeed if current Akka version is SNAPSHOT" in { - AkkaVersion.require("AkkaVersionSpec", "2.5.6", "2.5-SNAPSHOT") - } - - "succeed if current Akka version is timestamped SNAPSHOT" in { - AkkaVersion.require("AkkaVersionSpec", "2.5.6", "2.5-20180109-133700") - } - - "succeed if required Akka version is SNAPSHOT" in { - AkkaVersion.require("AkkaVersionSpec", "2.5-SNAPSHOT", "2.5-SNAPSHOT") - } - - "succeed if required Akka version is timestamped SNAPSHOT" in { - AkkaVersion.require("AkkaVersionSpec", "2.5-20180109-133700", "2.5-20180109-133700") - } - - "silently comply if current version is incomprehensible" in { - // because we may want to release with weird numbers for some reason - AkkaVersion.require("nonsense", "2.5.6", "nonsense") - } - - } - -} diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/AkkaExceptionSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/PekkoExceptionSpec.scala similarity index 60% rename from akka-actor-tests/src/test/scala/org/apache/pekko/AkkaExceptionSpec.scala rename to akka-actor-tests/src/test/scala/org/apache/pekko/PekkoExceptionSpec.scala index f6e298c187..e98ef473eb 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/AkkaExceptionSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/PekkoExceptionSpec.scala @@ -10,17 +10,17 @@ import org.scalatest.wordspec.AnyWordSpec import org.apache.pekko.actor._ /** - * A spec that verified that the AkkaException has at least a single argument constructor of type String. + * A spec that verified that the PekkoException has at least a single argument constructor of type String. * - * This is required to make Akka Exceptions be friends with serialization/deserialization. + * This is required to make Pekko Exceptions be friends with serialization/deserialization. */ -class AkkaExceptionSpec extends AnyWordSpec with Matchers { +class PekkoExceptionSpec extends AnyWordSpec with Matchers { - "AkkaException" must { - "have a AkkaException(String msg) constructor to be serialization friendly" in { + "PekkoException" must { + "have a PekkoException(String msg) constructor to be serialization friendly" in { // if the call to this method completes, we know what there is at least a single constructor which has // the expected argument type. - verify(classOf[AkkaException]) + verify(classOf[PekkoException]) // lets also try it for the exception that triggered this bug to be discovered. verify(classOf[ActorKilledException]) diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/PekkoVersionSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/PekkoVersionSpec.scala new file mode 100644 index 0000000000..fff3295464 --- /dev/null +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/PekkoVersionSpec.scala @@ -0,0 +1,86 @@ +/* + * Copyright (C) 2009-2022 Lightbend Inc. + */ + +package org.apache.pekko + +import org.scalatest.matchers.should.Matchers +import org.scalatest.wordspec.AnyWordSpec + +class PekkoVersionSpec extends AnyWordSpec with Matchers { + + "The Pekko version check" must { + + "succeed if version is ok" in { + PekkoVersion.require("PekkoVersionSpec", "2.5.6", "2.5.6") + PekkoVersion.require("PekkoVersionSpec", "2.5.6", "2.5.7") + PekkoVersion.require("PekkoVersionSpec", "2.5.6", "2.6.0") + } + + "succeed if version is RC and ok" in { + PekkoVersion.require("PekkoVersionSpec", "2.5.6", "2.5.7-RC10") + PekkoVersion.require("PekkoVersionSpec", "2.6.0-RC1", "2.6.0-RC1") + } + + "fail if version is RC and not ok" in { + intercept[UnsupportedPekkoVersion] { + PekkoVersion.require("PekkoVersionSpec", "2.5.6", "2.5.6-RC1") + } + } + + "succeed if version is milestone and ok" in { + PekkoVersion.require("PekkoVersionSpec", "2.5.6", "2.5.7-M10") + } + + "fail if version is milestone and not ok" in { + intercept[UnsupportedPekkoVersion] { + PekkoVersion.require("PekkoVersionSpec", "2.5.6", "2.5.6-M1") + } + } + + "fail if major version is different" in { + // because not bincomp + intercept[UnsupportedPekkoVersion] { + PekkoVersion.require("PekkoVersionSpec", "2.5.6", "3.0.0") + } + intercept[UnsupportedPekkoVersion] { + PekkoVersion.require("PekkoVersionSpec", "2.5.6", "1.0.0") + } + } + + "fail if minor version is too low" in { + intercept[UnsupportedPekkoVersion] { + PekkoVersion.require("PekkoVersionSpec", "2.5.6", "2.4.19") + } + } + + "fail if patch version is too low" in { + intercept[UnsupportedPekkoVersion] { + PekkoVersion.require("PekkoVersionSpec", "2.5.6", "2.5.5") + } + } + + "succeed if current Pekko version is SNAPSHOT" in { + PekkoVersion.require("PekkoVersionSpec", "2.5.6", "2.5-SNAPSHOT") + } + + "succeed if current Pekko version is timestamped SNAPSHOT" in { + PekkoVersion.require("PekkoVersionSpec", "2.5.6", "2.5-20180109-133700") + } + + "succeed if required Pekko version is SNAPSHOT" in { + PekkoVersion.require("PekkoVersionSpec", "2.5-SNAPSHOT", "2.5-SNAPSHOT") + } + + "succeed if required Pekko version is timestamped SNAPSHOT" in { + PekkoVersion.require("PekkoVersionSpec", "2.5-20180109-133700", "2.5-20180109-133700") + } + + "silently comply if current version is incomprehensible" in { + // because we may want to release with weird numbers for some reason + PekkoVersion.require("nonsense", "2.5.6", "nonsense") + } + + } + +} diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorConfigurationVerificationSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorConfigurationVerificationSpec.scala index dd4d0dfb76..6f65efcd41 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorConfigurationVerificationSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorConfigurationVerificationSpec.scala @@ -37,7 +37,7 @@ object ActorConfigurationVerificationSpec { } class ActorConfigurationVerificationSpec - extends AkkaSpec(ActorConfigurationVerificationSpec.config) + extends PekkoSpec(ActorConfigurationVerificationSpec.config) with DefaultTimeout with BeforeAndAfterEach { import ActorConfigurationVerificationSpec._ diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorCreationPerfSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorCreationPerfSpec.scala index 2fce01491f..19dfcc3827 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorCreationPerfSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorCreationPerfSpec.scala @@ -12,7 +12,7 @@ import com.typesafe.config.ConfigFactory import org.scalatest.BeforeAndAfterAll import org.apache.pekko -import pekko.testkit.{ AkkaSpec, ImplicitSender, PerformanceTest } +import pekko.testkit.{ ImplicitSender, PekkoSpec, PerformanceTest } import pekko.testkit.metrics._ import pekko.testkit.metrics.HeapMemoryUsage @@ -118,7 +118,7 @@ object ActorCreationPerfSpec { } class ActorCreationPerfSpec - extends AkkaSpec(ActorCreationPerfSpec.config) + extends PekkoSpec(ActorCreationPerfSpec.config) with ImplicitSender with MetricsKit with BeforeAndAfterAll { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorLifeCycleSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorLifeCycleSpec.scala index a71e4049d3..ce8b5cc399 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorLifeCycleSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorLifeCycleSpec.scala @@ -30,7 +30,7 @@ object ActorLifeCycleSpec { } -class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSender with DefaultTimeout { +class ActorLifeCycleSpec extends PekkoSpec with BeforeAndAfterEach with ImplicitSender with DefaultTimeout { import ActorLifeCycleSpec._ "An Actor" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorMailboxSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorMailboxSpec.scala index 705a85fe98..ca0bb6d895 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorMailboxSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorMailboxSpec.scala @@ -226,7 +226,7 @@ object ActorMailboxSpec { } -class ActorMailboxSpec(conf: Config) extends AkkaSpec(conf) with DefaultTimeout with ImplicitSender { +class ActorMailboxSpec(conf: Config) extends PekkoSpec(conf) with DefaultTimeout with ImplicitSender { import ActorMailboxSpec._ diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorRefSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorRefSpec.scala index 5bdd982ef9..0e6201140f 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorRefSpec.scala @@ -112,7 +112,7 @@ object ActorRefSpec { } } -class ActorRefSpec extends AkkaSpec(""" +class ActorRefSpec extends PekkoSpec(""" # testing Java serialization of ActorRef pekko.actor.allow-java-serialization = on """) with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorSelectionSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorSelectionSpec.scala index 595a5c9a89..47bb044277 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorSelectionSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorSelectionSpec.scala @@ -36,7 +36,7 @@ object ActorSelectionSpec { } -class ActorSelectionSpec extends AkkaSpec with DefaultTimeout { +class ActorSelectionSpec extends PekkoSpec with DefaultTimeout { import ActorSelectionSpec._ val c1 = system.actorOf(p, "c1") diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorSystemDispatcherSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorSystemDispatcherSpec.scala index 0c6be11882..a01b9bf76b 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorSystemDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorSystemDispatcherSpec.scala @@ -13,7 +13,7 @@ import org.apache.pekko import pekko.ConfigurationException import pekko.actor.setup.ActorSystemSetup import pekko.dispatch.{ Dispatchers, ExecutionContexts } -import pekko.testkit.{ AkkaSpec, ImplicitSender, TestActors, TestProbe } +import pekko.testkit.{ ImplicitSender, PekkoSpec, TestActors, TestProbe } object ActorSystemDispatchersSpec { @@ -32,7 +32,7 @@ object ActorSystemDispatchersSpec { } -class ActorSystemDispatchersSpec extends AkkaSpec(ConfigFactory.parseString(""" +class ActorSystemDispatchersSpec extends PekkoSpec(ConfigFactory.parseString(""" dispatcher-loop-1 = "dispatcher-loop-2" dispatcher-loop-2 = "dispatcher-loop-1" """)) with ImplicitSender { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorSystemSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorSystemSpec.scala index 298a6fe3a2..8e920d0f79 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorSystemSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorSystemSpec.scala @@ -116,7 +116,7 @@ object ActorSystemSpec { } @nowarn -class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSender { +class ActorSystemSpec extends PekkoSpec(ActorSystemSpec.config) with ImplicitSender { import ActorSystemSpec.FastActor @@ -142,7 +142,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend "log dead letters" in { val sys = - ActorSystem("LogDeadLetters", ConfigFactory.parseString("pekko.loglevel=INFO").withFallback(AkkaSpec.testConf)) + ActorSystem("LogDeadLetters", ConfigFactory.parseString("pekko.loglevel=INFO").withFallback(PekkoSpec.testConf)) try { val probe = TestProbe()(sys) val a = sys.actorOf(Props[ActorSystemSpec.Terminater]()) @@ -166,7 +166,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend "log dead letters sent without sender reference" in { val sys = - ActorSystem("LogDeadLetters", ConfigFactory.parseString("pekko.loglevel=INFO").withFallback(AkkaSpec.testConf)) + ActorSystem("LogDeadLetters", ConfigFactory.parseString("pekko.loglevel=INFO").withFallback(PekkoSpec.testConf)) try { val probe = TestProbe()(sys) val a = sys.actorOf(Props[ActorSystemSpec.Terminater]()) @@ -189,7 +189,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend } "run termination callbacks in order" in { - val system2 = ActorSystem("TerminationCallbacks", AkkaSpec.testConf) + val system2 = ActorSystem("TerminationCallbacks", PekkoSpec.testConf) val result = new ConcurrentLinkedQueue[Int] val count = 10 val latch = TestLatch(count) @@ -211,7 +211,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend } "awaitTermination after termination callbacks" in { - val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf) + val system2 = ActorSystem("AwaitTermination", PekkoSpec.testConf) @volatile var callbackWasRun = false @@ -240,7 +240,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend } "throw RejectedExecutionException when shutdown" in { - val system2 = ActorSystem("RejectedExecution-1", AkkaSpec.testConf) + val system2 = ActorSystem("RejectedExecution-1", PekkoSpec.testConf) Await.ready(system2.terminate(), 10 seconds) intercept[RejectedExecutionException] { @@ -249,7 +249,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend } "throw RejectedExecutionException or run termination callback when shutting down" in { - val system2 = ActorSystem("RejectedExecution-2", AkkaSpec.testConf) + val system2 = ActorSystem("RejectedExecution-2", PekkoSpec.testConf) // using counter to detect double calls val count = new AtomicInteger @@ -305,7 +305,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend } "shut down when /user fails" in { - implicit val system = ActorSystem("Stop", AkkaSpec.testConf) + implicit val system = ActorSystem("Stop", PekkoSpec.testConf) EventFilter[ActorKilledException]().intercept { system.actorSelection("/user") ! Kill Await.ready(system.whenTerminated, Duration.Inf) @@ -318,7 +318,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend "Stop", ConfigFactory .parseString("pekko.actor.guardian-supervisor-strategy=org.apache.pekko.actor.StoppingSupervisorStrategy") - .withFallback(AkkaSpec.testConf)) + .withFallback(PekkoSpec.testConf)) val a = system.actorOf(Props(new Actor { def receive = { case "die" => throw new Exception("hello") @@ -341,7 +341,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend "Stop", ConfigFactory .parseString("pekko.actor.guardian-supervisor-strategy=\"org.apache.pekko.actor.ActorSystemSpec$Strategy\"") - .withFallback(AkkaSpec.testConf)) + .withFallback(PekkoSpec.testConf)) val a = system.actorOf(Props(new Actor { def receive = { case "die" => throw new Exception("hello") diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorTimeoutSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorTimeoutSpec.scala index 6596a9399c..4605eafc25 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorTimeoutSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorTimeoutSpec.scala @@ -13,7 +13,7 @@ import pekko.testkit._ import pekko.testkit.TestEvent._ import pekko.util.Timeout -class ActorTimeoutSpec extends AkkaSpec { +class ActorTimeoutSpec extends PekkoSpec { val testTimeout = 200.millis.dilated val leeway = 500.millis.dilated diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorWithBoundedStashSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorWithBoundedStashSpec.scala index 3524bfdf74..76500fd0aa 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorWithBoundedStashSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorWithBoundedStashSpec.scala @@ -92,7 +92,7 @@ object ActorWithBoundedStashSpec { } class ActorWithBoundedStashSpec - extends AkkaSpec(ActorWithBoundedStashSpec.testConf) + extends PekkoSpec(ActorWithBoundedStashSpec.testConf) with BeforeAndAfterEach with DefaultTimeout with ImplicitSender { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorWithStashSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorWithStashSpec.scala index 93caf8e449..f9b1aca22a 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorWithStashSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ActorWithStashSpec.scala @@ -99,7 +99,7 @@ object ActorWithStashSpec { } @nowarn -class ActorWithStashSpec extends AkkaSpec with DefaultTimeout with BeforeAndAfterEach { +class ActorWithStashSpec extends PekkoSpec with DefaultTimeout with BeforeAndAfterEach { import ActorWithStashSpec._ override def atStartup(): Unit = { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ConsistencySpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ConsistencySpec.scala index c4adb55f3c..5ef6d5e32b 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ConsistencySpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ConsistencySpec.scala @@ -10,7 +10,7 @@ import language.postfixOps import org.apache.pekko import pekko.dispatch.ThreadPoolConfig -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec object ConsistencySpec { val minThreads = 1 @@ -54,12 +54,12 @@ object ConsistencySpec { } } -class ConsistencySpec extends AkkaSpec(ConsistencySpec.config) { +class ConsistencySpec extends PekkoSpec(ConsistencySpec.config) { import ConsistencySpec._ override def expectedTestDuration: FiniteDuration = 5.minutes - "The Akka actor model implementation" must { + "The Pekko actor model implementation" must { "provide memory consistency" in { val noOfActors = threads + 1 val props = Props[ConsistencyCheckingActor]().withDispatcher("consistency-dispatcher") diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/CoordinatedShutdownSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/CoordinatedShutdownSpec.scala index 4d82d0cee0..b77ec65831 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/CoordinatedShutdownSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/CoordinatedShutdownSpec.scala @@ -10,7 +10,7 @@ import pekko.Done import pekko.actor.CoordinatedShutdown.Phase import pekko.actor.CoordinatedShutdown.UnknownReason import pekko.dispatch.ExecutionContexts -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.EventFilter import pekko.testkit.TestKit import pekko.testkit.TestProbe @@ -26,7 +26,7 @@ import scala.concurrent.Future import scala.concurrent.Promise class CoordinatedShutdownSpec - extends AkkaSpec(ConfigFactory.parseString(""" + extends PekkoSpec(ConfigFactory.parseString(""" pekko.loglevel=INFO pekko.loggers = ["org.apache.pekko.testkit.TestEventListener"] """)) { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeadLetterSupressionSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeadLetterSupressionSpec.scala index a886688606..4562b419d7 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeadLetterSupressionSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeadLetterSupressionSpec.scala @@ -8,7 +8,7 @@ import scala.concurrent.duration._ import org.apache.pekko import pekko.event.Logging -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender import pekko.testkit.TestActors import pekko.testkit.TestProbe @@ -21,7 +21,7 @@ object DeadLetterSupressionSpec { } -class DeadLetterSupressionSpec extends AkkaSpec with ImplicitSender { +class DeadLetterSupressionSpec extends PekkoSpec with ImplicitSender { import DeadLetterSupressionSpec._ val deadActor = system.actorOf(TestActors.echoActorProps) diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeadLetterSuspensionSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeadLetterSuspensionSpec.scala index 118f5cdae7..98106e1be7 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeadLetterSuspensionSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeadLetterSuspensionSpec.scala @@ -5,7 +5,7 @@ package org.apache.pekko.actor import org.apache.pekko -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.EventFilter import pekko.testkit.ImplicitSender import pekko.testkit.TestActors @@ -34,7 +34,7 @@ object DeadLetterSuspensionSpec { } } -class DeadLetterSuspensionSpec extends AkkaSpec(""" +class DeadLetterSuspensionSpec extends PekkoSpec(""" pekko.loglevel = INFO pekko.log-dead-letters = 4 pekko.log-dead-letters-suspend-duration = 2s diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeathWatchSpec.scala index 39b8aae04a..c814a119bb 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeathWatchSpec.scala @@ -16,7 +16,7 @@ import pekko.dispatch.sysmsg.{ DeathWatchNotification, Failed } import pekko.pattern.ask import pekko.testkit._ -class LocalDeathWatchSpec extends AkkaSpec with ImplicitSender with DefaultTimeout with DeathWatchSpec +class LocalDeathWatchSpec extends PekkoSpec with ImplicitSender with DefaultTimeout with DeathWatchSpec object DeathWatchSpec { object Watcher { @@ -109,7 +109,7 @@ object DeathWatchSpec { } @nowarn -trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout => +trait DeathWatchSpec { this: PekkoSpec with ImplicitSender with DefaultTimeout => import DeathWatchSpec._ diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeployerSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeployerSpec.scala index bb211d13be..c247ad6ccf 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeployerSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/DeployerSpec.scala @@ -12,7 +12,7 @@ import language.postfixOps import org.apache.pekko import pekko.routing._ -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec object DeployerSpec { val deployerConf = ConfigFactory.parseString( @@ -79,7 +79,7 @@ object DeployerSpec { } -class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) { +class DeployerSpec extends PekkoSpec(DeployerSpec.deployerConf) { "A Deployer" must { "be able to parse 'pekko.actor.deployment._' with all default values" in { @@ -146,7 +146,7 @@ class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) { } """, ConfigParseOptions.defaults) - .withFallback(AkkaSpec.testConf) + .withFallback(PekkoSpec.testConf) shutdown(ActorSystem("invalid-number-of-instances", invalidDeployerConf)) } @@ -165,7 +165,7 @@ class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) { } """, ConfigParseOptions.defaults) - .withFallback(AkkaSpec.testConf) + .withFallback(PekkoSpec.testConf) shutdown(ActorSystem("invalid-path", invalidDeployerConf)) } diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FSMActorSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FSMActorSpec.scala index 2427df4c87..a521e78883 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FSMActorSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FSMActorSpec.scala @@ -101,7 +101,7 @@ object FSMActorSpec { final case class CodeState(soFar: String, code: String) } -class FSMActorSpec extends AkkaSpec(Map("pekko.actor.debug.fsm" -> true)) with ImplicitSender { +class FSMActorSpec extends PekkoSpec(Map("pekko.actor.debug.fsm" -> true)) with ImplicitSender { import FSMActorSpec._ val timeout = Timeout(2 seconds) diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FSMTimingSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FSMTimingSpec.scala index 50b9e44fb7..621abda129 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FSMTimingSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FSMTimingSpec.scala @@ -9,7 +9,7 @@ import language.postfixOps import org.apache.pekko.testkit._ import scala.concurrent.duration._ -class FSMTimingSpec extends AkkaSpec with ImplicitSender { +class FSMTimingSpec extends PekkoSpec with ImplicitSender { import FSMTimingSpec._ import FSM._ diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FSMTransitionSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FSMTransitionSpec.scala index c05cbc9c33..733927a658 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FSMTransitionSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FSMTransitionSpec.scala @@ -63,7 +63,7 @@ object FSMTransitionSpec { } -class FSMTransitionSpec extends AkkaSpec with ImplicitSender { +class FSMTransitionSpec extends PekkoSpec with ImplicitSender { import FSMTransitionSpec._ diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ForwardActorSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ForwardActorSpec.scala index 2022746c86..87368305ea 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ForwardActorSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ForwardActorSpec.scala @@ -30,7 +30,7 @@ object ForwardActorSpec { } } -class ForwardActorSpec extends AkkaSpec { +class ForwardActorSpec extends PekkoSpec { import ForwardActorSpec._ implicit val ec: ExecutionContextExecutor = system.dispatcher "A Forward Actor" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FunctionRefSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FunctionRefSpec.scala index e1233dd893..999971cf99 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FunctionRefSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/FunctionRefSpec.scala @@ -6,7 +6,7 @@ package org.apache.pekko.actor import org.apache.pekko import pekko.actor.dungeon.SerializationCheckFailedException -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.EventFilter import pekko.testkit.ImplicitSender @@ -37,7 +37,7 @@ object FunctionRefSpec { } -class FunctionRefSpec extends AkkaSpec(""" +class FunctionRefSpec extends PekkoSpec(""" # test is using Java serialization and relies on serialize-messages=on pekko.actor.allow-java-serialization = on pekko.actor.warn-about-java-serializer-usage = off diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/HotSwapSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/HotSwapSpec.scala index 433409e104..36585fc5e7 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/HotSwapSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/HotSwapSpec.scala @@ -10,7 +10,7 @@ object HotSwapSpec { abstract class Becomer extends Actor {} } -class HotSwapSpec extends AkkaSpec with ImplicitSender { +class HotSwapSpec extends PekkoSpec with ImplicitSender { import HotSwapSpec.Becomer "An Actor" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/LocalActorRefProviderSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/LocalActorRefProviderSpec.scala index f6d54d9b7f..0ff50028fa 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/LocalActorRefProviderSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/LocalActorRefProviderSpec.scala @@ -35,7 +35,7 @@ object LocalActorRefProviderSpec { } @nowarn -class LocalActorRefProviderSpec extends AkkaSpec(LocalActorRefProviderSpec.config) { +class LocalActorRefProviderSpec extends PekkoSpec(LocalActorRefProviderSpec.config) { "An LocalActorRefProvider" must { "find child actor with URL encoded name" in { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/PropsCreationSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/PropsCreationSpec.scala index c1aa8979a0..48716dc4ef 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/PropsCreationSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/PropsCreationSpec.scala @@ -7,7 +7,7 @@ package org.apache.pekko.actor import scala.annotation.nowarn import org.apache.pekko -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.util.unused object PropsCreationSpec { @@ -30,7 +30,7 @@ object PropsCreationSpec { } -class PropsCreationSpec extends AkkaSpec(""" +class PropsCreationSpec extends PekkoSpec(""" # test is using Java serialization and relies on serialize-creators=on pekko.actor.serialize-creators = on pekko.actor.no-serialization-verification-needed-class-prefix = [] diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ReceiveTimeoutSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ReceiveTimeoutSpec.scala index 48f05dbdfe..d78dd41891 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ReceiveTimeoutSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/ReceiveTimeoutSpec.scala @@ -67,7 +67,7 @@ object ReceiveTimeoutSpec { } } -class ReceiveTimeoutSpec extends AkkaSpec() { +class ReceiveTimeoutSpec extends PekkoSpec() { import ReceiveTimeoutSpec._ "An actor with receive timeout" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/RestartStrategySpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/RestartStrategySpec.scala index ed0452617c..e9470961a4 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/RestartStrategySpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/RestartStrategySpec.scala @@ -14,14 +14,14 @@ import language.postfixOps import org.apache.pekko import pekko.pattern.ask -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.DefaultTimeout import pekko.testkit.EventFilter import pekko.testkit.TestEvent._ import pekko.testkit.TestLatch @nowarn -class RestartStrategySpec extends AkkaSpec with DefaultTimeout { +class RestartStrategySpec extends PekkoSpec with DefaultTimeout { override def atStartup(): Unit = { system.eventStream.publish(Mute(EventFilter[Exception]("Crashing..."))) diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SchedulerSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SchedulerSpec.scala index 03c37d2efe..118c04e430 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SchedulerSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SchedulerSpec.scala @@ -29,11 +29,11 @@ object SchedulerSpec { ConfigFactory.parseString(""" pekko.scheduler.implementation = org.apache.pekko.actor.LightArrayRevolverScheduler pekko.scheduler.ticks-per-wheel = 32 - """).withFallback(AkkaSpec.testConf) + """).withFallback(PekkoSpec.testConf) } -trait SchedulerSpec extends BeforeAndAfterEach with DefaultTimeout with ImplicitSender { this: AkkaSpec => +trait SchedulerSpec extends BeforeAndAfterEach with DefaultTimeout with ImplicitSender { this: PekkoSpec => import system.dispatcher def collectCancellable(c: Cancellable): Cancellable @@ -419,7 +419,7 @@ trait SchedulerSpec extends BeforeAndAfterEach with DefaultTimeout with Implicit } } -class LightArrayRevolverSchedulerSpec extends AkkaSpec(SchedulerSpec.testConfRevolver) with SchedulerSpec { +class LightArrayRevolverSchedulerSpec extends PekkoSpec(SchedulerSpec.testConfRevolver) with SchedulerSpec { def collectCancellable(c: Cancellable): Cancellable = c diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorHierarchySpec.scala index 74ed14a14a..e6028815bd 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorHierarchySpec.scala @@ -25,7 +25,7 @@ import pekko.actor.SupervisorStrategy.seqThrowable2Decider import pekko.dispatch.{ Dispatcher, DispatcherConfigurator, DispatcherPrerequisites, MessageDispatcher } import pekko.event.Logging import pekko.pattern.ask -import pekko.testkit.{ AkkaSpec, DefaultTimeout, EventFilter, ImplicitSender } +import pekko.testkit.{ DefaultTimeout, EventFilter, ImplicitSender, PekkoSpec } import pekko.testkit.{ filterEvents, filterException, TestDuration, TestLatch } import pekko.testkit.LongRunningTest import pekko.testkit.TestEvent.Mute @@ -748,7 +748,8 @@ object SupervisorHierarchySpec { } -class SupervisorHierarchySpec extends AkkaSpec(SupervisorHierarchySpec.config) with DefaultTimeout with ImplicitSender { +class SupervisorHierarchySpec extends PekkoSpec(SupervisorHierarchySpec.config) with DefaultTimeout + with ImplicitSender { import SupervisorHierarchySpec._ override def expectedTestDuration = 2.minutes diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorMiscSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorMiscSpec.scala index 7d4a2b23ab..d6fe6f2158 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorMiscSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorMiscSpec.scala @@ -16,7 +16,7 @@ import language.postfixOps import org.apache.pekko import pekko.pattern.ask import pekko.testkit.{ filterEvents, EventFilter } -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.DefaultTimeout object SupervisorMiscSpec { @@ -31,7 +31,7 @@ object SupervisorMiscSpec { } @nowarn -class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with DefaultTimeout { +class SupervisorMiscSpec extends PekkoSpec(SupervisorMiscSpec.config) with DefaultTimeout { "A Supervisor" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorSpec.scala index 3e2b82032e..8540bdd26a 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorSpec.scala @@ -106,7 +106,7 @@ object SupervisorSpec { } class SupervisorSpec - extends AkkaSpec(SupervisorSpec.config) + extends PekkoSpec(SupervisorSpec.config) with BeforeAndAfterEach with ImplicitSender with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorTreeSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorTreeSpec.scala index d217a8a968..947cc2dff6 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorTreeSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/SupervisorTreeSpec.scala @@ -11,9 +11,9 @@ import language.postfixOps import org.apache.pekko import pekko.pattern.ask -import pekko.testkit.{ AkkaSpec, DefaultTimeout, EventFilter, ImplicitSender } +import pekko.testkit.{ DefaultTimeout, EventFilter, ImplicitSender, PekkoSpec } -class SupervisorTreeSpec extends AkkaSpec with ImplicitSender with DefaultTimeout { +class SupervisorTreeSpec extends PekkoSpec with ImplicitSender with DefaultTimeout { "In a 3 levels deep supervisor tree (linked in the constructor) we" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/Ticket669Spec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/Ticket669Spec.scala index 2766be3ccf..fe100741d6 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/Ticket669Spec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/Ticket669Spec.scala @@ -13,11 +13,11 @@ import org.scalatest.BeforeAndAfterAll import org.apache.pekko import pekko.pattern.ask import pekko.testkit.{ filterEvents, EventFilter } -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.DefaultTimeout import pekko.testkit.ImplicitSender -class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender with DefaultTimeout { +class Ticket669Spec extends PekkoSpec with BeforeAndAfterAll with ImplicitSender with DefaultTimeout { import Ticket669Spec._ // TODO: does this really make sense? diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/TimerSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/TimerSpec.scala index 46b76cd421..3ae054e173 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/TimerSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/TimerSpec.scala @@ -177,7 +177,7 @@ class FsmTimerSpec extends AbstractTimerSpec { TimerSpec.fsmTarget(monitor, interval, repeat, initial) } -abstract class AbstractTimerSpec extends AkkaSpec { +abstract class AbstractTimerSpec extends PekkoSpec { import TimerSpec._ val interval = 1.second @@ -312,7 +312,7 @@ object TimersAndStashSpec { case object StopStashing } -class TimersAndStashSpec extends AkkaSpec { +class TimersAndStashSpec extends PekkoSpec { import TimersAndStashSpec._ class ActorWithTimerAndStash(probe: ActorRef) extends Actor with Timers with Stash { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/TypedActorSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/TypedActorSpec.scala index 3c1b2a0500..0d2833a83c 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/TypedActorSpec.scala @@ -22,7 +22,7 @@ import pekko.japi.{ Option => JOption } import pekko.pattern.ask import pekko.routing.RoundRobinGroup import pekko.serialization.{ JavaSerializer, SerializerWithStringManifest } -import pekko.testkit.{ filterEvents, AkkaSpec, DefaultTimeout, EventFilter, TimingTest } +import pekko.testkit.{ filterEvents, DefaultTimeout, EventFilter, PekkoSpec, TimingTest } import pekko.util.Timeout object TypedActorSpec { @@ -251,7 +251,7 @@ object TypedActorSpec { @nowarn class TypedActorSpec - extends AkkaSpec(TypedActorSpec.config) + extends PekkoSpec(TypedActorSpec.config) with BeforeAndAfterEach with BeforeAndAfterAll with DefaultTimeout { @@ -577,7 +577,7 @@ class TypedActorSpec @nowarn class TypedActorRouterSpec - extends AkkaSpec(TypedActorSpec.config) + extends PekkoSpec(TypedActorSpec.config) with BeforeAndAfterEach with BeforeAndAfterAll with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/UidClashTest.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/UidClashTest.scala index c83359382e..9f481e001b 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/UidClashTest.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/UidClashTest.scala @@ -10,7 +10,7 @@ import org.apache.pekko import pekko.actor.SupervisorStrategy.{ Restart, Stop } import pekko.dispatch.sysmsg.SystemMessage import pekko.event.EventStream -import pekko.testkit.{ AkkaSpec, TestProbe } +import pekko.testkit.{ PekkoSpec, TestProbe } object UidClashTest { @@ -88,7 +88,7 @@ object UidClashTest { } -class UidClashTest extends AkkaSpec { +class UidClashTest extends PekkoSpec { import UidClashTest._ "The Terminated message for an old child stopped in preRestart" should { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/ActorModelSpec.scala index 2182571f81..e5558b1d57 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/ActorModelSpec.scala @@ -254,7 +254,7 @@ object ActorModelSpec { } else throw new AssertionError("await failed") } -abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with DefaultTimeout { +abstract class ActorModelSpec(config: String) extends PekkoSpec(config) with DefaultTimeout { import ActorModelSpec._ diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/BalancingDispatcherSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/BalancingDispatcherSpec.scala index fb7e77498e..11d7694b34 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/BalancingDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/BalancingDispatcherSpec.scala @@ -8,7 +8,7 @@ import java.util.concurrent.{ CountDownLatch, TimeUnit } import org.apache.pekko import pekko.actor.{ Actor, ActorCell, ActorRefWithCell, Props } -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec object BalancingDispatcherSpec { val config = """ @@ -19,7 +19,7 @@ object BalancingDispatcherSpec { """ } -class BalancingDispatcherSpec extends AkkaSpec(BalancingDispatcherSpec.config) { +class BalancingDispatcherSpec extends PekkoSpec(BalancingDispatcherSpec.config) { val delayableActorDispatcher = "pooled-dispatcher" diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/DispatcherActorSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/DispatcherActorSpec.scala index 9f6ba56b44..6e51b3a915 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/DispatcherActorSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/DispatcherActorSpec.scala @@ -15,7 +15,7 @@ import language.postfixOps import org.apache.pekko import pekko.actor.{ Actor, Props } import pekko.pattern.ask -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.DefaultTimeout object DispatcherActorSpec { @@ -56,7 +56,7 @@ object DispatcherActorSpec { } } -class DispatcherActorSpec extends AkkaSpec(DispatcherActorSpec.config) with DefaultTimeout { +class DispatcherActorSpec extends PekkoSpec(DispatcherActorSpec.config) with DefaultTimeout { import DispatcherActorSpec._ "A Dispatcher and an Actor" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/DispatcherActorsSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/DispatcherActorsSpec.scala index 268506efdb..0bd8960439 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/DispatcherActorsSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/DispatcherActorsSpec.scala @@ -8,12 +8,12 @@ import java.util.concurrent.CountDownLatch import org.apache.pekko import pekko.actor._ -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec /** * Tests the behavior of the executor based event driven dispatcher when multiple actors are being dispatched on it. */ -class DispatcherActorsSpec extends AkkaSpec { +class DispatcherActorsSpec extends PekkoSpec { class SlowActor(finishedCounter: CountDownLatch) extends Actor { def receive = { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/DispatchersSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/DispatchersSpec.scala index eeaf85713f..6c961ca497 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/DispatchersSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/DispatchersSpec.scala @@ -17,7 +17,7 @@ import pekko.ConfigurationException import pekko.actor._ import pekko.dispatch._ import pekko.routing.FromConfig -import pekko.testkit.{ AkkaSpec, ImplicitSender } +import pekko.testkit.{ ImplicitSender, PekkoSpec } import pekko.util.unused object DispatchersSpec { @@ -101,7 +101,7 @@ object DispatchersSpec { } } -class DispatchersSpec extends AkkaSpec(DispatchersSpec.config) with ImplicitSender { +class DispatchersSpec extends PekkoSpec(DispatchersSpec.config) with ImplicitSender { import DispatchersSpec._ val df = system.dispatchers import df._ diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/PinnedActorSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/PinnedActorSpec.scala index 9e5bab57d4..e289984b84 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/PinnedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dispatch/PinnedActorSpec.scala @@ -14,7 +14,7 @@ import org.apache.pekko import pekko.actor.{ Actor, Props } import pekko.pattern.ask import pekko.testkit._ -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec object PinnedActorSpec { val config = """ @@ -32,7 +32,7 @@ object PinnedActorSpec { } } -class PinnedActorSpec extends AkkaSpec(PinnedActorSpec.config) with BeforeAndAfterEach with DefaultTimeout { +class PinnedActorSpec extends PekkoSpec(PinnedActorSpec.config) with BeforeAndAfterEach with DefaultTimeout { import PinnedActorSpec._ "A PinnedActor" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dungeon/DispatchSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dungeon/DispatchSpec.scala index 0eb7aba247..ff7fe49e71 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dungeon/DispatchSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/dungeon/DispatchSpec.scala @@ -17,7 +17,7 @@ object DispatchSpec { } } } -class DispatchSpec extends AkkaSpec(""" +class DispatchSpec extends PekkoSpec(""" pekko.actor.serialize-messages = on pekko.actor.no-serialization-verification-needed-class-prefix = [] """) with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/routing/ListenerSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/routing/ListenerSpec.scala index f28823ecfb..46e79d6da1 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/actor/routing/ListenerSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/actor/routing/ListenerSpec.scala @@ -13,7 +13,7 @@ import pekko.actor._ import pekko.routing._ import pekko.testkit._ -class ListenerSpec extends AkkaSpec { +class ListenerSpec extends PekkoSpec { "Listener" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/config/ConfigSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/config/ConfigSpec.scala index 6a710a6ea0..1103e66e3e 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/config/ConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/config/ConfigSpec.scala @@ -16,9 +16,9 @@ import pekko.actor.ActorSystem import pekko.actor.ExtendedActorSystem import pekko.event.DefaultLoggingFilter import pekko.event.Logging.DefaultLogger -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.findClassLoader())) with Assertions { +class ConfigSpec extends PekkoSpec(ConfigFactory.defaultReference(ActorSystem.findClassLoader())) with Assertions { "The default configuration file (i.e. reference.conf)" must { "contain all configuration properties for akka-actor that are used in code with their correct defaults" in { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/dataflow/Future2Actor.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/dataflow/Future2Actor.scala index 6edbdaa70e..fe3b41cfa7 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/dataflow/Future2Actor.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/dataflow/Future2Actor.scala @@ -16,9 +16,9 @@ import org.apache.pekko import pekko.actor.{ Actor, Props } import pekko.actor.ActorRef import pekko.pattern.{ ask, pipe } -import pekko.testkit.{ AkkaSpec, DefaultTimeout } +import pekko.testkit.{ DefaultTimeout, PekkoSpec } -class Future2ActorSpec extends AkkaSpec with DefaultTimeout { +class Future2ActorSpec extends PekkoSpec with DefaultTimeout { implicit val ec: ExecutionContextExecutor = system.dispatcher "The Future2Actor bridge" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/ControlAwareDispatcherSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/ControlAwareDispatcherSpec.scala index d0e9fc7496..6b59898757 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/ControlAwareDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/ControlAwareDispatcherSpec.scala @@ -6,7 +6,7 @@ package org.apache.pekko.dispatch import org.apache.pekko import pekko.actor.{ Actor, Props } -import pekko.testkit.{ AkkaSpec, DefaultTimeout } +import pekko.testkit.{ DefaultTimeout, PekkoSpec } object ControlAwareDispatcherSpec { val config = """ @@ -21,7 +21,7 @@ object ControlAwareDispatcherSpec { case object ImportantMessage extends ControlMessage } -class ControlAwareDispatcherSpec extends AkkaSpec(ControlAwareDispatcherSpec.config) with DefaultTimeout { +class ControlAwareDispatcherSpec extends PekkoSpec(ControlAwareDispatcherSpec.config) with DefaultTimeout { import ControlAwareDispatcherSpec.ImportantMessage "A ControlAwareDispatcher" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/ExecutionContextSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/ExecutionContextSpec.scala index 3f3548f725..d9f3a894f1 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/ExecutionContextSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/ExecutionContextSpec.scala @@ -14,13 +14,13 @@ import scala.concurrent.duration._ import org.apache.pekko import pekko.actor.Actor import pekko.actor.Props -import pekko.testkit.{ AkkaSpec, DefaultTimeout, TestLatch } +import pekko.testkit.{ DefaultTimeout, PekkoSpec, TestLatch } import pekko.testkit.CallingThreadDispatcher import pekko.testkit.TestActorRef import pekko.testkit.TestProbe import pekko.util.SerializedSuspendableExecutionContext -class ExecutionContextSpec extends AkkaSpec with DefaultTimeout { +class ExecutionContextSpec extends PekkoSpec with DefaultTimeout { "An ExecutionContext" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/ForkJoinPoolStarvationSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/ForkJoinPoolStarvationSpec.scala index 718389f301..044ed9beff 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/ForkJoinPoolStarvationSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/ForkJoinPoolStarvationSpec.scala @@ -8,7 +8,7 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.{ Actor, Props } -import pekko.testkit.{ AkkaSpec, ImplicitSender } +import pekko.testkit.{ ImplicitSender, PekkoSpec } object ForkJoinPoolStarvationSpec { val config = ConfigFactory.parseString(""" @@ -44,12 +44,12 @@ object ForkJoinPoolStarvationSpec { } -class ForkJoinPoolStarvationSpec extends AkkaSpec(ForkJoinPoolStarvationSpec.config) with ImplicitSender { +class ForkJoinPoolStarvationSpec extends PekkoSpec(ForkJoinPoolStarvationSpec.config) with ImplicitSender { import ForkJoinPoolStarvationSpec._ val Iterations = 1000 - "AkkaForkJoinPool" must { + "PekkoForkJoinPool" must { "not starve tasks arriving from external dispatchers under high internal traffic" in { // TODO issue #31117: starvation with JDK 17 FJP diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/MailboxConfigSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/MailboxConfigSpec.scala index 811ec611b5..fca306ff48 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/MailboxConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/MailboxConfigSpec.scala @@ -15,10 +15,10 @@ import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } import org.apache.pekko import pekko.actor._ -import pekko.testkit.{ AkkaSpec, EventFilter } +import pekko.testkit.{ EventFilter, PekkoSpec } import pekko.util.unused -abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAndAfterEach { +abstract class MailboxSpec extends PekkoSpec with BeforeAndAfterAll with BeforeAndAfterEach { def name: String def factory: MailboxType => MessageQueue @@ -244,7 +244,7 @@ object CustomMailboxSpec { } } -class CustomMailboxSpec extends AkkaSpec(CustomMailboxSpec.config) { +class CustomMailboxSpec extends PekkoSpec(CustomMailboxSpec.config) { "Dispatcher configuration" must { "support custom mailboxType" in { val actor = system.actorOf(Props.empty.withDispatcher("my-dispatcher")) @@ -283,7 +283,7 @@ object SingleConsumerOnlyMailboxVerificationSpec { } class SingleConsumerOnlyMailboxVerificationSpec - extends AkkaSpec(SingleConsumerOnlyMailboxVerificationSpec.mailboxConf) { + extends PekkoSpec(SingleConsumerOnlyMailboxVerificationSpec.mailboxConf) { import SingleConsumerOnlyMailboxVerificationSpec.Ping def pathologicalPingPong(dispatcherId: String): Unit = { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/PriorityDispatcherSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/PriorityDispatcherSpec.scala index 38fdc2428d..2b77c78ecf 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/PriorityDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/PriorityDispatcherSpec.scala @@ -11,7 +11,7 @@ import language.postfixOps import org.apache.pekko import pekko.actor.{ Actor, ActorSystem, Props } -import pekko.testkit.{ AkkaSpec, DefaultTimeout } +import pekko.testkit.{ DefaultTimeout, PekkoSpec } import pekko.util.unused object PriorityDispatcherSpec { @@ -43,7 +43,7 @@ object PriorityDispatcherSpec { } -class PriorityDispatcherSpec extends AkkaSpec(PriorityDispatcherSpec.config) with DefaultTimeout { +class PriorityDispatcherSpec extends PekkoSpec(PriorityDispatcherSpec.config) with DefaultTimeout { import PriorityDispatcherSpec._ "A PriorityDispatcher" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/SameThreadExecutionContextSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/SameThreadExecutionContextSpec.scala index 14221f607b..f1b6d76b73 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/SameThreadExecutionContextSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/SameThreadExecutionContextSpec.scala @@ -12,9 +12,9 @@ import org.scalatest.matchers.should.Matchers import org.apache.pekko import pekko.Done import pekko.dispatch.internal.SameThreadExecutionContext -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class SameThreadExecutionContextSpec extends AkkaSpec with Matchers { +class SameThreadExecutionContextSpec extends PekkoSpec with Matchers { "The SameThreadExecutionContext" should { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/StablePriorityDispatcherSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/StablePriorityDispatcherSpec.scala index 7a40e9c14a..35176df5f6 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/StablePriorityDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/StablePriorityDispatcherSpec.scala @@ -11,7 +11,7 @@ import language.postfixOps import org.apache.pekko import pekko.actor.{ Actor, ActorSystem, Props } -import pekko.testkit.{ AkkaSpec, DefaultTimeout } +import pekko.testkit.{ DefaultTimeout, PekkoSpec } import pekko.util.unused object StablePriorityDispatcherSpec { @@ -44,7 +44,7 @@ object StablePriorityDispatcherSpec { } -class StablePriorityDispatcherSpec extends AkkaSpec(StablePriorityDispatcherSpec.config) with DefaultTimeout { +class StablePriorityDispatcherSpec extends PekkoSpec(StablePriorityDispatcherSpec.config) with DefaultTimeout { import StablePriorityDispatcherSpec._ "A StablePriorityDispatcher" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/sysmsg/SystemMessageListSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/sysmsg/SystemMessageListSpec.scala index b8f515e98f..fdc2a50be9 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/sysmsg/SystemMessageListSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/dispatch/sysmsg/SystemMessageListSpec.scala @@ -6,9 +6,9 @@ package org.apache.pekko.dispatch.sysmsg import org.apache.pekko import pekko.actor.Props -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class SystemMessageListSpec extends AkkaSpec { +class SystemMessageListSpec extends PekkoSpec { import SystemMessageList.ENil import SystemMessageList.LNil diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/event/AddressTerminatedTopicBenchSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/event/AddressTerminatedTopicBenchSpec.scala index 60a4dc30ab..2c8611cb15 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/event/AddressTerminatedTopicBenchSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/event/AddressTerminatedTopicBenchSpec.scala @@ -27,7 +27,7 @@ object AddressTerminatedTopicBenchSpec { } } -class AddressTerminatedTopicBenchSpec extends AkkaSpec("pekko.loglevel=INFO") { +class AddressTerminatedTopicBenchSpec extends PekkoSpec("pekko.loglevel=INFO") { import AddressTerminatedTopicBenchSpec._ "Subscribe and unsubscribe of AddressTerminated" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/event/EventBusSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/event/EventBusSpec.scala index 00bfd73c6a..6ab1dbc28b 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/event/EventBusSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/event/EventBusSpec.scala @@ -21,7 +21,7 @@ object EventBusSpec { } abstract class EventBusSpec(busName: String, conf: Config = ConfigFactory.empty()) - extends AkkaSpec(conf) + extends PekkoSpec(conf) with BeforeAndAfterEach { type BusType <: EventBus @@ -172,7 +172,7 @@ class ActorEventBusSpec(conf: Config) extends EventBusSpec("ActorEventBus", conf import org.apache.pekko.event.ActorEventBusSpec._ def this() = - this(ConfigFactory.parseString("pekko.actor.debug.event-stream = on").withFallback(AkkaSpec.testConf)) + this(ConfigFactory.parseString("pekko.actor.debug.event-stream = on").withFallback(PekkoSpec.testConf)) type BusType = MyActorEventBus def createNewEventBus(): BusType = new MyActorEventBus(system) diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/event/EventStreamSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/event/EventStreamSpec.scala index 732f5e8894..7956c7208a 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/event/EventStreamSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/event/EventStreamSpec.scala @@ -10,7 +10,7 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor._ -import pekko.testkit.{ AkkaSpec, TestProbe } +import pekko.testkit.{ PekkoSpec, TestProbe } object EventStreamSpec { @@ -66,7 +66,7 @@ object EventStreamSpec { class CCATBT extends CC with ATT with BTT } -class EventStreamSpec extends AkkaSpec(EventStreamSpec.config) { +class EventStreamSpec extends PekkoSpec(EventStreamSpec.config) { import EventStreamSpec._ diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/event/LoggerSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/event/LoggerSpec.scala index 771fe49161..7d61d9b701 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/event/LoggerSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/event/LoggerSpec.scala @@ -34,7 +34,7 @@ object LoggerSpec { loglevel = "DEBUG" # test verifies debug loggers = ["org.apache.pekko.event.LoggerSpec$TestLogger1"] } - """).withFallback(AkkaSpec.testConf) + """).withFallback(PekkoSpec.testConf) val slowConfig = ConfigFactory.parseString(""" pekko { @@ -42,7 +42,7 @@ object LoggerSpec { loglevel = "ERROR" loggers = ["org.apache.pekko.event.LoggerSpec$SlowLogger"] } - """).withFallback(AkkaSpec.testConf) + """).withFallback(PekkoSpec.testConf) val noLoggingConfig = ConfigFactory.parseString(""" pekko { @@ -50,7 +50,7 @@ object LoggerSpec { loglevel = "OFF" loggers = ["org.apache.pekko.event.LoggerSpec$TestLogger1"] } - """).withFallback(AkkaSpec.testConf) + """).withFallback(PekkoSpec.testConf) val multipleConfig = ConfigFactory.parseString(""" @@ -59,7 +59,7 @@ object LoggerSpec { loglevel = "WARNING" loggers = ["org.apache.pekko.event.LoggerSpec$TestLogger1", "org.apache.pekko.event.LoggerSpec$TestLogger2"] } - """).withFallback(AkkaSpec.testConf) + """).withFallback(PekkoSpec.testConf) val ticket3165Config = ConfigFactory.parseString(s""" pekko { @@ -75,7 +75,7 @@ object LoggerSpec { } } } - """).withFallback(AkkaSpec.testConf) + """).withFallback(PekkoSpec.testConf) val ticket3671Config = ConfigFactory.parseString(""" pekko { @@ -83,7 +83,7 @@ object LoggerSpec { loglevel = "WARNING" loggers = ["org.apache.pekko.event.LoggerSpec$TestLogger1"] } - """).withFallback(AkkaSpec.testConf) + """).withFallback(PekkoSpec.testConf) final case class SetTarget(ref: ActorRef, qualifier: Int) diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/event/LoggingReceiveSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/event/LoggingReceiveSpec.scala index a64a02ebbd..3091cf0d5a 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/event/LoggingReceiveSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/event/LoggingReceiveSpec.scala @@ -30,7 +30,7 @@ class LoggingReceiveSpec extends AnyWordSpec with BeforeAndAfterAll { import LoggingReceiveSpec._ val config = ConfigFactory.parseString(""" pekko.loglevel=DEBUG # test verifies debug - """).withFallback(AkkaSpec.testConf) + """).withFallback(PekkoSpec.testConf) val appLogging = ActorSystem("logging", ConfigFactory.parseMap(Map("pekko.actor.debug.receive" -> true).asJava).withFallback(config)) val appAuto = ActorSystem( diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/event/MarkerLoggingSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/event/MarkerLoggingSpec.scala index d7830e18b6..b633a4337a 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/event/MarkerLoggingSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/event/MarkerLoggingSpec.scala @@ -8,7 +8,7 @@ import org.apache.pekko import pekko.event.Logging._ import pekko.testkit._ -class MarkerLoggingSpec extends AkkaSpec with ImplicitSender { +class MarkerLoggingSpec extends PekkoSpec with ImplicitSender { "A MarkerLoggerAdapter" should { val markerLogging = new MarkerLoggingAdapter( system.eventStream, diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/event/jul/JavaLoggerSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/event/jul/JavaLoggerSpec.scala index b8be932a86..cf41de8580 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/event/jul/JavaLoggerSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/event/jul/JavaLoggerSpec.scala @@ -12,7 +12,7 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.{ Actor, ActorLogging, Props } -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec @deprecated("Use SLF4J instead.", "2.6.0") object JavaLoggerSpec { @@ -37,7 +37,7 @@ object JavaLoggerSpec { } @deprecated("Use SLF4J instead.", "2.6.0") -class JavaLoggerSpec extends AkkaSpec(JavaLoggerSpec.config) { +class JavaLoggerSpec extends PekkoSpec(JavaLoggerSpec.config) { val logger = logging.Logger.getLogger(classOf[JavaLoggerSpec.LogProducer].getName) logger.setUseParentHandlers(false) // turn off output of test LogRecords diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/io/CapacityLimitSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/io/CapacityLimitSpec.scala index 842288d5dc..7a8e6f4180 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/io/CapacityLimitSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/io/CapacityLimitSpec.scala @@ -7,10 +7,10 @@ package org.apache.pekko.io import Tcp._ import org.apache.pekko -import pekko.testkit.{ AkkaSpec, TestProbe } +import pekko.testkit.{ PekkoSpec, TestProbe } import pekko.testkit.SocketUtil.temporaryServerAddresses -class CapacityLimitSpec extends AkkaSpec(""" +class CapacityLimitSpec extends PekkoSpec(""" pekko.loglevel = ERROR pekko.io.tcp.max-channels = 4 """) with TcpIntegrationSpecSupport { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/io/InetAddressDnsResolverSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/io/InetAddressDnsResolverSpec.scala index 95d556be62..af7cf7c387 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/io/InetAddressDnsResolverSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/io/InetAddressDnsResolverSpec.scala @@ -11,10 +11,10 @@ import scala.annotation.nowarn import org.apache.pekko import pekko.actor.Props -import pekko.testkit.{ AkkaSpec, TestActorRef } +import pekko.testkit.{ PekkoSpec, TestActorRef } @nowarn -class InetAddressDnsResolverSpec extends AkkaSpec(""" +class InetAddressDnsResolverSpec extends PekkoSpec(""" pekko.io.dns.inet-address.positive-ttl = default pekko.io.dns.inet-address.negative-ttl = default """) { thisSpecs => @@ -122,7 +122,7 @@ class InetAddressDnsResolverSpec extends AkkaSpec(""" } @nowarn -class InetAddressDnsResolverConfigSpec extends AkkaSpec(""" +class InetAddressDnsResolverConfigSpec extends PekkoSpec(""" pekko.io.dns.inet-address.positive-ttl = forever pekko.io.dns.inet-address.negative-ttl = never """) { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpConnectionSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpConnectionSpec.scala index c7a028aa10..aae5c62824 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpConnectionSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpConnectionSpec.scala @@ -29,7 +29,7 @@ import pekko.actor._ import pekko.io.Inet.SocketOption import pekko.io.SelectionHandler._ import pekko.io.Tcp._ -import pekko.testkit.{ AkkaSpec, EventFilter, SocketUtil, TestActorRef, TestProbe } +import pekko.testkit.{ EventFilter, PekkoSpec, SocketUtil, TestActorRef, TestProbe } import pekko.testkit.SocketUtil.temporaryServerAddress import pekko.testkit.WithLogCapturing import pekko.util.{ ByteString, Helpers } @@ -40,7 +40,7 @@ object TcpConnectionSpec { final case class Registration(channel: SelectableChannel, initialOps: Int) extends NoSerializationVerificationNeeded } -class TcpConnectionSpec extends AkkaSpec(""" +class TcpConnectionSpec extends PekkoSpec(""" pekko.loglevel = DEBUG pekko.loggers = ["org.apache.pekko.testkit.SilenceAllTestEventListener"] pekko.io.tcp.trace-logging = on @@ -377,7 +377,7 @@ class TcpConnectionSpec extends AkkaSpec(""" "respect pull mode" in new EstablishedConnectionTest(pullMode = true) { // override config to decrease default buffer size def config = - ConfigFactory.parseString("pekko.io.tcp.direct-buffer-size = 1k").withFallback(AkkaSpec.testConf) + ConfigFactory.parseString("pekko.io.tcp.direct-buffer-size = 1k").withFallback(PekkoSpec.testConf) override implicit lazy val system: ActorSystem = ActorSystem("respectPullModeTest", config) try run { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpIntegrationSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpIntegrationSpec.scala index 249981e1fd..6dbf16b117 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpIntegrationSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpIntegrationSpec.scala @@ -15,11 +15,11 @@ import org.scalatest.concurrent.TimeLimits import org.apache.pekko import pekko.actor.{ ActorRef, PoisonPill } import pekko.io.Tcp._ -import pekko.testkit.{ AkkaSpec, TestProbe } +import pekko.testkit.{ PekkoSpec, TestProbe } import pekko.testkit.WithLogCapturing import pekko.util.ByteString -class TcpIntegrationSpec extends AkkaSpec(""" +class TcpIntegrationSpec extends PekkoSpec(""" pekko.loglevel = debug pekko.loggers = ["org.apache.pekko.testkit.SilenceAllTestEventListener"] pekko.io.tcp.trace-logging = on diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpIntegrationSpecSupport.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpIntegrationSpecSupport.scala index 6efa7b3a8a..7c3bfa73cb 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpIntegrationSpecSupport.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpIntegrationSpecSupport.scala @@ -14,10 +14,10 @@ import pekko.actor.ActorRef import pekko.actor.ActorSystem import pekko.dispatch.ExecutionContexts import pekko.io.Inet.SocketOption -import pekko.testkit.{ AkkaSpec, TestProbe } +import pekko.testkit.{ PekkoSpec, TestProbe } import pekko.testkit.SocketUtil.temporaryServerAddress -trait TcpIntegrationSpecSupport { this: AkkaSpec => +trait TcpIntegrationSpecSupport { this: PekkoSpec => class TestSetup(shouldBindServer: Boolean = true, runClientInExtraSystem: Boolean = true) { val clientSystem = diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpListenerSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpListenerSpec.scala index bfcf5ffb0a..ac32a1e560 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpListenerSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/io/TcpListenerSpec.scala @@ -17,10 +17,10 @@ import pekko.actor._ import pekko.io.SelectionHandler._ import pekko.io.TcpListener.{ FailedRegisterIncoming, RegisterIncoming } import pekko.io.TcpListenerSpec.RegisterChannel -import pekko.testkit.{ AkkaSpec, EventFilter, TestActorRef, TestProbe } +import pekko.testkit.{ EventFilter, PekkoSpec, TestActorRef, TestProbe } import pekko.testkit.SocketUtil -class TcpListenerSpec extends AkkaSpec(""" +class TcpListenerSpec extends PekkoSpec(""" pekko.io.tcp.batch-accept-limit = 2 """) { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/io/UdpConnectedIntegrationSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/io/UdpConnectedIntegrationSpec.scala index e4eff683ae..c3fd2837b6 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/io/UdpConnectedIntegrationSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/io/UdpConnectedIntegrationSpec.scala @@ -10,14 +10,14 @@ import scala.concurrent.duration._ import org.apache.pekko import pekko.actor.ActorRef -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender import pekko.testkit.SocketUtil.temporaryServerAddresses import pekko.testkit.TestProbe import pekko.testkit.WithLogCapturing import pekko.util.ByteString -class UdpConnectedIntegrationSpec extends AkkaSpec(""" +class UdpConnectedIntegrationSpec extends PekkoSpec(""" pekko.loglevel = DEBUG pekko.actor.debug.lifecycle = on pekko.actor.debug.autoreceive = on diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/io/UdpIntegrationSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/io/UdpIntegrationSpec.scala index 5298b3b839..768ff1d4d5 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/io/UdpIntegrationSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/io/UdpIntegrationSpec.scala @@ -10,11 +10,11 @@ import org.apache.pekko import pekko.actor.ActorRef import pekko.io.Inet._ import pekko.io.Udp._ -import pekko.testkit.{ AkkaSpec, ImplicitSender, TestProbe } +import pekko.testkit.{ ImplicitSender, PekkoSpec, TestProbe } import pekko.testkit.SocketUtil.temporaryServerAddresses import pekko.util.ByteString -class UdpIntegrationSpec extends AkkaSpec(""" +class UdpIntegrationSpec extends PekkoSpec(""" pekko.loglevel = INFO # tests expect to be able to mutate messages """) with ImplicitSender { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/DnsSettingsSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/DnsSettingsSpec.scala index cf14275d08..f99eda9c4d 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/DnsSettingsSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/DnsSettingsSpec.scala @@ -12,9 +12,9 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.ExtendedActorSystem -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class DnsSettingsSpec extends AkkaSpec { +class DnsSettingsSpec extends PekkoSpec { val eas = system.asInstanceOf[ExtendedActorSystem] diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/DockerBindDnsService.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/DockerBindDnsService.scala index eb725fba0d..e89b877640 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/DockerBindDnsService.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/DockerBindDnsService.scala @@ -15,10 +15,10 @@ import com.spotify.docker.client.messages.{ ContainerConfig, HostConfig, PortBin import org.scalatest.concurrent.Eventually import org.apache.pekko -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.util.ccompat.JavaConverters._ -abstract class DockerBindDnsService(config: Config) extends AkkaSpec(config) with Eventually { +abstract class DockerBindDnsService(config: Config) extends PekkoSpec(config) with Eventually { val client = DefaultDockerClient.fromEnv().build() val hostPort: Int diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/AsyncDnsManagerSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/AsyncDnsManagerSpec.scala index 5e91e04bb8..ab83a8bbe4 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/AsyncDnsManagerSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/AsyncDnsManagerSpec.scala @@ -15,12 +15,12 @@ import pekko.io.Dns import pekko.io.dns.AAAARecord import pekko.io.dns.CachePolicy.Ttl import pekko.io.dns.DnsProtocol.{ Resolve, Resolved } -import pekko.testkit.{ AkkaSpec, ImplicitSender } +import pekko.testkit.{ ImplicitSender, PekkoSpec } import pekko.testkit.WithLogCapturing // tests deprecated DNS API @nowarn("msg=deprecated") -class AsyncDnsManagerSpec extends AkkaSpec(""" +class AsyncDnsManagerSpec extends PekkoSpec(""" pekko.loglevel = DEBUG pekko.loggers = ["org.apache.pekko.testkit.SilenceAllTestEventListener"] pekko.io.dns.resolver = async-dns diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/AsyncDnsResolverSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/AsyncDnsResolverSpec.scala index 3e3ab4659d..ec1e8b4ebd 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/AsyncDnsResolverSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/AsyncDnsResolverSpec.scala @@ -20,9 +20,9 @@ import pekko.io.dns.CachePolicy.Ttl import pekko.io.dns.DnsProtocol._ import pekko.io.dns.internal.AsyncDnsResolver.ResolveFailedException import pekko.io.dns.internal.DnsClient.{ Answer, Question4, Question6, SrvQuestion } -import pekko.testkit.{ AkkaSpec, TestProbe, WithLogCapturing } +import pekko.testkit.{ PekkoSpec, TestProbe, WithLogCapturing } -class AsyncDnsResolverSpec extends AkkaSpec(""" +class AsyncDnsResolverSpec extends PekkoSpec(""" pekko.loglevel = DEBUG pekko.loggers = ["org.apache.pekko.testkit.SilenceAllTestEventListener"] """) with WithLogCapturing { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/DnsClientSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/DnsClientSpec.scala index 09e0ea6f6d..5faf26f904 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/DnsClientSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/DnsClientSpec.scala @@ -14,9 +14,9 @@ import pekko.actor.Props import pekko.io.Udp import pekko.io.dns.{ RecordClass, RecordType } import pekko.io.dns.internal.DnsClient.{ Answer, Question4 } -import pekko.testkit.{ AkkaSpec, ImplicitSender, TestProbe } +import pekko.testkit.{ ImplicitSender, PekkoSpec, TestProbe } -class DnsClientSpec extends AkkaSpec with ImplicitSender { +class DnsClientSpec extends PekkoSpec with ImplicitSender { "The async DNS client" should { val exampleRequest = Question4(42, "pekko.io") val exampleRequestMessage = diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/TcpDnsClientSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/TcpDnsClientSpec.scala index 6562f312e2..c6467797b0 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/TcpDnsClientSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/io/dns/internal/TcpDnsClientSpec.scala @@ -14,9 +14,9 @@ import pekko.io.Tcp import pekko.io.Tcp.{ Connected, PeerClosed, Register } import pekko.io.dns.{ RecordClass, RecordType } import pekko.io.dns.internal.DnsClient.Answer -import pekko.testkit.{ AkkaSpec, ImplicitSender, TestProbe } +import pekko.testkit.{ ImplicitSender, PekkoSpec, TestProbe } -class TcpDnsClientSpec extends AkkaSpec with ImplicitSender { +class TcpDnsClientSpec extends PekkoSpec with ImplicitSender { import TcpDnsClient._ "The async TCP DNS client" should { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/AskSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/AskSpec.scala index cff0fb8136..aecce3ef40 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/AskSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/AskSpec.scala @@ -13,12 +13,12 @@ import language.postfixOps import org.apache.pekko import pekko.actor._ -import pekko.testkit.{ AkkaSpec, TestProbe } +import pekko.testkit.{ PekkoSpec, TestProbe } import pekko.testkit.WithLogCapturing import pekko.util.Timeout @nowarn -class AskSpec extends AkkaSpec(""" +class AskSpec extends PekkoSpec(""" pekko.loglevel = DEBUG pekko.loggers = ["org.apache.pekko.testkit.SilenceAllTestEventListener"] """) with WithLogCapturing { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/BackoffOnRestartSupervisorSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/BackoffOnRestartSupervisorSpec.scala index 78bf2ad94c..c0592e3935 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/BackoffOnRestartSupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/BackoffOnRestartSupervisorSpec.scala @@ -14,7 +14,7 @@ import scala.annotation.nowarn import org.apache.pekko import pekko.actor._ import pekko.pattern.TestActor.NormalException -import pekko.testkit.{ filterException, AkkaSpec, ImplicitSender, TestProbe } +import pekko.testkit.{ filterException, ImplicitSender, PekkoSpec, TestProbe } import pekko.testkit.WithLogCapturing object TestActor { @@ -54,7 +54,7 @@ class TestParentActor(probe: ActorRef, supervisorProps: Props) extends Actor { } } -class BackoffOnRestartSupervisorSpec extends AkkaSpec(""" +class BackoffOnRestartSupervisorSpec extends PekkoSpec(""" pekko.loglevel = DEBUG pekko.loggers = ["org.apache.pekko.testkit.SilenceAllTestEventListener"] """) with WithLogCapturing with ImplicitSender { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/BackoffSupervisorSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/BackoffSupervisorSpec.scala index 2a000abc0d..41052938e2 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/BackoffSupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/BackoffSupervisorSpec.scala @@ -45,7 +45,7 @@ object BackoffSupervisorSpec { } } -class BackoffSupervisorSpec extends AkkaSpec with ImplicitSender with Eventually { +class BackoffSupervisorSpec extends PekkoSpec with ImplicitSender with Eventually { import BackoffSupervisorSpec._ def onStopOptions(props: Props = Child.props(testActor), maxNrOfRetries: Int = -1): BackoffOnStopOptions = diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/CircuitBreakerMTSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/CircuitBreakerMTSpec.scala index 66bda8cf3f..de9da4eb45 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/CircuitBreakerMTSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/CircuitBreakerMTSpec.scala @@ -11,7 +11,7 @@ import scala.concurrent.duration._ import org.apache.pekko.testkit._ -class CircuitBreakerMTSpec extends AkkaSpec { +class CircuitBreakerMTSpec extends PekkoSpec { implicit val ec: ExecutionContextExecutor = system.dispatcher "A circuit breaker being called by many threads" must { val callTimeout = 2.seconds.dilated diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/CircuitBreakerSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/CircuitBreakerSpec.scala index fde8fb39da..6f2debe9f2 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/CircuitBreakerSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/CircuitBreakerSpec.scala @@ -83,7 +83,7 @@ object CircuitBreakerSpec { } } -class CircuitBreakerSpec extends AkkaSpec(""" +class CircuitBreakerSpec extends PekkoSpec(""" pekko.circuit-breaker { identified { max-failures = 1 diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/CircuitBreakerStressSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/CircuitBreakerStressSpec.scala index e8f89eeefb..bebe12745e 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/CircuitBreakerStressSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/CircuitBreakerStressSpec.scala @@ -15,7 +15,7 @@ import pekko.actor.Actor import pekko.actor.ActorLogging import pekko.actor.Props import pekko.actor.Status.Failure -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender object CircuitBreakerStressSpec { @@ -61,7 +61,7 @@ object CircuitBreakerStressSpec { } // reproducer for issue #17415 -class CircuitBreakerStressSpec extends AkkaSpec with ImplicitSender { +class CircuitBreakerStressSpec extends PekkoSpec with ImplicitSender { import CircuitBreakerStressSpec._ muteDeadLetters(classOf[AnyRef])(system) diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/PatternSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/PatternSpec.scala index e944eadafc..5d369b104b 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/PatternSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/PatternSpec.scala @@ -12,7 +12,7 @@ import language.postfixOps import org.apache.pekko import pekko.actor.{ Actor, Props } -import pekko.testkit.{ AkkaSpec, TestLatch } +import pekko.testkit.{ PekkoSpec, TestLatch } object PatternSpec { final case class Work(duration: Duration) @@ -24,7 +24,7 @@ object PatternSpec { } } -class PatternSpec extends AkkaSpec { +class PatternSpec extends PekkoSpec { implicit val ec: ExecutionContextExecutor = system.dispatcher import PatternSpec._ diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/PipeToSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/PipeToSpec.scala index 19d48ceade..d175f8a937 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/PipeToSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/PipeToSpec.scala @@ -8,10 +8,10 @@ import scala.concurrent.Future import org.apache.pekko import pekko.actor.Status -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TestProbe -class PipeToSpec extends AkkaSpec { +class PipeToSpec extends PekkoSpec { import system.dispatcher diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/PromiseRefSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/PromiseRefSpec.scala index f0dda848e5..20e2ac0ec6 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/PromiseRefSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/PromiseRefSpec.scala @@ -9,7 +9,7 @@ import scala.concurrent.duration._ import org.apache.pekko import pekko.actor._ -import pekko.testkit.{ AkkaSpec, ImplicitSender, TestProbe } +import pekko.testkit.{ ImplicitSender, PekkoSpec, TestProbe } object PromiseRefSpec { case class Request(replyTo: ActorRef) @@ -19,7 +19,7 @@ object PromiseRefSpec { case object SecondMessage } -class PromiseRefSpec extends AkkaSpec with ImplicitSender { +class PromiseRefSpec extends PekkoSpec with ImplicitSender { import PromiseRefSpec._ import pekko.pattern._ diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/RetrySpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/RetrySpec.scala index ce6f719b4e..5dd6fc01a3 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/RetrySpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/RetrySpec.scala @@ -11,9 +11,9 @@ import language.postfixOps import org.apache.pekko import pekko.actor.Scheduler -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class RetrySpec extends AkkaSpec with RetrySupport { +class RetrySpec extends PekkoSpec with RetrySupport { implicit val ec: ExecutionContextExecutor = system.dispatcher implicit val scheduler: Scheduler = system.scheduler diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/StatusReplySpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/StatusReplySpec.scala index 0370d19466..e5de885a26 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/StatusReplySpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/StatusReplySpec.scala @@ -11,12 +11,12 @@ import org.scalatest.concurrent.ScalaFutures import org.apache.pekko import pekko.Done -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TestException import pekko.testkit.TestProbe import pekko.util.Timeout -class StatusReplySpec extends AkkaSpec with ScalaFutures { +class StatusReplySpec extends PekkoSpec with ScalaFutures { "StatusReply" should { "pattern match success" in { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/extended/ExplicitAskSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/extended/ExplicitAskSpec.scala index ed8a784ca4..50b5c00be0 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/extended/ExplicitAskSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/pattern/extended/ExplicitAskSpec.scala @@ -9,7 +9,7 @@ import scala.concurrent.duration._ import org.apache.pekko import pekko.actor._ -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.util.Timeout object ExplicitAskSpec { @@ -17,7 +17,7 @@ object ExplicitAskSpec { case class Response(sentFrom: ActorRef) } -class ExplicitAskSpec extends AkkaSpec { +class ExplicitAskSpec extends PekkoSpec { import ExplicitAskSpec._ "The “ask” pattern with explicit sender" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/BalancingSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/BalancingSpec.scala index 4f7f6b4e67..7dbe2d468a 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/BalancingSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/BalancingSpec.scala @@ -12,7 +12,7 @@ import scala.concurrent.duration._ import org.apache.pekko import pekko.actor.{ Actor, ActorLogging, ActorRef, PoisonPill, Props } -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender import pekko.testkit.TestLatch import org.scalatest.BeforeAndAfterEach @@ -60,7 +60,7 @@ object BalancingSpec { } } -class BalancingSpec extends AkkaSpec(""" +class BalancingSpec extends PekkoSpec(""" pekko.loglevel=debug pekko.actor.deployment { /balancingPool-2 { @@ -81,7 +81,7 @@ class BalancingSpec extends AkkaSpec(""" """) with ImplicitSender with BeforeAndAfterEach { import BalancingSpec._ - val poolSize = 5 // must be less than fork-join parallelism-min, which is 8 in AkkaSpec + val poolSize = 5 // must be less than fork-join parallelism-min, which is 8 in PekkoSpec override def beforeEach(): Unit = { counter.set(1) diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/BroadcastSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/BroadcastSpec.scala index a154bef611..32ab552557 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/BroadcastSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/BroadcastSpec.scala @@ -11,7 +11,7 @@ import scala.concurrent.Await import org.apache.pekko import pekko.actor.{ Actor, Props } import pekko.pattern.ask -import pekko.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender, TestLatch } +import pekko.testkit.{ DefaultTimeout, ImplicitSender, PekkoSpec, TestLatch } object BroadcastSpec { class TestActor extends Actor { @@ -19,7 +19,7 @@ object BroadcastSpec { } } -class BroadcastSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { +class BroadcastSpec extends PekkoSpec with DefaultTimeout with ImplicitSender { "broadcast group" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ConfiguredLocalRoutingSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ConfiguredLocalRoutingSpec.scala index 3f26b2e124..cb6ce3ab84 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ConfiguredLocalRoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ConfiguredLocalRoutingSpec.scala @@ -20,7 +20,7 @@ import pekko.actor.ActorSystem import pekko.actor.ExtendedActorSystem import pekko.actor.UnstartedCell import pekko.pattern.gracefulStop -import pekko.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender } +import pekko.testkit.{ DefaultTimeout, ImplicitSender, PekkoSpec } import pekko.testkit.TestActors.echoActorProps import pekko.testkit.TestProbe @@ -104,7 +104,7 @@ object ConfiguredLocalRoutingSpec { } class ConfiguredLocalRoutingSpec - extends AkkaSpec(ConfiguredLocalRoutingSpec.config) + extends PekkoSpec(ConfiguredLocalRoutingSpec.config) with DefaultTimeout with ImplicitSender { import ConfiguredLocalRoutingSpec._ diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ConsistentHashingRouterSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ConsistentHashingRouterSpec.scala index 7f815fb4bc..bb1464364e 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ConsistentHashingRouterSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ConsistentHashingRouterSpec.scala @@ -16,7 +16,7 @@ import pekko.routing.ConsistentHashingRouter.ConsistentHashMapping import pekko.routing.ConsistentHashingRouter.ConsistentHashable import pekko.routing.ConsistentHashingRouter.ConsistentHashableEnvelope import pekko.testkit._ -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec object ConsistentHashingRouterSpec { @@ -55,7 +55,7 @@ object ConsistentHashingRouterSpec { } class ConsistentHashingRouterSpec - extends AkkaSpec(ConsistentHashingRouterSpec.config) + extends PekkoSpec(ConsistentHashingRouterSpec.config) with DefaultTimeout with ImplicitSender { import ConsistentHashingRouterSpec._ diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/MetricsBasedResizerSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/MetricsBasedResizerSpec.scala index 9a849df65a..a07597c8d6 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/MetricsBasedResizerSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/MetricsBasedResizerSpec.scala @@ -70,7 +70,7 @@ object MetricsBasedResizerSpec { } -class MetricsBasedResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with ImplicitSender { +class MetricsBasedResizerSpec extends PekkoSpec(ResizerSpec.config) with DefaultTimeout with ImplicitSender { override def atStartup(): Unit = { // when shutting down some Resize messages might hang around diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RandomSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RandomSpec.scala index 240ebd1566..8055fa939a 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RandomSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RandomSpec.scala @@ -14,9 +14,9 @@ import language.postfixOps import org.apache.pekko import pekko.actor.{ Actor, Props } import pekko.pattern.ask -import pekko.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender, TestLatch } +import pekko.testkit.{ DefaultTimeout, ImplicitSender, PekkoSpec, TestLatch } -class RandomSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { +class RandomSpec extends PekkoSpec with DefaultTimeout with ImplicitSender { "random pool" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ResizerSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ResizerSpec.scala index 4eee4bade4..6b6df3c2b8 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ResizerSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ResizerSpec.scala @@ -38,7 +38,7 @@ object ResizerSpec { } -class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with ImplicitSender { +class ResizerSpec extends PekkoSpec(ResizerSpec.config) with DefaultTimeout with ImplicitSender { import pekko.routing.ResizerSpec._ diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RoundRobinSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RoundRobinSpec.scala index 3a69962adf..5f359c13fb 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RoundRobinSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RoundRobinSpec.scala @@ -19,7 +19,7 @@ import pekko.actor.Terminated import pekko.pattern.ask import pekko.testkit._ -class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { +class RoundRobinSpec extends PekkoSpec with DefaultTimeout with ImplicitSender { def routeeSize(router: ActorRef): Int = Await.result(router ? GetRoutees, timeout.duration).asInstanceOf[Routees].routees.size diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RouteeCreationSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RouteeCreationSpec.scala index 20e6871ff1..d9a781ecdd 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RouteeCreationSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RouteeCreationSpec.scala @@ -11,9 +11,9 @@ import pekko.actor.Actor import pekko.actor.ActorIdentity import pekko.actor.Identify import pekko.actor.Props -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class RouteeCreationSpec extends AkkaSpec { +class RouteeCreationSpec extends PekkoSpec { "Creating Routees" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RoutingSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RoutingSpec.scala index 0a4bd1e0eb..b0ee2e78ad 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/RoutingSpec.scala @@ -49,7 +49,7 @@ object RoutingSpec { } -class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with ImplicitSender { +class RoutingSpec extends PekkoSpec(RoutingSpec.config) with DefaultTimeout with ImplicitSender { implicit val ec: ExecutionContextExecutor = system.dispatcher import RoutingSpec._ diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ScatterGatherFirstCompletedSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ScatterGatherFirstCompletedSpec.scala index 2fcf8d1a6c..ec2ad43deb 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ScatterGatherFirstCompletedSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/ScatterGatherFirstCompletedSpec.scala @@ -15,7 +15,7 @@ import pekko.actor.{ Actor, Props } import pekko.actor.ActorSystem import pekko.actor.Status import pekko.pattern.ask -import pekko.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender, TestLatch } +import pekko.testkit.{ DefaultTimeout, ImplicitSender, PekkoSpec, TestLatch } import pekko.testkit.TestProbe object ScatterGatherFirstCompletedSpec { @@ -45,7 +45,7 @@ object ScatterGatherFirstCompletedSpec { "Actor:" + id) } -class ScatterGatherFirstCompletedSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { +class ScatterGatherFirstCompletedSpec extends PekkoSpec with DefaultTimeout with ImplicitSender { import ScatterGatherFirstCompletedSpec._ "Scatter-gather group" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/SmallestMailboxSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/SmallestMailboxSpec.scala index 0bd276ffb1..18748c68a3 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/SmallestMailboxSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/SmallestMailboxSpec.scala @@ -10,9 +10,9 @@ import scala.concurrent.Await import org.apache.pekko import pekko.actor.{ Actor, Props } -import pekko.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender, TestLatch } +import pekko.testkit.{ DefaultTimeout, ImplicitSender, PekkoSpec, TestLatch } -class SmallestMailboxSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { +class SmallestMailboxSpec extends PekkoSpec with DefaultTimeout with ImplicitSender { "smallest mailbox pool" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/TailChoppingSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/TailChoppingSpec.scala index f293321259..9b2781a54d 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/routing/TailChoppingSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/routing/TailChoppingSpec.scala @@ -31,7 +31,7 @@ object TailChoppingSpec { }), "Actor:" + id) } -class TailChoppingSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { +class TailChoppingSpec extends PekkoSpec with DefaultTimeout with ImplicitSender { import TailChoppingSpec._ def oneOfShouldEqual(what: Any, default: Any, ref: ActorRef*)(f: ActorRef => Any): Unit = { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/AsyncSerializeSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/AsyncSerializeSpec.scala index bedeb9fd59..b20c568569 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/AsyncSerializeSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/AsyncSerializeSpec.scala @@ -10,7 +10,7 @@ import scala.concurrent.Future import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.ExtendedActorSystem -import pekko.testkit.{ AkkaSpec, EventFilter } +import pekko.testkit.{ EventFilter, PekkoSpec } object AsyncSerializeSpec { @@ -93,7 +93,7 @@ object AsyncSerializeSpec { } -class AsyncSerializeSpec extends AkkaSpec(AsyncSerializeSpec.config) { +class AsyncSerializeSpec extends PekkoSpec(AsyncSerializeSpec.config) { import AsyncSerializeSpec._ val ser = SerializationExtension(system) diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/DisabledJavaSerializerWarningSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/DisabledJavaSerializerWarningSpec.scala index 9bdd8a64df..88030218af 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/DisabledJavaSerializerWarningSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/DisabledJavaSerializerWarningSpec.scala @@ -16,7 +16,7 @@ object DisabledJavaSerializerWarningSpec { final case class Msg(s: String) } -class DisabledJavaSerializerWarningSpec extends AkkaSpec(""" +class DisabledJavaSerializerWarningSpec extends PekkoSpec(""" pekko.actor { allow-java-serialization = off serialize-messages = on diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/PrimitivesSerializationSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/PrimitivesSerializationSpec.scala index 35b74b2177..38296a1f44 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/PrimitivesSerializationSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/PrimitivesSerializationSpec.scala @@ -12,16 +12,16 @@ import scala.util.Random import com.typesafe.config.ConfigFactory import org.apache.pekko -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.util.ByteString object PrimitivesSerializationSpec { val serializationTestOverrides = "" - val testConfig = ConfigFactory.parseString(serializationTestOverrides).withFallback(AkkaSpec.testConf) + val testConfig = ConfigFactory.parseString(serializationTestOverrides).withFallback(PekkoSpec.testConf) } -class PrimitivesSerializationSpec extends AkkaSpec(PrimitivesSerializationSpec.testConfig) { +class PrimitivesSerializationSpec extends PekkoSpec(PrimitivesSerializationSpec.testConfig) { val buffer = { val b = ByteBuffer.allocate(4096) diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/SerializationSetupSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/SerializationSetupSpec.scala index 61f7b2bde4..7897704fa4 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/SerializationSetupSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/SerializationSetupSpec.scala @@ -14,7 +14,7 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.{ ActorSystem, BootstrapSetup, ExtendedActorSystem } import pekko.actor.setup.ActorSystemSetup -import pekko.testkit.{ AkkaSpec, TestKit } +import pekko.testkit.{ PekkoSpec, TestKit } class ConfigurationDummy class ProgrammaticDummy @@ -87,7 +87,7 @@ object SerializationSetupSpec { } class SerializationSetupSpec - extends AkkaSpec(ActorSystem("SerializationSettingsSpec", SerializationSetupSpec.actorSystemSettings)) { + extends PekkoSpec(ActorSystem("SerializationSettingsSpec", SerializationSetupSpec.actorSystemSettings)) { import SerializationSetupSpec._ diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/SerializeSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/SerializeSpec.scala index 3f3bd8a065..65b3042a31 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/SerializeSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/serialization/SerializeSpec.scala @@ -21,7 +21,7 @@ import org.apache.pekko import pekko.actor._ import pekko.actor.dungeon.SerializationCheckFailedException import pekko.pattern.ask -import pekko.testkit.{ AkkaSpec, EventFilter } +import pekko.testkit.{ EventFilter, PekkoSpec } import pekko.util.{ unused, Timeout } import pekko.util.ByteString @@ -33,7 +33,7 @@ object SerializationTests { serializers { test = "org.apache.pekko.serialization.NoopSerializer" test2 = "org.apache.pekko.serialization.NoopSerializer2" - other = "other.SerializerOutsideAkkaPackage" + other = "other.SerializerOutsidePekkoPackage" } serialization-bindings { @@ -120,7 +120,7 @@ object SerializationTests { def mostlyReferenceSystem: ActorSystem = { val referenceConf = ConfigFactory.defaultReference() - val mostlyReferenceConf = AkkaSpec.testConf.withFallback(referenceConf) + val mostlyReferenceConf = PekkoSpec.testConf.withFallback(referenceConf) ActorSystem("SerializationSystem", mostlyReferenceConf) } @@ -132,7 +132,7 @@ object SerializationTests { pekko.actor.allow-java-serialization = on """) .withFallback(ConfigFactory.parseString(serializeConf)) - .withFallback(AkkaSpec.testConf.withFallback(referenceConf)) + .withFallback(PekkoSpec.testConf.withFallback(referenceConf)) ActorSystem("SerializationSystem", conf) } @@ -152,7 +152,7 @@ object SerializationTests { } -class SerializeSpec extends AkkaSpec(SerializationTests.serializeConf) { +class SerializeSpec extends PekkoSpec(SerializationTests.serializeConf) { val ser = SerializationExtension(system) @@ -266,7 +266,7 @@ class SerializeSpec extends AkkaSpec(SerializationTests.serializeConf) { s"${classOf[ByteArraySerializer].getName} only serializes byte arrays, not [java.lang.String]") } - "log warning if non-Akka serializer is configured for Akka message" in { + "log warning if non-Pekko serializer is configured for Pekko message" in { EventFilter.warning(pattern = ".*not implemented by Pekko.*", occurrences = 1).intercept { ser.serialize(new Other).get } @@ -297,7 +297,7 @@ class SerializeSpec extends AkkaSpec(SerializationTests.serializeConf) { } } -class VerifySerializabilitySpec extends AkkaSpec(SerializationTests.verifySerializabilityConf) { +class VerifySerializabilitySpec extends PekkoSpec(SerializationTests.verifySerializabilityConf) { implicit val timeout: Timeout = Timeout(5 seconds) "verify config" in { @@ -353,7 +353,7 @@ class VerifySerializabilitySpec extends AkkaSpec(SerializationTests.verifySerial } } -class ReferenceSerializationSpec extends AkkaSpec(SerializationTests.mostlyReferenceSystem) { +class ReferenceSerializationSpec extends PekkoSpec(SerializationTests.mostlyReferenceSystem) { val ser = SerializationExtension(system) def serializerMustBe(toSerialize: Class[_], expectedSerializer: Class[_]) = @@ -391,7 +391,7 @@ class ReferenceSerializationSpec extends AkkaSpec(SerializationTests.mostlyRefer } } -class AllowJavaSerializationSpec extends AkkaSpec(SerializationTests.allowJavaSerializationSystem) { +class AllowJavaSerializationSpec extends PekkoSpec(SerializationTests.allowJavaSerializationSystem) { val ser = SerializationExtension(system) def serializerMustBe(toSerialize: Class[_], expectedSerializer: Class[_]) = @@ -483,7 +483,7 @@ class AllowJavaSerializationSpec extends AkkaSpec(SerializationTests.allowJavaSe "serialize DeadLetterActorRef" in { val outbuf = new ByteArrayOutputStream() val out = new ObjectOutputStream(outbuf) - val a = ActorSystem("SerializeDeadLeterActorRef", AkkaSpec.testConf) + val a = ActorSystem("SerializeDeadLeterActorRef", PekkoSpec.testConf) try { out.writeObject(a.deadLetters) out.flush() @@ -503,7 +503,7 @@ class AllowJavaSerializationSpec extends AkkaSpec(SerializationTests.allowJavaSe } class NoVerificationWarningSpec - extends AkkaSpec(ConfigFactory.parseString(""" + extends PekkoSpec(ConfigFactory.parseString(""" pekko.actor.allow-java-serialization = on pekko.actor.warn-about-java-serializer-usage = on pekko.actor.warn-on-no-serialization-verification = on @@ -529,7 +529,7 @@ class NoVerificationWarningSpec } class NoVerificationWarningOffSpec - extends AkkaSpec(ConfigFactory.parseString(""" + extends PekkoSpec(ConfigFactory.parseString(""" pekko.actor.allow-java-serialization = on pekko.actor.warn-about-java-serializer-usage = on pekko.actor.warn-on-no-serialization-verification = off @@ -554,7 +554,7 @@ class NoVerificationWarningOffSpec } } -class SerializerDeadlockSpec extends AkkaSpec { +class SerializerDeadlockSpec extends PekkoSpec { "SerializationExtension" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/util/DurationSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/util/DurationSpec.scala index 703da93ee1..a3331b070b 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/util/DurationSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/util/DurationSpec.scala @@ -8,9 +8,9 @@ import scala.concurrent.duration._ import language.postfixOps -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec -class DurationSpec extends AkkaSpec { +class DurationSpec extends PekkoSpec { "Duration" must { diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/util/IndexSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/util/IndexSpec.scala index b80350dc92..57567dfdcf 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/util/IndexSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/util/IndexSpec.scala @@ -14,10 +14,10 @@ import scala.util.Random import org.scalatest.matchers.should.Matchers import org.apache.pekko -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.DefaultTimeout -class IndexSpec extends AkkaSpec with Matchers with DefaultTimeout { +class IndexSpec extends PekkoSpec with Matchers with DefaultTimeout { implicit val ec: ExecutionContextExecutor = system.dispatcher private def emptyIndex = new Index[String, Int](100, diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/util/ManifestInfoSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/util/ManifestInfoSpec.scala index be8b5c27b0..9ff58ff7ad 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/util/ManifestInfoSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/util/ManifestInfoSpec.scala @@ -4,9 +4,9 @@ package org.apache.pekko.util -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec -class ManifestInfoSpec extends AkkaSpec { +class ManifestInfoSpec extends PekkoSpec { "ManifestInfo" should { "produce a clear message" in { val versions = Map( diff --git a/akka-actor-tests/src/test/scala/org/apache/pekko/util/TokenBucketSpec.scala b/akka-actor-tests/src/test/scala/org/apache/pekko/util/TokenBucketSpec.scala index 528903ff16..2e8af85d83 100644 --- a/akka-actor-tests/src/test/scala/org/apache/pekko/util/TokenBucketSpec.scala +++ b/akka-actor-tests/src/test/scala/org/apache/pekko/util/TokenBucketSpec.scala @@ -6,9 +6,9 @@ package org.apache.pekko.util import scala.util.Random -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec -class TokenBucketSpec extends AkkaSpec { +class TokenBucketSpec extends PekkoSpec { class TestBucket(_cap: Long, _period: Long) extends TokenBucket(_cap, _period) { var currentTime: Long = 0L diff --git a/akka-actor-tests/src/test/scala/other/SerializerOutsideAkkaPackage.scala b/akka-actor-tests/src/test/scala/other/SerializerOutsidePekkoPackage.scala similarity index 87% rename from akka-actor-tests/src/test/scala/other/SerializerOutsideAkkaPackage.scala rename to akka-actor-tests/src/test/scala/other/SerializerOutsidePekkoPackage.scala index c946248e73..d787e51be2 100644 --- a/akka-actor-tests/src/test/scala/other/SerializerOutsideAkkaPackage.scala +++ b/akka-actor-tests/src/test/scala/other/SerializerOutsidePekkoPackage.scala @@ -8,7 +8,7 @@ import java.nio.charset.StandardCharsets import org.apache.pekko.serialization.SerializerWithStringManifest -class SerializerOutsideAkkaPackage extends SerializerWithStringManifest { +class SerializerOutsidePekkoPackage extends SerializerWithStringManifest { override def identifier: Int = 999 override def manifest(o: AnyRef): String = "A" diff --git a/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/ActorContextAskTest.java b/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/ActorContextAskTest.java index eeefff2f0b..470dd43701 100644 --- a/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/ActorContextAskTest.java +++ b/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/ActorContextAskTest.java @@ -9,7 +9,7 @@ import org.apache.pekko.actor.testkit.typed.javadsl.LogCapturing; import org.apache.pekko.actor.typed.ActorRef; import org.apache.pekko.actor.typed.Behavior; import org.apache.pekko.pattern.StatusReply; -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoSpec; import org.apache.pekko.actor.testkit.typed.javadsl.TestKitJunitResource; import org.apache.pekko.actor.testkit.typed.javadsl.TestProbe; import org.junit.ClassRule; @@ -22,7 +22,7 @@ import java.time.Duration; public class ActorContextAskTest extends JUnitSuite { @ClassRule - public static final TestKitJunitResource testKit = new TestKitJunitResource(AkkaSpec.testConf()); + public static final TestKitJunitResource testKit = new TestKitJunitResource(PekkoSpec.testConf()); @Rule public final LogCapturing logCapturing = new LogCapturing(); diff --git a/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/AdapterTest.java b/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/AdapterTest.java index 60471f9551..9c81785ca9 100644 --- a/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/AdapterTest.java +++ b/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/AdapterTest.java @@ -14,8 +14,8 @@ import org.scalatestplus.junit.JUnitSuite; import java.time.Duration; import org.apache.pekko.actor.ActorSystem; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoSpec; import org.apache.pekko.actor.typed.ActorRef; import org.apache.pekko.actor.typed.Behavior; import org.apache.pekko.actor.typed.Signal; @@ -246,8 +246,8 @@ public class AdapterTest extends JUnitSuite { } @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("ActorSelectionTest", AkkaSpec.testConf()); + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("ActorSelectionTest", PekkoSpec.testConf()); @Rule public final LogCapturing logCapturing = new LogCapturing(); diff --git a/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/InterceptTest.java b/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/InterceptTest.java index ff05669323..9e4e81822e 100644 --- a/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/InterceptTest.java +++ b/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/InterceptTest.java @@ -8,7 +8,7 @@ import org.apache.pekko.actor.testkit.typed.javadsl.LogCapturing; import org.apache.pekko.actor.testkit.typed.javadsl.TestKitJunitResource; import org.apache.pekko.actor.testkit.typed.javadsl.TestProbe; import org.apache.pekko.actor.typed.*; -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoSpec; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; @@ -17,7 +17,7 @@ import org.scalatestplus.junit.JUnitSuite; public class InterceptTest extends JUnitSuite { @ClassRule - public static final TestKitJunitResource testKit = new TestKitJunitResource(AkkaSpec.testConf()); + public static final TestKitJunitResource testKit = new TestKitJunitResource(PekkoSpec.testConf()); @Rule public final LogCapturing logCapturing = new LogCapturing(); diff --git a/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/RoutersTest.java b/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/RoutersTest.java index 44c6609c14..155a44653c 100644 --- a/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/RoutersTest.java +++ b/akka-actor-typed-tests/src/test/java/org/apache/pekko/actor/typed/javadsl/RoutersTest.java @@ -12,7 +12,7 @@ import org.apache.pekko.actor.testkit.typed.javadsl.TestProbe; import org.apache.pekko.actor.typed.ActorRef; import org.apache.pekko.actor.typed.Behavior; import org.apache.pekko.actor.typed.receptionist.ServiceKey; -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoSpec; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; @@ -24,7 +24,7 @@ import static org.junit.Assert.assertTrue; public class RoutersTest extends JUnitSuite { @ClassRule - public static final TestKitJunitResource testKit = new TestKitJunitResource(AkkaSpec.testConf()); + public static final TestKitJunitResource testKit = new TestKitJunitResource(PekkoSpec.testConf()); @Rule public final LogCapturing logCapturing = new LogCapturing(); diff --git a/akka-actor/src/main/scala/org/apache/pekko/AkkaException.scala b/akka-actor/src/main/scala/org/apache/pekko/PekkoException.scala similarity index 81% rename from akka-actor/src/main/scala/org/apache/pekko/AkkaException.scala rename to akka-actor/src/main/scala/org/apache/pekko/PekkoException.scala index a93afb196b..8d94dc98f2 100644 --- a/akka-actor/src/main/scala/org/apache/pekko/AkkaException.scala +++ b/akka-actor/src/main/scala/org/apache/pekko/PekkoException.scala @@ -5,10 +5,10 @@ package org.apache.pekko /** - * Akka base Exception. + * Pekko base Exception. */ @SerialVersionUID(1L) -class AkkaException(message: String, cause: Throwable) extends RuntimeException(message, cause) with Serializable { +class PekkoException(message: String, cause: Throwable) extends RuntimeException(message, cause) with Serializable { def this(msg: String) = this(msg, null) } @@ -29,6 +29,6 @@ trait OnlyCauseStackTrace { self: Throwable => /** * This exception is thrown when Akka detects a problem with the provided configuration */ -class ConfigurationException(message: String, cause: Throwable) extends AkkaException(message, cause) { +class ConfigurationException(message: String, cause: Throwable) extends PekkoException(message, cause) { def this(msg: String) = this(msg, null) } diff --git a/akka-actor/src/main/scala/org/apache/pekko/AkkaVersion.scala b/akka-actor/src/main/scala/org/apache/pekko/PekkoVersion.scala similarity index 76% rename from akka-actor/src/main/scala/org/apache/pekko/AkkaVersion.scala rename to akka-actor/src/main/scala/org/apache/pekko/PekkoVersion.scala index f76e633593..d8e75795a9 100644 --- a/akka-actor/src/main/scala/org/apache/pekko/AkkaVersion.scala +++ b/akka-actor/src/main/scala/org/apache/pekko/PekkoVersion.scala @@ -6,18 +6,18 @@ package org.apache.pekko import org.apache.pekko.annotation.InternalApi -final class UnsupportedAkkaVersion private[pekko] (msg: String) extends RuntimeException(msg) +final class UnsupportedPekkoVersion private[pekko] (msg: String) extends RuntimeException(msg) -object AkkaVersion { +object PekkoVersion { /** - * Check that the version of Akka is a specific patch version or higher and throw an [[UnsupportedAkkaVersion]] + * Check that the version of Pekko is a specific patch version or higher and throw an [[UnsupportedPekkoVersion]] * exception if the version requirement is not fulfilled. * - * For example: `require("my-library", "2.5.4")` would fail if used with Akka 2.4.19 and 2.5.3, but succeed with 2.5.4 + * For example: `require("my-library", "2.5.4")` would fail if used with Pekko 2.4.19 and 2.5.3, but succeed with 2.5.4 * and 2.6.1 * - * @param libraryName The name of the library or component requiring the Akka version, used in the error message. + * @param libraryName The name of the library or component requiring the Pekko version, used in the error message. * @param requiredVersion Minimal version that this library works with */ def require(libraryName: String, requiredVersion: String): Unit = { @@ -42,8 +42,8 @@ object AkkaVersion { if (requiredMajorStr.toInt != currentMajorStr.toInt || requiredMinorStr.toInt > currentMinorStr.toInt || (requiredMinorStr == currentMinorStr && requiredPatchStr.toInt > currentPatch)) - throw new UnsupportedAkkaVersion( - s"Current version of Akka is [$currentVersion], but $libraryName requires version [$requiredVersion]") + throw new UnsupportedPekkoVersion( + s"Current version of Pekko is [$currentVersion], but $libraryName requires version [$requiredVersion]") case _ => // SNAPSHOT or unknown - you're on your own } diff --git a/akka-actor/src/main/scala/org/apache/pekko/actor/Actor.scala b/akka-actor/src/main/scala/org/apache/pekko/actor/Actor.scala index 3d94e7ab04..11aa173148 100644 --- a/akka-actor/src/main/scala/org/apache/pekko/actor/Actor.scala +++ b/akka-actor/src/main/scala/org/apache/pekko/actor/Actor.scala @@ -11,7 +11,7 @@ import scala.beans.BeanProperty import scala.util.control.NoStackTrace import org.apache.pekko -import pekko.AkkaException +import pekko.PekkoException import pekko.annotation.InternalApi import pekko.event.LoggingAdapter import pekko.util.unused @@ -159,20 +159,20 @@ trait NotInfluenceReceiveTimeout * For instance, if you try to create an Actor that doesn't extend Actor. */ @SerialVersionUID(1L) -final case class IllegalActorStateException private[pekko] (message: String) extends AkkaException(message) +final case class IllegalActorStateException private[pekko] (message: String) extends PekkoException(message) /** * ActorKilledException is thrown when an Actor receives the [[pekko.actor.Kill]] message */ @SerialVersionUID(1L) -final case class ActorKilledException private[pekko] (message: String) extends AkkaException(message) with NoStackTrace +final case class ActorKilledException private[pekko] (message: String) extends PekkoException(message) with NoStackTrace /** * An InvalidActorNameException is thrown when you try to convert something, usually a String, to an Actor name * which doesn't validate. */ @SerialVersionUID(1L) -final case class InvalidActorNameException(message: String) extends AkkaException(message) +final case class InvalidActorNameException(message: String) extends PekkoException(message) /** * An ActorInitializationException is thrown when the initialization logic for an Actor fails. @@ -187,7 +187,7 @@ final case class InvalidActorNameException(message: String) extends AkkaExceptio */ @SerialVersionUID(1L) class ActorInitializationException protected (actor: ActorRef, message: String, cause: Throwable) - extends AkkaException(ActorInitializationException.enrichedMessage(actor, message), cause) { + extends PekkoException(ActorInitializationException.enrichedMessage(actor, message), cause) { def getActor: ActorRef = actor } object ActorInitializationException { @@ -263,7 +263,7 @@ object OriginalRestartException { * Currently only `null` is an invalid message. */ @SerialVersionUID(1L) -final case class InvalidMessageException private[pekko] (message: String) extends AkkaException(message) +final case class InvalidMessageException private[pekko] (message: String) extends PekkoException(message) /** * A DeathPactException is thrown by an Actor that receives a Terminated(someActor) message @@ -271,7 +271,7 @@ final case class InvalidMessageException private[pekko] (message: String) extend */ @SerialVersionUID(1L) final case class DeathPactException private[pekko] (dead: ActorRef) - extends AkkaException("Monitored actor [" + dead + "] terminated") + extends PekkoException("Monitored actor [" + dead + "] terminated") with NoStackTrace /** @@ -279,7 +279,7 @@ final case class DeathPactException private[pekko] (dead: ActorRef) * avoid cascading interrupts to other threads than the originally interrupted one. */ @SerialVersionUID(1L) -class ActorInterruptedException private[pekko] (cause: Throwable) extends AkkaException(cause.getMessage, cause) +class ActorInterruptedException private[pekko] (cause: Throwable) extends PekkoException(cause.getMessage, cause) /** * This message is published to the EventStream whenever an Actor receives a message it doesn't understand diff --git a/akka-actor/src/main/scala/org/apache/pekko/actor/Scheduler.scala b/akka-actor/src/main/scala/org/apache/pekko/actor/Scheduler.scala index c701845e95..fd3a6e07c5 100644 --- a/akka-actor/src/main/scala/org/apache/pekko/actor/Scheduler.scala +++ b/akka-actor/src/main/scala/org/apache/pekko/actor/Scheduler.scala @@ -21,7 +21,7 @@ import pekko.util.JavaDurationConverters * This exception is thrown by Scheduler.schedule* when scheduling is not * possible, e.g. after shutting down the Scheduler. */ -private final case class SchedulerException(msg: String) extends pekko.AkkaException(msg) with NoStackTrace +private final case class SchedulerException(msg: String) extends pekko.PekkoException(msg) with NoStackTrace /** * An Akka scheduler service. diff --git a/akka-actor/src/main/scala/org/apache/pekko/actor/Stash.scala b/akka-actor/src/main/scala/org/apache/pekko/actor/Stash.scala index c41c120abd..a7a1792d39 100644 --- a/akka-actor/src/main/scala/org/apache/pekko/actor/Stash.scala +++ b/akka-actor/src/main/scala/org/apache/pekko/actor/Stash.scala @@ -8,7 +8,7 @@ import scala.collection.immutable import scala.util.control.NoStackTrace import org.apache.pekko -import pekko.AkkaException +import pekko.PekkoException import pekko.annotation.InternalStableApi import pekko.dispatch.{ DequeBasedMessageQueueSemantics, @@ -272,5 +272,5 @@ private[pekko] trait StashSupport { * Is thrown when the size of the Stash exceeds the capacity of the Stash */ class StashOverflowException(message: String, cause: Throwable = null) - extends AkkaException(message, cause) + extends PekkoException(message, cause) with NoStackTrace diff --git a/akka-actor/src/main/scala/org/apache/pekko/actor/dungeon/Dispatch.scala b/akka-actor/src/main/scala/org/apache/pekko/actor/dungeon/Dispatch.scala index 84e52c2ebc..fda92267f8 100644 --- a/akka-actor/src/main/scala/org/apache/pekko/actor/dungeon/Dispatch.scala +++ b/akka-actor/src/main/scala/org/apache/pekko/actor/dungeon/Dispatch.scala @@ -11,7 +11,7 @@ import scala.util.control.Exception.Catcher import scala.annotation.nowarn import org.apache.pekko -import pekko.AkkaException +import pekko.PekkoException import pekko.actor._ import pekko.annotation.InternalApi import pekko.dispatch.{ Envelope, Mailbox } @@ -26,7 +26,7 @@ import pekko.util.Unsafe @SerialVersionUID(1L) final case class SerializationCheckFailedException private[dungeon] (msg: Object, cause: Throwable) - extends AkkaException( + extends PekkoException( s"Failed to serialize and deserialize message of type ${msg.getClass.getName} for testing. " + "To avoid this error, either disable 'pekko.actor.serialize-messages', mark the message with 'org.apache.pekko.actor.NoSerializationVerificationNeeded', or configure serialization to support this message", cause) diff --git a/akka-actor/src/main/scala/org/apache/pekko/dispatch/ForkJoinExecutorConfigurator.scala b/akka-actor/src/main/scala/org/apache/pekko/dispatch/ForkJoinExecutorConfigurator.scala index 3c6d1e3c61..d01d28c365 100644 --- a/akka-actor/src/main/scala/org/apache/pekko/dispatch/ForkJoinExecutorConfigurator.scala +++ b/akka-actor/src/main/scala/org/apache/pekko/dispatch/ForkJoinExecutorConfigurator.scala @@ -13,7 +13,7 @@ object ForkJoinExecutorConfigurator { /** * INTERNAL AKKA USAGE ONLY */ - final class AkkaForkJoinPool( + final class PekkoForkJoinPool( parallelism: Int, threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory, unhandledExceptionHandler: Thread.UncaughtExceptionHandler, @@ -29,7 +29,7 @@ object ForkJoinExecutorConfigurator { override def execute(r: Runnable): Unit = if (r ne null) super.execute( - (if (r.isInstanceOf[ForkJoinTask[_]]) r else new AkkaForkJoinTask(r)).asInstanceOf[ForkJoinTask[Any]]) + (if (r.isInstanceOf[ForkJoinTask[_]]) r else new PekkoForkJoinTask(r)).asInstanceOf[ForkJoinTask[Any]]) else throw new NullPointerException("Runnable was null") @@ -40,7 +40,7 @@ object ForkJoinExecutorConfigurator { * INTERNAL AKKA USAGE ONLY */ @SerialVersionUID(1L) - final class AkkaForkJoinTask(runnable: Runnable) extends ForkJoinTask[Unit] { + final class PekkoForkJoinTask(runnable: Runnable) extends ForkJoinTask[Unit] { override def getRawResult(): Unit = () override def setRawResult(unit: Unit): Unit = () override def exec(): Boolean = @@ -80,7 +80,7 @@ class ForkJoinExecutorConfigurator(config: Config, prerequisites: DispatcherPrer def this(threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory, parallelism: Int) = this(threadFactory, parallelism, asyncMode = true) def createExecutorService: ExecutorService = - new AkkaForkJoinPool(parallelism, threadFactory, MonitorableThreadFactory.doNothing, asyncMode) + new PekkoForkJoinPool(parallelism, threadFactory, MonitorableThreadFactory.doNothing, asyncMode) } final def createExecutorServiceFactory(id: String, threadFactory: ThreadFactory): ExecutorServiceFactory = { diff --git a/akka-actor/src/main/scala/org/apache/pekko/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/org/apache/pekko/dispatch/ThreadPoolBuilder.scala index a5d66ea2ca..7a23da0973 100644 --- a/akka-actor/src/main/scala/org/apache/pekko/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/org/apache/pekko/dispatch/ThreadPoolBuilder.scala @@ -164,7 +164,7 @@ object MonitorableThreadFactory { val doNothing: Thread.UncaughtExceptionHandler = new Thread.UncaughtExceptionHandler() { def uncaughtException(thread: Thread, cause: Throwable) = () } - private[pekko] class AkkaForkJoinWorkerThread(_pool: ForkJoinPool) + private[pekko] class PekkoForkJoinWorkerThread(_pool: ForkJoinPool) extends ForkJoinWorkerThread(_pool) with BlockContext { override def blockOn[T](thunk: => T)(implicit permission: CanAwait): T = { @@ -191,7 +191,7 @@ final case class MonitorableThreadFactory( with ForkJoinPool.ForkJoinWorkerThreadFactory { def newThread(pool: ForkJoinPool): ForkJoinWorkerThread = { - val t = wire(new MonitorableThreadFactory.AkkaForkJoinWorkerThread(pool)) + val t = wire(new MonitorableThreadFactory.PekkoForkJoinWorkerThread(pool)) // Name of the threads for the ForkJoinPool are not customizable. Change it here. t.setName(name + "-" + counter.incrementAndGet()) t diff --git a/akka-actor/src/main/scala/org/apache/pekko/event/Logging.scala b/akka-actor/src/main/scala/org/apache/pekko/event/Logging.scala index d8a3901a47..5f4f3d2a8c 100644 --- a/akka-actor/src/main/scala/org/apache/pekko/event/Logging.scala +++ b/akka-actor/src/main/scala/org/apache/pekko/event/Logging.scala @@ -16,7 +16,7 @@ import scala.util.control.{ NoStackTrace, NonFatal } import scala.annotation.nowarn import org.apache.pekko -import pekko.{ AkkaException, ConfigurationException } +import pekko.{ ConfigurationException, PekkoException } import pekko.actor._ import pekko.actor.ActorSystem.Settings import pekko.annotation.{ DoNotInherit, InternalApi } @@ -700,7 +700,7 @@ object Logging { * Artificial exception injected into Error events if no Throwable is * supplied; used for getting a stack dump of error locations. */ - class LoggerException extends AkkaException("") + class LoggerException extends PekkoException("") /** * Exception that wraps a LogEvent. @@ -980,7 +980,7 @@ object Logging { /** * LoggerInitializationException is thrown to indicate that there was a problem initializing a logger */ - class LoggerInitializationException(msg: String) extends AkkaException(msg) + class LoggerInitializationException(msg: String) extends PekkoException(msg) trait StdOutLogger { diff --git a/akka-actor/src/main/scala/org/apache/pekko/io/dns/internal/TcpDnsClient.scala b/akka-actor/src/main/scala/org/apache/pekko/io/dns/internal/TcpDnsClient.scala index 2dbfbdd602..ac0da23a09 100644 --- a/akka-actor/src/main/scala/org/apache/pekko/io/dns/internal/TcpDnsClient.scala +++ b/akka-actor/src/main/scala/org/apache/pekko/io/dns/internal/TcpDnsClient.scala @@ -7,7 +7,7 @@ package org.apache.pekko.io.dns.internal import java.net.InetSocketAddress import org.apache.pekko -import pekko.AkkaException +import pekko.PekkoException import pekko.actor.{ Actor, ActorLogging, ActorRef, Stash } import pekko.annotation.InternalApi import pekko.io.Tcp @@ -95,8 +95,8 @@ private[internal] object TcpDnsClient { def throwFailure(message: String, cause: Option[Throwable]): Unit = cause match { case None => - throw new AkkaException(message) + throw new PekkoException(message) case Some(throwable) => - throw new AkkaException(message, throwable) + throw new PekkoException(message, throwable) } } diff --git a/akka-actor/src/main/scala/org/apache/pekko/pattern/CircuitBreaker.scala b/akka-actor/src/main/scala/org/apache/pekko/pattern/CircuitBreaker.scala index cdb68fb830..ddd667c460 100644 --- a/akka-actor/src/main/scala/org/apache/pekko/pattern/CircuitBreaker.scala +++ b/akka-actor/src/main/scala/org/apache/pekko/pattern/CircuitBreaker.scala @@ -18,7 +18,7 @@ import scala.util.{ Failure, Success, Try } import scala.util.control.NoStackTrace import scala.util.control.NonFatal import org.apache.pekko -import pekko.AkkaException +import pekko.PekkoException import pekko.actor.{ ExtendedActorSystem, Scheduler } import pekko.dispatch.ExecutionContexts.parasitic import pekko.pattern.internal.{ CircuitBreakerNoopTelemetry, CircuitBreakerTelemetry } @@ -1127,5 +1127,5 @@ class CircuitBreaker( class CircuitBreakerOpenException( val remainingDuration: FiniteDuration, message: String = "Circuit Breaker is open; calls are failing fast") - extends AkkaException(message) + extends PekkoException(message) with NoStackTrace diff --git a/akka-actor/src/main/scala/org/apache/pekko/routing/Resizer.scala b/akka-actor/src/main/scala/org/apache/pekko/routing/Resizer.scala index 01c9ecbcaa..ca2aaf5e3c 100644 --- a/akka-actor/src/main/scala/org/apache/pekko/routing/Resizer.scala +++ b/akka-actor/src/main/scala/org/apache/pekko/routing/Resizer.scala @@ -12,7 +12,7 @@ import scala.collection.immutable import com.typesafe.config.Config import org.apache.pekko -import pekko.AkkaException +import pekko.PekkoException import pekko.actor.Actor import pekko.actor.ActorCell import pekko.actor.ActorInitializationException @@ -71,7 +71,7 @@ object Resizer { } @SerialVersionUID(1L) -class ResizerInitializationException(message: String, cause: Throwable) extends AkkaException(message, cause) +class ResizerInitializationException(message: String, cause: Throwable) extends PekkoException(message, cause) case object DefaultResizer { diff --git a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/ClusterMetricsExtensionSpec.scala b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/ClusterMetricsExtensionSpec.scala index 1ea1fa4d1a..d9a5d229c6 100644 --- a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/ClusterMetricsExtensionSpec.scala +++ b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/ClusterMetricsExtensionSpec.scala @@ -13,7 +13,7 @@ import pekko.cluster.metrics.StandardMetrics._ import pekko.testkit._ class ClusterMetricsExtensionSpec - extends AkkaSpec(MetricsConfig.clusterSigarMock) + extends PekkoSpec(MetricsConfig.clusterSigarMock) with ImplicitSender with RedirectLogging { diff --git a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/ClusterMetricsSettingsSpec.scala b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/ClusterMetricsSettingsSpec.scala index f7b0440262..d4a6c4d959 100644 --- a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/ClusterMetricsSettingsSpec.scala +++ b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/ClusterMetricsSettingsSpec.scala @@ -11,9 +11,9 @@ import language.postfixOps import org.apache.pekko import pekko.dispatch.Dispatchers -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class ClusterMetricsSettingsSpec extends AkkaSpec { +class ClusterMetricsSettingsSpec extends PekkoSpec { "ClusterMetricsSettings" must { diff --git a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/EWMASpec.scala b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/EWMASpec.scala index 632f9c9731..f107de5a41 100644 --- a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/EWMASpec.scala +++ b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/EWMASpec.scala @@ -10,10 +10,10 @@ import scala.concurrent.duration._ import scala.annotation.nowarn -import org.apache.pekko.testkit.{ AkkaSpec, LongRunningTest } +import org.apache.pekko.testkit.{ LongRunningTest, PekkoSpec } @nowarn -class EWMASpec extends AkkaSpec(MetricsConfig.defaultEnabled) with MetricsCollectorFactory { +class EWMASpec extends PekkoSpec(MetricsConfig.defaultEnabled) with MetricsCollectorFactory { val collector = createMetricsCollector diff --git a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/MetricSpec.scala b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/MetricSpec.scala index 31102b0817..4fbe5e3c48 100644 --- a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/MetricSpec.scala +++ b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/MetricSpec.scala @@ -15,7 +15,7 @@ import org.scalatest.wordspec.AnyWordSpec import org.apache.pekko import pekko.actor.Address import pekko.cluster.metrics.StandardMetrics._ -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender @nowarn @@ -136,7 +136,7 @@ class NodeMetricsSpec extends AnyWordSpec with Matchers { } class MetricsGossipSpec - extends AkkaSpec(MetricsConfig.defaultEnabled) + extends PekkoSpec(MetricsConfig.defaultEnabled) with ImplicitSender with MetricsCollectorFactory { @@ -240,7 +240,7 @@ class MetricsGossipSpec } @nowarn -class MetricValuesSpec extends AkkaSpec(MetricsConfig.defaultEnabled) with MetricsCollectorFactory { +class MetricValuesSpec extends PekkoSpec(MetricsConfig.defaultEnabled) with MetricsCollectorFactory { import pekko.cluster.metrics.StandardMetrics._ val collector = createMetricsCollector diff --git a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/MetricsCollectorSpec.scala b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/MetricsCollectorSpec.scala index b3a5f5a4f7..7350613253 100644 --- a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/MetricsCollectorSpec.scala +++ b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/MetricsCollectorSpec.scala @@ -16,7 +16,7 @@ import pekko.testkit._ @nowarn class MetricsCollectorSpec - extends AkkaSpec(MetricsConfig.defaultEnabled) + extends PekkoSpec(MetricsConfig.defaultEnabled) with ImplicitSender with MetricsCollectorFactory { diff --git a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/TestUtil.scala b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/TestUtil.scala index 3eb545d268..417fbc17b5 100644 --- a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/TestUtil.scala +++ b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/TestUtil.scala @@ -26,7 +26,7 @@ import pekko.dispatch.Dispatchers import pekko.dispatch.RequiresMessageQueue import pekko.dispatch.UnboundedMessageQueueSemantics import pekko.remote.RARP -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec /** * Redirect different logging sources to SLF4J. @@ -97,7 +97,7 @@ case class MockitoSigarProvider( * * TODO change factory after https://github.com/akka/akka/issues/16369 */ -trait MetricsCollectorFactory { this: AkkaSpec => +trait MetricsCollectorFactory { this: PekkoSpec => import MetricsConfig._ import org.hyperic.sigar.Sigar diff --git a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/WeightedRouteesSpec.scala b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/WeightedRouteesSpec.scala index c2900fc081..5834ffb8e9 100644 --- a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/WeightedRouteesSpec.scala +++ b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/WeightedRouteesSpec.scala @@ -14,9 +14,9 @@ import pekko.actor.RootActorPath import pekko.remote.RARP import pekko.routing.ActorRefRoutee import pekko.routing.ActorSelectionRoutee -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString(""" +class WeightedRouteesSpec extends PekkoSpec(ConfigFactory.parseString(""" pekko.actor.provider = "cluster" pekko.remote.classic.netty.tcp.port = 0 pekko.remote.artery.canonical.port = 0 diff --git a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/protobuf/MessageSerializerSpec.scala b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/protobuf/MessageSerializerSpec.scala index 168fc3949a..4b207f07fb 100644 --- a/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/protobuf/MessageSerializerSpec.scala +++ b/akka-cluster-metrics/src/test/scala/org/apache/pekko/cluster/metrics/protobuf/MessageSerializerSpec.scala @@ -9,9 +9,9 @@ import pekko.actor.{ Address, ExtendedActorSystem } import pekko.cluster.MemberStatus import pekko.cluster.TestMember import pekko.cluster.metrics._ -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class MessageSerializerSpec extends AkkaSpec(""" +class MessageSerializerSpec extends PekkoSpec(""" pekko.actor.provider = cluster pekko.remote.classic.netty.tcp.port = 0 pekko.remote.artery.canonical.port = 0 diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingHealthCheckSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingHealthCheckSpec.scala index 34f930d53e..35c6323a5f 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingHealthCheckSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingHealthCheckSpec.scala @@ -5,7 +5,7 @@ package org.apache.pekko.cluster.sharding import org.apache.pekko -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TestProbe import pekko.testkit.WithLogCapturing import com.typesafe.config.ConfigFactory @@ -21,7 +21,7 @@ object ClusterShardingHealthCheckSpec { } class ClusterShardingHealthCheckSpec - extends AkkaSpec(ClusterShardingHealthCheckSpec.config) + extends PekkoSpec(ClusterShardingHealthCheckSpec.config) with WithLogCapturing with ScalaFutures { diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingInternalsSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingInternalsSpec.scala index e631b50f29..1de710578e 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingInternalsSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingInternalsSpec.scala @@ -13,7 +13,7 @@ import pekko.cluster.sharding.ShardCoordinator.Internal.ShardStopped import pekko.cluster.sharding.ShardCoordinator.ShardAllocationStrategy import pekko.cluster.sharding.ShardRegion.{ ExtractEntityId, ExtractShardId, HandOffStopper, Msg } import pekko.testkit.WithLogCapturing -import pekko.testkit.{ AkkaSpec, TestProbe } +import pekko.testkit.{ PekkoSpec, TestProbe } object ClusterShardingInternalsSpec { case class HandOffStopMessage() extends NoSerializationVerificationNeeded @@ -28,7 +28,7 @@ object ClusterShardingInternalsSpec { } } -class ClusterShardingInternalsSpec extends AkkaSpec(""" +class ClusterShardingInternalsSpec extends PekkoSpec(""" |pekko.actor.provider = cluster |pekko.remote.classic.netty.tcp.port = 0 |pekko.remote.artery.canonical.port = 0 diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingLeaseSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingLeaseSpec.scala index 5e07d7e6b1..b9dcaab601 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingLeaseSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingLeaseSpec.scala @@ -14,7 +14,7 @@ import pekko.cluster.sharding.ShardRegion.StartEntity import pekko.cluster.{ Cluster, MemberStatus } import pekko.coordination.lease.TestLease import pekko.coordination.lease.TestLeaseExt -import pekko.testkit.{ AkkaSpec, ImplicitSender, WithLogCapturing } +import pekko.testkit.{ ImplicitSender, PekkoSpec, WithLogCapturing } import pekko.testkit.TestActors.EchoActor object ClusterShardingLeaseSpec { @@ -67,7 +67,7 @@ class PersistenceClusterShardingLeaseSpec class DDataClusterShardingLeaseSpec extends ClusterShardingLeaseSpec(ClusterShardingLeaseSpec.ddataConfig, true) class ClusterShardingLeaseSpec(config: Config, rememberEntities: Boolean) - extends AkkaSpec(config.withFallback(ClusterShardingLeaseSpec.config)) + extends PekkoSpec(config.withFallback(ClusterShardingLeaseSpec.config)) with ImplicitSender with WithLogCapturing { import ClusterShardingLeaseSpec._ diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingSettingsSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingSettingsSpec.scala index 999f174609..a8d44d89dc 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingSettingsSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ClusterShardingSettingsSpec.scala @@ -6,7 +6,7 @@ package org.apache.pekko.cluster.sharding import org.apache.pekko import pekko.actor.ActorSystem -import pekko.testkit.{ AkkaSpec, TestKit } +import pekko.testkit.{ PekkoSpec, TestKit } import com.typesafe.config.ConfigFactory import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec @@ -16,7 +16,7 @@ import scala.concurrent.duration._ class ClusterShardingSettingsSpec extends AnyWordSpec with Matchers { def settings(conf: String): ClusterShardingSettings = { - val config = ConfigFactory.parseString(conf).withFallback(AkkaSpec.testConf) + val config = ConfigFactory.parseString(conf).withFallback(PekkoSpec.testConf) val system = ActorSystem("ClusterShardingSettingsSpec", config) val clusterShardingSettings = ClusterShardingSettings(system) TestKit.shutdownActorSystem(system) diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ConcurrentStartupShardingSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ConcurrentStartupShardingSpec.scala index 8bd689793c..37e12566bd 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ConcurrentStartupShardingSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ConcurrentStartupShardingSpec.scala @@ -11,7 +11,7 @@ import pekko.actor.ActorRef import pekko.actor.Props import pekko.cluster.Cluster import pekko.cluster.MemberStatus -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.DeadLettersFilter import pekko.testkit.TestEvent.Mute import pekko.testkit.WithLogCapturing @@ -61,7 +61,7 @@ object ConcurrentStartupShardingSpec { } } -class ConcurrentStartupShardingSpec extends AkkaSpec(ConcurrentStartupShardingSpec.config) with WithLogCapturing { +class ConcurrentStartupShardingSpec extends PekkoSpec(ConcurrentStartupShardingSpec.config) with WithLogCapturing { import ConcurrentStartupShardingSpec._ // mute logging of deadLetters diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/CoordinatedShutdownShardingSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/CoordinatedShutdownShardingSpec.scala index d4edeb8fab..2e04c33d16 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/CoordinatedShutdownShardingSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/CoordinatedShutdownShardingSpec.scala @@ -14,7 +14,7 @@ import pekko.actor.CoordinatedShutdown import pekko.actor.Props import pekko.cluster.Cluster import pekko.cluster.MemberStatus -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TestActors.EchoActor import pekko.testkit.TestProbe import pekko.testkit.WithLogCapturing @@ -42,7 +42,7 @@ object CoordinatedShutdownShardingSpec { } @ccompatUsedUntil213 -class CoordinatedShutdownShardingSpec extends AkkaSpec(CoordinatedShutdownShardingSpec.config) with WithLogCapturing { +class CoordinatedShutdownShardingSpec extends PekkoSpec(CoordinatedShutdownShardingSpec.config) with WithLogCapturing { import CoordinatedShutdownShardingSpec._ val sys1 = ActorSystem(system.name, system.settings.config) diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/DeprecatedLeastShardAllocationStrategySpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/DeprecatedLeastShardAllocationStrategySpec.scala index 9b8ed7d775..5758917dd7 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/DeprecatedLeastShardAllocationStrategySpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/DeprecatedLeastShardAllocationStrategySpec.scala @@ -14,12 +14,12 @@ import pekko.cluster.MemberStatus import pekko.cluster.UniqueAddress import pekko.cluster.sharding.internal.AbstractLeastShardAllocationStrategy import pekko.cluster.sharding.internal.AbstractLeastShardAllocationStrategy.RegionEntry -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.util.Version import scala.collection.immutable.SortedSet -class DeprecatedLeastShardAllocationStrategySpec extends AkkaSpec { +class DeprecatedLeastShardAllocationStrategySpec extends PekkoSpec { import LeastShardAllocationStrategySpec._ val memberA = newUpMember("127.0.0.1") diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/EntityTerminationSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/EntityTerminationSpec.scala index d51bfe1fbf..a15bece198 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/EntityTerminationSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/EntityTerminationSpec.scala @@ -9,7 +9,7 @@ import pekko.actor.Actor import pekko.actor.Props import pekko.cluster.Cluster import pekko.cluster.MemberStatus -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender import pekko.testkit.WithLogCapturing import com.typesafe.config.ConfigFactory @@ -53,7 +53,7 @@ object EntityTerminationSpec { } } -class EntityTerminationSpec extends AkkaSpec(EntityTerminationSpec.config) with ImplicitSender with WithLogCapturing { +class EntityTerminationSpec extends PekkoSpec(EntityTerminationSpec.config) with ImplicitSender with WithLogCapturing { import EntityTerminationSpec._ diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/GetShardTypeNamesSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/GetShardTypeNamesSpec.scala index 3f4333f925..ccd11571a4 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/GetShardTypeNamesSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/GetShardTypeNamesSpec.scala @@ -7,7 +7,7 @@ package org.apache.pekko.cluster.sharding import org.apache.pekko import pekko.actor.Props import pekko.cluster.Cluster -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TestActors.EchoActor import pekko.testkit.WithLogCapturing @@ -33,7 +33,7 @@ object GetShardTypeNamesSpec { } } -class GetShardTypeNamesSpec extends AkkaSpec(GetShardTypeNamesSpec.config) with WithLogCapturing { +class GetShardTypeNamesSpec extends PekkoSpec(GetShardTypeNamesSpec.config) with WithLogCapturing { import GetShardTypeNamesSpec._ "GetShardTypeNames" must { diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/JoinConfigCompatCheckShardingSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/JoinConfigCompatCheckShardingSpec.scala index 16241b7af0..24e901ea3c 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/JoinConfigCompatCheckShardingSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/JoinConfigCompatCheckShardingSpec.scala @@ -13,9 +13,9 @@ import org.apache.pekko import pekko.actor.ActorSystem import pekko.cluster.{ Cluster, ClusterReadView } import pekko.testkit.WithLogCapturing -import pekko.testkit.{ AkkaSpec, LongRunningTest } +import pekko.testkit.{ LongRunningTest, PekkoSpec } -class JoinConfigCompatCheckShardingSpec extends AkkaSpec() with WithLogCapturing { +class JoinConfigCompatCheckShardingSpec extends PekkoSpec() with WithLogCapturing { def initCluster(system: ActorSystem): ClusterReadView = { val cluster = Cluster(system) diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/LeastShardAllocationStrategyRandomizedSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/LeastShardAllocationStrategyRandomizedSpec.scala index 1a6bc4daaa..5a572735dc 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/LeastShardAllocationStrategyRandomizedSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/LeastShardAllocationStrategyRandomizedSpec.scala @@ -15,11 +15,11 @@ import pekko.cluster.Member import pekko.cluster.sharding.ShardCoordinator.ShardAllocationStrategy import pekko.cluster.sharding.ShardRegion.ShardId import pekko.cluster.sharding.internal.LeastShardAllocationStrategy -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import scala.collection.immutable.SortedSet -class LeastShardAllocationStrategyRandomizedSpec extends AkkaSpec("pekko.loglevel = INFO") { +class LeastShardAllocationStrategyRandomizedSpec extends PekkoSpec("pekko.loglevel = INFO") { import LeastShardAllocationStrategySpec.{ afterRebalance, countShards, diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/LeastShardAllocationStrategySpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/LeastShardAllocationStrategySpec.scala index 6709c41c7d..f8d721c40e 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/LeastShardAllocationStrategySpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/LeastShardAllocationStrategySpec.scala @@ -23,7 +23,7 @@ import pekko.cluster.sharding.ShardRegion.ShardId import pekko.cluster.sharding.internal.AbstractLeastShardAllocationStrategy import pekko.cluster.sharding.internal.AbstractLeastShardAllocationStrategy.RegionEntry import pekko.cluster.sharding.internal.LeastShardAllocationStrategy -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.util.Version import scala.collection.immutable.SortedSet @@ -80,7 +80,7 @@ object LeastShardAllocationStrategySpec { new DummyActorRef(RootActorPath(member.address) / "system" / "fake" / idForDebug) } -class LeastShardAllocationStrategySpec extends AkkaSpec { +class LeastShardAllocationStrategySpec extends PekkoSpec { import LeastShardAllocationStrategySpec._ val memberA = newUpMember("127.0.0.1") diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/PersistentShardingMigrationSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/PersistentShardingMigrationSpec.scala index cea55ac196..24e70f4a61 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/PersistentShardingMigrationSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/PersistentShardingMigrationSpec.scala @@ -10,7 +10,7 @@ import pekko.actor.{ ActorRef, ActorSystem, Props } import pekko.cluster.sharding.ShardRegion.CurrentRegions import pekko.cluster.{ Cluster, MemberStatus } import pekko.persistence.PersistentActor -import pekko.testkit.{ AkkaSpec, ImplicitSender, TestProbe } +import pekko.testkit.{ ImplicitSender, PekkoSpec, TestProbe } import com.typesafe.config.{ Config, ConfigFactory } import scala.concurrent.Await @@ -104,7 +104,7 @@ object PersistentShardingMigrationSpec { } } -class PersistentShardingMigrationSpec extends AkkaSpec(PersistentShardingMigrationSpec.config) with ImplicitSender { +class PersistentShardingMigrationSpec extends PekkoSpec(PersistentShardingMigrationSpec.config) with ImplicitSender { import PersistentShardingMigrationSpec._ diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ProxyShardingSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ProxyShardingSpec.scala index 4f39180bd2..74cd0fe066 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ProxyShardingSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ProxyShardingSpec.scala @@ -10,7 +10,7 @@ import scala.concurrent.duration.FiniteDuration import org.apache.pekko import pekko.actor.ActorRef -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TestActors import pekko.testkit.WithLogCapturing @@ -26,7 +26,7 @@ object ProxyShardingSpec { """ } -class ProxyShardingSpec extends AkkaSpec(ProxyShardingSpec.config) with WithLogCapturing { +class ProxyShardingSpec extends PekkoSpec(ProxyShardingSpec.config) with WithLogCapturing { val role = "Shard" val clusterSharding: ClusterSharding = ClusterSharding(system) diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesAndStartEntitySpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesAndStartEntitySpec.scala index abf4970682..126e79ee1e 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesAndStartEntitySpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesAndStartEntitySpec.scala @@ -18,7 +18,7 @@ import pekko.cluster.sharding.Shard.GetShardStats import pekko.cluster.sharding.Shard.ShardStats import pekko.cluster.sharding.ShardRegion.StartEntity import pekko.cluster.sharding.ShardRegion.StartEntityAck -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender import pekko.testkit.WithLogCapturing @@ -58,7 +58,7 @@ object RememberEntitiesAndStartEntitySpec { // this test covers remember entities + StartEntity class RememberEntitiesAndStartEntitySpec - extends AkkaSpec(RememberEntitiesAndStartEntitySpec.config) + extends PekkoSpec(RememberEntitiesAndStartEntitySpec.config) with AnyWordSpecLike with ImplicitSender with WithLogCapturing { diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesBatchedUpdatesSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesBatchedUpdatesSpec.scala index 65dd431d94..ee37f62318 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesBatchedUpdatesSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesBatchedUpdatesSpec.scala @@ -7,7 +7,7 @@ package org.apache.pekko.cluster.sharding import org.apache.pekko import pekko.actor.{ Actor, ActorLogging, ActorRef, Props } import pekko.cluster.{ Cluster, MemberStatus } -import pekko.testkit.{ AkkaSpec, ImplicitSender, TestProbe } +import pekko.testkit.{ ImplicitSender, PekkoSpec, TestProbe } import com.typesafe.config.ConfigFactory import org.scalatest.wordspec.AnyWordSpecLike @@ -55,7 +55,7 @@ object RememberEntitiesBatchedUpdatesSpec { """.stripMargin) } class RememberEntitiesBatchedUpdatesSpec - extends AkkaSpec(RememberEntitiesBatchedUpdatesSpec.config) + extends PekkoSpec(RememberEntitiesBatchedUpdatesSpec.config) with AnyWordSpecLike with ImplicitSender { diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesFailureSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesFailureSpec.scala index 0c5528e395..f010f07927 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesFailureSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesFailureSpec.scala @@ -13,7 +13,7 @@ import pekko.cluster.sharding.ShardRegion.ShardId import pekko.cluster.sharding.internal.RememberEntitiesCoordinatorStore import pekko.cluster.sharding.internal.RememberEntitiesShardStore import pekko.cluster.sharding.internal.RememberEntitiesProvider -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TestException import pekko.testkit.TestProbe import pekko.testkit.WithLogCapturing @@ -190,7 +190,7 @@ object RememberEntitiesFailureSpec { } class RememberEntitiesFailureSpec - extends AkkaSpec(RememberEntitiesFailureSpec.config) + extends PekkoSpec(RememberEntitiesFailureSpec.config) with AnyWordSpecLike with WithLogCapturing { diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesShardIdExtractorChangeSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesShardIdExtractorChangeSpec.scala index 89d30bf194..30e216aa03 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesShardIdExtractorChangeSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RememberEntitiesShardIdExtractorChangeSpec.scala @@ -13,7 +13,7 @@ import pekko.actor.Props import pekko.cluster.Cluster import pekko.cluster.sharding.ShardRegion.CurrentRegions import pekko.persistence.PersistentActor -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender import pekko.testkit.TestProbe import com.typesafe.config.ConfigFactory @@ -81,7 +81,7 @@ object RememberEntitiesShardIdExtractorChangeSpec { val TypeName = "ShardIdExtractorChange" } class RememberEntitiesShardIdExtractorChangeSpec - extends AkkaSpec(PersistentShardingMigrationSpec.config) + extends PekkoSpec(PersistentShardingMigrationSpec.config) with ImplicitSender { import RememberEntitiesShardIdExtractorChangeSpec._ diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RemoveInternalClusterShardingDataSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RemoveInternalClusterShardingDataSpec.scala index 6c484cdd7a..178f7a7476 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RemoveInternalClusterShardingDataSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/RemoveInternalClusterShardingDataSpec.scala @@ -23,7 +23,7 @@ import pekko.persistence.Recovery import pekko.persistence.RecoveryCompleted import pekko.persistence.SnapshotOffer import pekko.persistence.SnapshotSelectionCriteria -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender import pekko.testkit.TestActors.EchoActor import pekko.testkit.WithLogCapturing @@ -99,7 +99,7 @@ object RemoveInternalClusterShardingDataSpec { } class RemoveInternalClusterShardingDataSpec - extends AkkaSpec(RemoveInternalClusterShardingDataSpec.config) + extends PekkoSpec(RemoveInternalClusterShardingDataSpec.config) with ImplicitSender with WithLogCapturing { import RemoveInternalClusterShardingDataSpec._ diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardRegionSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardRegionSpec.scala index 8465db2d0f..dcb24b1005 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardRegionSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardRegionSpec.scala @@ -12,7 +12,7 @@ import org.apache.pekko import pekko.actor.{ Actor, ActorLogging, ActorRef, ActorSystem, PoisonPill, Props } import pekko.cluster.{ Cluster, MemberStatus } import pekko.cluster.ClusterEvent.CurrentClusterState -import pekko.testkit.{ AkkaSpec, DeadLettersFilter, TestProbe, WithLogCapturing } +import pekko.testkit.{ DeadLettersFilter, PekkoSpec, TestProbe, WithLogCapturing } import pekko.testkit.TestEvent.Mute object ShardRegionSpec { @@ -63,7 +63,7 @@ object ShardRegionSpec { } } } -class ShardRegionSpec extends AkkaSpec(ShardRegionSpec.config) with WithLogCapturing { +class ShardRegionSpec extends PekkoSpec(ShardRegionSpec.config) with WithLogCapturing { import scala.concurrent.duration._ diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardWithLeaseSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardWithLeaseSpec.scala index 3a08b0b804..d15b646619 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardWithLeaseSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardWithLeaseSpec.scala @@ -12,7 +12,7 @@ import pekko.cluster.Cluster import pekko.cluster.MemberStatus import pekko.cluster.sharding.ShardRegion.ShardId import pekko.coordination.lease.{ LeaseUsageSettings, TestLeaseExt } -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.EventFilter import pekko.testkit.TestProbe import pekko.testkit.WithLogCapturing @@ -66,7 +66,7 @@ object ShardWithLeaseSpec { case class BadLease(msg: String) extends RuntimeException(msg) with NoStackTrace } -class ShardWithLeaseSpec extends AkkaSpec(ShardWithLeaseSpec.config) with WithLogCapturing { +class ShardWithLeaseSpec extends PekkoSpec(ShardWithLeaseSpec.config) with WithLogCapturing { import ShardWithLeaseSpec._ diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardingQueriesSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardingQueriesSpec.scala index 3d8b4810c0..ccc12b86e3 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardingQueriesSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/ShardingQueriesSpec.scala @@ -8,9 +8,9 @@ import org.apache.pekko import pekko.cluster.sharding.Shard.{ CurrentShardState, ShardStats } import pekko.cluster.sharding.ShardRegion.ShardState import pekko.cluster.sharding.ShardingQueries.ShardsQueryResult -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class ShardingQueriesSpec extends AkkaSpec { +class ShardingQueriesSpec extends PekkoSpec { private val shards = Seq("a", "b", "busy") private val failures = Set("busy") diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/StartEntitySpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/StartEntitySpec.scala index a7df4e6126..e0aed2ff41 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/StartEntitySpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/StartEntitySpec.scala @@ -10,7 +10,7 @@ import pekko.actor.ActorRef import pekko.actor.Props import pekko.cluster.Cluster import pekko.cluster.MemberStatus -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender import pekko.testkit.WithLogCapturing import com.typesafe.config.ConfigFactory @@ -62,7 +62,7 @@ object StartEntitySpec { } -class StartEntitySpec extends AkkaSpec(StartEntitySpec.config) with ImplicitSender with WithLogCapturing { +class StartEntitySpec extends PekkoSpec(StartEntitySpec.config) with ImplicitSender with WithLogCapturing { import StartEntitySpec._ val extractEntityId: ShardRegion.ExtractEntityId = { diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/SupervisionSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/SupervisionSpec.scala index fc9b348b6c..dd98164fd8 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/SupervisionSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/SupervisionSpec.scala @@ -13,7 +13,7 @@ import pekko.cluster.sharding.ShardRegion.Passivate import pekko.pattern.{ BackoffOpts, BackoffSupervisor } import pekko.testkit.EventFilter import pekko.testkit.WithLogCapturing -import pekko.testkit.{ AkkaSpec, ImplicitSender } +import pekko.testkit.{ ImplicitSender, PekkoSpec } object SupervisionSpec { val config = @@ -70,7 +70,7 @@ object SupervisionSpec { } -class DeprecatedSupervisionSpec extends AkkaSpec(SupervisionSpec.config) with ImplicitSender with WithLogCapturing { +class DeprecatedSupervisionSpec extends PekkoSpec(SupervisionSpec.config) with ImplicitSender with WithLogCapturing { import SupervisionSpec._ "Supervision for a sharded actor (deprecated)" must { @@ -116,7 +116,7 @@ class DeprecatedSupervisionSpec extends AkkaSpec(SupervisionSpec.config) with Im } } -class SupervisionSpec extends AkkaSpec(SupervisionSpec.config) with ImplicitSender with WithLogCapturing { +class SupervisionSpec extends PekkoSpec(SupervisionSpec.config) with ImplicitSender with WithLogCapturing { import SupervisionSpec._ diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/external/ExternalShardAllocationStrategySpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/external/ExternalShardAllocationStrategySpec.scala index 4fe2702c11..77b31bd07f 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/external/ExternalShardAllocationStrategySpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/external/ExternalShardAllocationStrategySpec.scala @@ -10,11 +10,11 @@ import org.apache.pekko import pekko.cluster.sharding.external.ExternalShardAllocationStrategy.GetShardLocation import pekko.cluster.sharding.external.ExternalShardAllocationStrategy.GetShardLocationResponse import pekko.cluster.sharding.external.ExternalShardAllocationStrategy.GetShardLocations -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TestProbe import pekko.util.Timeout -class ExternalShardAllocationStrategySpec extends AkkaSpec(""" +class ExternalShardAllocationStrategySpec extends PekkoSpec(""" pekko.actor.provider = cluster pekko.loglevel = INFO pekko.remote.artery.canonical.port = 0 diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/internal/RememberEntitiesShardStoreSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/internal/RememberEntitiesShardStoreSpec.scala index 9717575498..2d212a7689 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/internal/RememberEntitiesShardStoreSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/internal/RememberEntitiesShardStoreSpec.scala @@ -12,7 +12,7 @@ import pekko.cluster.ddata.{ Replicator, ReplicatorSettings } import pekko.cluster.sharding.ClusterShardingSettings import pekko.cluster.sharding.ShardRegion.ShardId import pekko.cluster.{ Cluster, MemberStatus } -import pekko.testkit.{ AkkaSpec, ImplicitSender, WithLogCapturing } +import pekko.testkit.{ ImplicitSender, PekkoSpec, WithLogCapturing } import com.typesafe.config.ConfigFactory import org.scalatest.wordspec.AnyWordSpecLike @@ -42,7 +42,7 @@ object RememberEntitiesShardStoreSpec { // shared base class for both persistence and ddata specs abstract class RememberEntitiesShardStoreSpec - extends AkkaSpec(RememberEntitiesShardStoreSpec.config) + extends PekkoSpec(RememberEntitiesShardStoreSpec.config) with AnyWordSpecLike with ImplicitSender with WithLogCapturing { diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/internal/RememberEntitiesStarterSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/internal/RememberEntitiesStarterSpec.scala index 885fce37ce..8e362f06fc 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/internal/RememberEntitiesStarterSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/internal/RememberEntitiesStarterSpec.scala @@ -9,13 +9,13 @@ import pekko.cluster.sharding.ClusterShardingSettings import pekko.cluster.sharding.Shard import pekko.cluster.sharding.ShardRegion import pekko.cluster.sharding.ShardRegion.ShardId -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TestProbe import com.typesafe.config.ConfigFactory import scala.concurrent.duration._ -class RememberEntitiesStarterSpec extends AkkaSpec { +class RememberEntitiesStarterSpec extends PekkoSpec { var shardIdCounter = 1 def nextShardId(): ShardId = { diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/passivation/EntityPassivationSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/passivation/EntityPassivationSpec.scala index 5614a46c76..6dbbda9cd3 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/passivation/EntityPassivationSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/passivation/EntityPassivationSpec.scala @@ -13,7 +13,7 @@ import pekko.cluster.sharding.ClusterSharding import pekko.cluster.sharding.ClusterShardingSettings import pekko.cluster.sharding.ShardRegion import pekko.testkit.WithLogCapturing -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TestProbe import com.typesafe.config.Config import com.typesafe.config.ConfigFactory @@ -77,7 +77,7 @@ object EntityPassivationSpec { } abstract class AbstractEntityPassivationSpec(config: Config, expectedEntities: Int) - extends AkkaSpec(config) + extends PekkoSpec(config) with Eventually with WithLogCapturing { diff --git a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/protobuf/ClusterShardingMessageSerializerSpec.scala b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/protobuf/ClusterShardingMessageSerializerSpec.scala index 9e62e039b4..4a4b36d6b7 100644 --- a/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/protobuf/ClusterShardingMessageSerializerSpec.scala +++ b/akka-cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/protobuf/ClusterShardingMessageSerializerSpec.scala @@ -16,9 +16,9 @@ import pekko.cluster.sharding.ShardRegion.ShardId import pekko.cluster.sharding.internal.EventSourcedRememberEntitiesShardStore import pekko.cluster.sharding.internal.EventSourcedRememberEntitiesShardStore.EntitiesStarted import pekko.serialization.SerializationExtension -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class ClusterShardingMessageSerializerSpec extends AkkaSpec { +class ClusterShardingMessageSerializerSpec extends PekkoSpec { import ShardCoordinator.Internal._ val serializer = new ClusterShardingMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) diff --git a/akka-cluster-tools/src/main/scala/org/apache/pekko/cluster/singleton/ClusterSingletonManager.scala b/akka-cluster-tools/src/main/scala/org/apache/pekko/cluster/singleton/ClusterSingletonManager.scala index a5c4004054..1aa48c23f3 100644 --- a/akka-cluster-tools/src/main/scala/org/apache/pekko/cluster/singleton/ClusterSingletonManager.scala +++ b/akka-cluster-tools/src/main/scala/org/apache/pekko/cluster/singleton/ClusterSingletonManager.scala @@ -11,7 +11,7 @@ import scala.concurrent.duration._ import scala.util.control.NonFatal import com.typesafe.config.Config import org.apache.pekko -import pekko.AkkaException +import pekko.PekkoException import pekko.Done import pekko.actor.Actor import pekko.actor.ActorRef @@ -430,7 +430,7 @@ object ClusterSingletonManager { * state. Parent supervisor should typically restart the actor, i.e. * default decision. */ -class ClusterSingletonManagerIsStuck(message: String) extends AkkaException(message, null) +class ClusterSingletonManagerIsStuck(message: String) extends PekkoException(message, null) /** * Manages singleton actor instance among all cluster nodes or a group diff --git a/akka-cluster-tools/src/test/java/org/apache/pekko/cluster/client/ClusterClientTest.java b/akka-cluster-tools/src/test/java/org/apache/pekko/cluster/client/ClusterClientTest.java index 2248156882..b7a24640dc 100644 --- a/akka-cluster-tools/src/test/java/org/apache/pekko/cluster/client/ClusterClientTest.java +++ b/akka-cluster-tools/src/test/java/org/apache/pekko/cluster/client/ClusterClientTest.java @@ -12,14 +12,14 @@ import java.util.Set; import org.junit.ClassRule; import org.junit.Test; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import org.scalatestplus.junit.JUnitSuite; public class ClusterClientTest extends JUnitSuite { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource( + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource( "DistributedPubSubMediatorTest", ConfigFactory.parseString( "pekko.actor.provider = \"cluster\"\n" diff --git a/akka-cluster-tools/src/test/java/org/apache/pekko/cluster/pubsub/DistributedPubSubMediatorTest.java b/akka-cluster-tools/src/test/java/org/apache/pekko/cluster/pubsub/DistributedPubSubMediatorTest.java index 178f390191..226674f1cc 100644 --- a/akka-cluster-tools/src/test/java/org/apache/pekko/cluster/pubsub/DistributedPubSubMediatorTest.java +++ b/akka-cluster-tools/src/test/java/org/apache/pekko/cluster/pubsub/DistributedPubSubMediatorTest.java @@ -6,7 +6,7 @@ package org.apache.pekko.cluster.pubsub; import com.typesafe.config.ConfigFactory; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import org.junit.ClassRule; import org.junit.Test; @@ -22,8 +22,8 @@ import org.scalatestplus.junit.JUnitSuite; public class DistributedPubSubMediatorTest extends JUnitSuite { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource( + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource( "DistributedPubSubMediatorTest", ConfigFactory.parseString( "pekko.actor.provider = \"cluster\"\n" diff --git a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/client/protobuf/ClusterClientMessageSerializerSpec.scala b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/client/protobuf/ClusterClientMessageSerializerSpec.scala index 129b534fcb..7de840e131 100644 --- a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/client/protobuf/ClusterClientMessageSerializerSpec.scala +++ b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/client/protobuf/ClusterClientMessageSerializerSpec.scala @@ -9,10 +9,10 @@ import scala.annotation.nowarn import org.apache.pekko import pekko.actor.ExtendedActorSystem import pekko.cluster.client.ClusterReceptionist.Internal._ -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec @nowarn("msg=deprecated") -class ClusterClientMessageSerializerSpec extends AkkaSpec { +class ClusterClientMessageSerializerSpec extends PekkoSpec { val serializer = new ClusterClientMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) diff --git a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/pubsub/DistributedPubSubMediatorDeadLettersSpec.scala b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/pubsub/DistributedPubSubMediatorDeadLettersSpec.scala index 01a500704e..c590e10abe 100644 --- a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/pubsub/DistributedPubSubMediatorDeadLettersSpec.scala +++ b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/pubsub/DistributedPubSubMediatorDeadLettersSpec.scala @@ -32,7 +32,7 @@ trait DeadLettersProbe { this: TestKitBase => } class DistributedPubSubMediatorSendingToDeadLettersSpec - extends AkkaSpec(DistributedPubSubMediatorDeadLettersSpec.config(sendToDeadLettersWhenNoSubscribers = true)) + extends PekkoSpec(DistributedPubSubMediatorDeadLettersSpec.config(sendToDeadLettersWhenNoSubscribers = true)) with DeadLettersProbe { val mediator = DistributedPubSub(system).mediator @@ -80,7 +80,7 @@ class DistributedPubSubMediatorSendingToDeadLettersSpec } class DistributedPubSubMediatorNotSendingToDeadLettersSpec - extends AkkaSpec(DistributedPubSubMediatorDeadLettersSpec.config(sendToDeadLettersWhenNoSubscribers = false)) + extends PekkoSpec(DistributedPubSubMediatorDeadLettersSpec.config(sendToDeadLettersWhenNoSubscribers = false)) with DeadLettersProbe { val mediator = DistributedPubSub(system).mediator diff --git a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/pubsub/DistributedPubSubMediatorRouterSpec.scala b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/pubsub/DistributedPubSubMediatorRouterSpec.scala index 8fef3aef43..ddc83393bb 100644 --- a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/pubsub/DistributedPubSubMediatorRouterSpec.scala +++ b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/pubsub/DistributedPubSubMediatorRouterSpec.scala @@ -91,7 +91,7 @@ trait DistributedPubSubMediatorRouterSpec { this: AnyWordSpecLike with TestKit w } class DistributedPubSubMediatorWithRandomRouterSpec - extends AkkaSpec(DistributedPubSubMediatorRouterSpec.config("random")) + extends PekkoSpec(DistributedPubSubMediatorRouterSpec.config("random")) with DistributedPubSubMediatorRouterSpec with DefaultTimeout with ImplicitSender { @@ -110,7 +110,7 @@ class DistributedPubSubMediatorWithRandomRouterSpec } class DistributedPubSubMediatorWithHashRouterSpec - extends AkkaSpec(DistributedPubSubMediatorRouterSpec.config("consistent-hashing")) + extends PekkoSpec(DistributedPubSubMediatorRouterSpec.config("consistent-hashing")) with DistributedPubSubMediatorRouterSpec with DefaultTimeout with ImplicitSender { diff --git a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/pubsub/protobuf/DistributedPubSubMessageSerializerSpec.scala b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/pubsub/protobuf/DistributedPubSubMessageSerializerSpec.scala index d89cd9169a..014592b054 100644 --- a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/pubsub/protobuf/DistributedPubSubMessageSerializerSpec.scala +++ b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/pubsub/protobuf/DistributedPubSubMessageSerializerSpec.scala @@ -11,9 +11,9 @@ import pekko.actor.{ Address, ExtendedActorSystem } import pekko.actor.Props import pekko.cluster.pubsub.DistributedPubSubMediator._ import pekko.cluster.pubsub.DistributedPubSubMediator.Internal._ -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class DistributedPubSubMessageSerializerSpec extends AkkaSpec { +class DistributedPubSubMessageSerializerSpec extends PekkoSpec { val serializer = new DistributedPubSubMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) diff --git a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeaseSpec.scala b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeaseSpec.scala index 18994b4d82..09ed29d831 100644 --- a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeaseSpec.scala +++ b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeaseSpec.scala @@ -23,7 +23,7 @@ import pekko.cluster.Cluster import pekko.cluster.MemberStatus import pekko.coordination.lease.TestLease import pekko.coordination.lease.TestLeaseExt -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TestException import pekko.testkit.TestProbe @@ -45,7 +45,7 @@ class ImportantSingleton(lifeCycleProbe: ActorRef) extends Actor with ActorLoggi } } -class ClusterSingletonLeaseSpec extends AkkaSpec(ConfigFactory.parseString(""" +class ClusterSingletonLeaseSpec extends PekkoSpec(ConfigFactory.parseString(""" pekko.loglevel = INFO pekko.actor.provider = cluster diff --git a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala index 81167048b0..8e58a60b72 100644 --- a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala +++ b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala @@ -18,7 +18,7 @@ import pekko.actor.Props import pekko.cluster.Cluster import pekko.cluster.MemberStatus import pekko.cluster.singleton.ClusterSingletonLeavingSpeedSpec.TheSingleton -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TestProbe object ClusterSingletonLeavingSpeedSpec { @@ -42,7 +42,7 @@ object ClusterSingletonLeavingSpeedSpec { } class ClusterSingletonLeavingSpeedSpec - extends AkkaSpec( + extends PekkoSpec( """ pekko.loglevel = DEBUG pekko.actor.provider = org.apache.pekko.cluster.ClusterActorRefProvider diff --git a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestart2Spec.scala b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestart2Spec.scala index cb715cce58..7912d5d121 100644 --- a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestart2Spec.scala +++ b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestart2Spec.scala @@ -16,7 +16,7 @@ import pekko.actor.Props import pekko.cluster.Cluster import pekko.cluster.MemberStatus import pekko.cluster.UniqueAddress -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TestProbe object ClusterSingletonRestart2Spec { @@ -30,7 +30,7 @@ object ClusterSingletonRestart2Spec { } class ClusterSingletonRestart2Spec - extends AkkaSpec(""" + extends PekkoSpec(""" pekko.loglevel = INFO pekko.cluster.roles = [singleton] pekko.actor.provider = org.apache.pekko.cluster.ClusterActorRefProvider diff --git a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestartSpec.scala b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestartSpec.scala index affda39aa4..4ffd770328 100644 --- a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestartSpec.scala +++ b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestartSpec.scala @@ -13,12 +13,12 @@ import pekko.actor.ActorSystem import pekko.actor.PoisonPill import pekko.cluster.Cluster import pekko.cluster.MemberStatus -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TestActors import pekko.testkit.TestProbe class ClusterSingletonRestartSpec - extends AkkaSpec(""" + extends PekkoSpec(""" pekko.loglevel = INFO pekko.actor.provider = org.apache.pekko.cluster.ClusterActorRefProvider pekko.cluster.downing-provider-class = org.apache.pekko.cluster.testkit.AutoDowning diff --git a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/protobuf/ClusterSingletonMessageSerializerSpec.scala b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/protobuf/ClusterSingletonMessageSerializerSpec.scala index 7893afc17a..d2fc6dff9e 100644 --- a/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/protobuf/ClusterSingletonMessageSerializerSpec.scala +++ b/akka-cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/protobuf/ClusterSingletonMessageSerializerSpec.scala @@ -10,9 +10,9 @@ import pekko.cluster.singleton.ClusterSingletonManager.Internal.HandOverDone import pekko.cluster.singleton.ClusterSingletonManager.Internal.HandOverInProgress import pekko.cluster.singleton.ClusterSingletonManager.Internal.HandOverToMe import pekko.cluster.singleton.ClusterSingletonManager.Internal.TakeOverFromMe -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class ClusterSingletonMessageSerializerSpec extends AkkaSpec { +class ClusterSingletonMessageSerializerSpec extends PekkoSpec { val serializer = new ClusterSingletonMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) diff --git a/akka-cluster-typed/src/main/resources/reference.conf b/akka-cluster-typed/src/main/resources/reference.conf index 064fcf7d4e..b49cf3a699 100644 --- a/akka-cluster-typed/src/main/resources/reference.conf +++ b/akka-cluster-typed/src/main/resources/reference.conf @@ -47,11 +47,11 @@ pekko.cluster.ddata.typed { pekko { actor { serialization-identifiers { - "org.apache.pekko.cluster.typed.internal.AkkaClusterTypedSerializer" = 28 + "org.apache.pekko.cluster.typed.internal.PekkoClusterTypedSerializer" = 28 "org.apache.pekko.cluster.typed.internal.delivery.ReliableDeliverySerializer" = 36 } serializers { - typed-cluster = "org.apache.pekko.cluster.typed.internal.AkkaClusterTypedSerializer" + typed-cluster = "org.apache.pekko.cluster.typed.internal.PekkoClusterTypedSerializer" reliable-delivery = "org.apache.pekko.cluster.typed.internal.delivery.ReliableDeliverySerializer" } serialization-bindings { diff --git a/akka-cluster-typed/src/main/scala/org/apache/pekko/cluster/typed/internal/AkkaClusterTypedSerializer.scala b/akka-cluster-typed/src/main/scala/org/apache/pekko/cluster/typed/internal/PekkoClusterTypedSerializer.scala similarity index 97% rename from akka-cluster-typed/src/main/scala/org/apache/pekko/cluster/typed/internal/AkkaClusterTypedSerializer.scala rename to akka-cluster-typed/src/main/scala/org/apache/pekko/cluster/typed/internal/PekkoClusterTypedSerializer.scala index 3c7f33a581..96ff195cc3 100644 --- a/akka-cluster-typed/src/main/scala/org/apache/pekko/cluster/typed/internal/AkkaClusterTypedSerializer.scala +++ b/akka-cluster-typed/src/main/scala/org/apache/pekko/cluster/typed/internal/PekkoClusterTypedSerializer.scala @@ -21,7 +21,7 @@ import pekko.serialization.{ BaseSerializer, SerializerWithStringManifest } * INTERNAL API */ @InternalApi -private[pekko] final class AkkaClusterTypedSerializer(override val system: ExtendedActorSystem) +private[pekko] final class PekkoClusterTypedSerializer(override val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer { diff --git a/akka-cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/RemoteMessageSpec.scala b/akka-cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/RemoteMessageSpec.scala index e4753f6d4b..3516cd578b 100644 --- a/akka-cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/RemoteMessageSpec.scala +++ b/akka-cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/RemoteMessageSpec.scala @@ -16,7 +16,7 @@ import pekko.actor.typed.ActorRefResolver import pekko.actor.typed.scaladsl.Behaviors import pekko.actor.typed.scaladsl.adapter._ import pekko.serialization.jackson.CborSerializable -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec object RemoteMessageSpec { def config = ConfigFactory.parseString(s""" @@ -36,7 +36,7 @@ object RemoteMessageSpec { case class Ping(sender: ActorRef[String]) extends CborSerializable } -class RemoteMessageSpec extends AkkaSpec(RemoteMessageSpec.config) { +class RemoteMessageSpec extends PekkoSpec(RemoteMessageSpec.config) { import RemoteMessageSpec._ diff --git a/akka-cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/internal/AkkaClusterTypedSerializerSpec.scala b/akka-cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/internal/PekkoClusterTypedSerializerSpec.scala similarity index 80% rename from akka-cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/internal/AkkaClusterTypedSerializerSpec.scala rename to akka-cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/internal/PekkoClusterTypedSerializerSpec.scala index 60fc5ed7f3..be87f91074 100644 --- a/akka-cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/internal/AkkaClusterTypedSerializerSpec.scala +++ b/akka-cluster-typed/src/test/scala/org/apache/pekko/cluster/typed/internal/PekkoClusterTypedSerializerSpec.scala @@ -15,19 +15,19 @@ import pekko.actor.typed.scaladsl.adapter._ import pekko.cluster.typed.internal.receptionist.ClusterReceptionist import pekko.serialization.SerializationExtension -class AkkaClusterTypedSerializerSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing { +class PekkoClusterTypedSerializerSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing { val ref = spawn(Behaviors.empty[String]) val classicSystem = system.toClassic - val serializer = new AkkaClusterTypedSerializer(classicSystem.asInstanceOf[ExtendedActorSystem]) + val serializer = new PekkoClusterTypedSerializer(classicSystem.asInstanceOf[ExtendedActorSystem]) - "AkkaClusterTypedSerializer" must { + "PekkoClusterTypedSerializer" must { Seq("ReceptionistEntry" -> ClusterReceptionist.Entry(ref, 666L)(System.currentTimeMillis())).foreach { case (scenario, item) => s"resolve serializer for $scenario" in { val serializer = SerializationExtension(classicSystem) - serializer.serializerFor(item.getClass).getClass should be(classOf[AkkaClusterTypedSerializer]) + serializer.serializerFor(item.getClass).getClass should be(classOf[PekkoClusterTypedSerializer]) } s"serialize and de-serialize $scenario" in { diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterConfigSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterConfigSpec.scala index 0c2605b888..264e8f80da 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterConfigSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterConfigSpec.scala @@ -14,12 +14,12 @@ import org.apache.pekko import pekko.actor.Address import pekko.dispatch.Dispatchers import pekko.remote.PhiAccrualFailureDetector -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.util.Helpers.ConfigOps import pekko.util.Version @nowarn -class ClusterConfigSpec extends AkkaSpec { +class ClusterConfigSpec extends PekkoSpec { "Clustering" must { diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterDeployerSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterDeployerSpec.scala index e4520fd16b..136c9ddbf7 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterDeployerSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterDeployerSpec.scala @@ -48,7 +48,7 @@ object ClusterDeployerSpec { } -class ClusterDeployerSpec extends AkkaSpec(ClusterDeployerSpec.deployerConf) { +class ClusterDeployerSpec extends PekkoSpec(ClusterDeployerSpec.deployerConf) { "A RemoteDeployer" must { diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterDomainEventPublisherSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterDomainEventPublisherSpec.scala index 4c2d98efdc..0b4e7bb243 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterDomainEventPublisherSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterDomainEventPublisherSpec.scala @@ -18,7 +18,7 @@ import pekko.cluster.ClusterSettings.DefaultDataCenter import pekko.cluster.InternalClusterAction._ import pekko.cluster.MemberStatus._ import pekko.remote.RARP -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender import pekko.testkit.TestProbe @@ -31,7 +31,7 @@ object ClusterDomainEventPublisherSpec { } class ClusterDomainEventPublisherSpec - extends AkkaSpec(ClusterDomainEventPublisherSpec.config) + extends PekkoSpec(ClusterDomainEventPublisherSpec.config) with BeforeAndAfterEach with ImplicitSender { diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterHeartbeatReceiverSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterHeartbeatReceiverSpec.scala index f04cbe188f..cc748687f4 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterHeartbeatReceiverSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterHeartbeatReceiverSpec.scala @@ -6,9 +6,9 @@ package org.apache.pekko.cluster import org.apache.pekko import pekko.cluster.ClusterHeartbeatSender.{ Heartbeat, HeartbeatRsp } -import pekko.testkit.{ AkkaSpec, ImplicitSender } +import pekko.testkit.{ ImplicitSender, PekkoSpec } -class ClusterHeartbeatReceiverSpec extends AkkaSpec(""" +class ClusterHeartbeatReceiverSpec extends PekkoSpec(""" pekko.actor.provider = cluster """.stripMargin) with ImplicitSender { "ClusterHeartbeatReceiver" should { diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterHeartbeatSenderSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterHeartbeatSenderSpec.scala index fa7fa92e91..35fb43182b 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterHeartbeatSenderSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterHeartbeatSenderSpec.scala @@ -9,7 +9,7 @@ import pekko.actor.{ ActorSelection, Address, Props } import pekko.cluster.ClusterEvent.{ CurrentClusterState, MemberUp } import pekko.cluster.ClusterHeartbeatSender.Heartbeat import pekko.cluster.ClusterHeartbeatSenderSpec.TestClusterHeartBeatSender -import pekko.testkit.{ AkkaSpec, ImplicitSender, TestProbe } +import pekko.testkit.{ ImplicitSender, PekkoSpec, TestProbe } import pekko.util.Version object ClusterHeartbeatSenderSpec { @@ -24,7 +24,7 @@ object ClusterHeartbeatSenderSpec { } } -class ClusterHeartbeatSenderSpec extends AkkaSpec(""" +class ClusterHeartbeatSenderSpec extends PekkoSpec(""" pekko.loglevel = DEBUG pekko.actor.provider = cluster pekko.cluster.failure-detector.heartbeat-interval = 0.2s diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterLogSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterLogSpec.scala index eff2141e9d..c0f2b73647 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterLogSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterLogSpec.scala @@ -8,7 +8,7 @@ import com.typesafe.config.{ Config, ConfigFactory } import org.apache.pekko import pekko.actor.{ Address, ExtendedActorSystem } -import pekko.testkit.{ AkkaSpec, EventFilter, ImplicitSender } +import pekko.testkit.{ EventFilter, ImplicitSender, PekkoSpec } object ClusterLogSpec { val config = """ @@ -28,7 +28,7 @@ object ClusterLogSpec { } -abstract class ClusterLogSpec(config: Config) extends AkkaSpec(config) with ImplicitSender { +abstract class ClusterLogSpec(config: Config) extends PekkoSpec(config) with ImplicitSender { def this(s: String) = this(ConfigFactory.parseString(s)) diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterSpec.scala index 2250bf3ea6..e98f4a7a31 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterSpec.scala @@ -25,7 +25,7 @@ import pekko.stream.Materializer import pekko.stream.scaladsl.Sink import pekko.stream.scaladsl.Source import pekko.stream.scaladsl.StreamRefs -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender import pekko.testkit.TestProbe import pekko.util.Version @@ -49,7 +49,7 @@ object ClusterSpec { final case class GossipTo(address: Address) } -class ClusterSpec extends AkkaSpec(ClusterSpec.config) with ImplicitSender { +class ClusterSpec extends PekkoSpec(ClusterSpec.config) with ImplicitSender { val selfAddress = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterTestKit.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterTestKit.scala index 767bc71b93..c3678d6442 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterTestKit.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ClusterTestKit.scala @@ -4,14 +4,14 @@ package org.apache.pekko.cluster -import scala.concurrent.duration.{ FiniteDuration, _ } +import scala.concurrent.duration._ import scala.util.Random import com.typesafe.config.{ Config, ConfigFactory } import org.apache.pekko import pekko.actor.ActorSystem -import pekko.testkit.{ AkkaSpec, TestKitBase } +import pekko.testkit.{ PekkoSpec, TestKitBase } /** * Builds on TestKitBase to provide some extra utilities to run cluster test. @@ -158,7 +158,7 @@ trait ClusterTestKit extends TestKitBase { } } -abstract class RollingUpgradeClusterSpec(config: Config) extends AkkaSpec(config) with ClusterTestKit { +abstract class RollingUpgradeClusterSpec(config: Config) extends PekkoSpec(config) with ClusterTestKit { /** * Starts `size` diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/CrossDcHeartbeatSenderSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/CrossDcHeartbeatSenderSpec.scala index bb7bdcb6d1..edfb1cbea8 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/CrossDcHeartbeatSenderSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/CrossDcHeartbeatSenderSpec.scala @@ -14,7 +14,7 @@ import pekko.cluster.ClusterEvent.CurrentClusterState import pekko.cluster.ClusterHeartbeatSender.Heartbeat import pekko.cluster.CrossDcHeartbeatSender.ReportStatus import pekko.cluster.CrossDcHeartbeatSenderSpec.TestCrossDcHeartbeatSender -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender import pekko.testkit.TestProbe import pekko.util.Version @@ -30,7 +30,7 @@ object CrossDcHeartbeatSenderSpec { } } -class CrossDcHeartbeatSenderSpec extends AkkaSpec(""" +class CrossDcHeartbeatSenderSpec extends PekkoSpec(""" pekko.loglevel = DEBUG pekko.actor.provider = cluster # should not be used here diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/JoinConfigCompatCheckClusterSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/JoinConfigCompatCheckClusterSpec.scala index f050cfad95..e47797654b 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/JoinConfigCompatCheckClusterSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/JoinConfigCompatCheckClusterSpec.scala @@ -9,9 +9,9 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.ExtendedActorSystem -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class JoinConfigCompatCheckClusterSpec extends AkkaSpec { +class JoinConfigCompatCheckClusterSpec extends PekkoSpec { private val extSystem = system.asInstanceOf[ExtendedActorSystem] private val clusterSettings = new ClusterSettings(system.settings.config, system.name) diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/JoinConfigCompatCheckerSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/JoinConfigCompatCheckerSpec.scala index 5aa00eeb9a..ed7695b4e2 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/JoinConfigCompatCheckerSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/JoinConfigCompatCheckerSpec.scala @@ -9,7 +9,7 @@ import scala.concurrent.duration._ import com.typesafe.config.{ Config, ConfigFactory } -import org.apache.pekko.testkit.{ AkkaSpec, LongRunningTest } +import org.apache.pekko.testkit.{ LongRunningTest, PekkoSpec } object JoinConfigCompatCheckerSpec { @@ -45,7 +45,7 @@ object JoinConfigCompatCheckerSpec { """).withFallback(baseConfig) } -class JoinConfigCompatCheckerSpec extends AkkaSpec with ClusterTestKit { +class JoinConfigCompatCheckerSpec extends PekkoSpec with ClusterTestKit { import JoinConfigCompatCheckerSpec._ "A Joining Node" must { @@ -253,7 +253,7 @@ class JoinConfigCompatCheckerSpec extends AkkaSpec with ClusterTestKit { } } - /** This test verifies the built-in JoinConfigCompatCheckerAkkaCluster */ + /** This test verifies the built-in JoinConfigCompatCheckerPekkoCluster */ "NOT be allowed to join a cluster using a different value for pekko.cluster.downing-provider-class" taggedAs LongRunningTest in { val joinNodeConfig = diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ShutdownAfterJoinSeedNodesSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ShutdownAfterJoinSeedNodesSpec.scala index 962614ce5e..f72f1bffde 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/ShutdownAfterJoinSeedNodesSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/ShutdownAfterJoinSeedNodesSpec.scala @@ -29,7 +29,7 @@ object ShutdownAfterJoinSeedNodesSpec { """ } -class ShutdownAfterJoinSeedNodesSpec extends AkkaSpec(ShutdownAfterJoinSeedNodesSpec.config) { +class ShutdownAfterJoinSeedNodesSpec extends PekkoSpec(ShutdownAfterJoinSeedNodesSpec.config) { val seed1 = ActorSystem(system.name, system.settings.config) val seed2 = ActorSystem(system.name, system.settings.config) diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/StartupWithOneThreadSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/StartupWithOneThreadSpec.scala index 59a1b027a9..bda645cd8d 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/StartupWithOneThreadSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/StartupWithOneThreadSpec.scala @@ -11,7 +11,7 @@ import pekko.actor.Actor import pekko.actor.ActorLogging import pekko.actor.Address import pekko.actor.Props -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender object StartupWithOneThreadSpec { @@ -42,7 +42,7 @@ object StartupWithOneThreadSpec { }) } -class StartupWithOneThreadSpec(startTime: Long) extends AkkaSpec(StartupWithOneThreadSpec.config) with ImplicitSender { +class StartupWithOneThreadSpec(startTime: Long) extends PekkoSpec(StartupWithOneThreadSpec.config) with ImplicitSender { import StartupWithOneThreadSpec._ def this() = this(System.nanoTime()) @@ -56,7 +56,7 @@ class StartupWithOneThreadSpec(startTime: Long) extends AkkaSpec(StartupWithOneT // they also tried to get the Cluster extension and thereby blocking // dispatcher threads. // Note that the Cluster extension is started via ClusterActorRefProvider - // before ActorSystem.apply returns, i.e. in the constructor of AkkaSpec. + // before ActorSystem.apply returns, i.e. in the constructor of PekkoSpec. (System.nanoTime - startTime).nanos.toMillis should be < (system.settings.CreationTimeout.duration - 2.second).toMillis system.actorOf(testProps) ! "hello" diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/VectorClockSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/VectorClockSpec.scala index 66c484f0e9..a84f3009ce 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/VectorClockSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/VectorClockSpec.scala @@ -6,9 +6,9 @@ package org.apache.pekko.cluster import scala.collection.immutable.TreeMap -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec -class VectorClockSpec extends AkkaSpec { +class VectorClockSpec extends PekkoSpec { import VectorClock._ "A VectorClock" must { diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/protobuf/ClusterMessageSerializerSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/protobuf/ClusterMessageSerializerSpec.scala index 5e75de2413..9241c8af06 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/protobuf/ClusterMessageSerializerSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/protobuf/ClusterMessageSerializerSpec.scala @@ -15,11 +15,11 @@ import pekko.cluster._ import pekko.cluster.InternalClusterAction.CompatibleConfig import pekko.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings } import pekko.routing.RoundRobinPool -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.util.Version @nowarn -class ClusterMessageSerializerSpec extends AkkaSpec("pekko.actor.provider = cluster") { +class ClusterMessageSerializerSpec extends PekkoSpec("pekko.actor.provider = cluster") { val serializer = new ClusterMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/routing/ClusterRouterSupervisorSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/routing/ClusterRouterSupervisorSpec.scala index 5649f511f7..71a6da6d46 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/routing/ClusterRouterSupervisorSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/routing/ClusterRouterSupervisorSpec.scala @@ -23,7 +23,7 @@ object ClusterRouterSupervisorSpec { } -class ClusterRouterSupervisorSpec extends AkkaSpec(""" +class ClusterRouterSupervisorSpec extends PekkoSpec(""" pekko.actor.provider = "cluster" pekko.remote.classic.netty.tcp.port = 0 pekko.remote.artery.canonical.port = 0 diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/sbr/LeaseMajoritySpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/sbr/LeaseMajoritySpec.scala index 4b39e3e103..2caa064352 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/sbr/LeaseMajoritySpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/sbr/LeaseMajoritySpec.scala @@ -4,11 +4,11 @@ package org.apache.pekko.cluster.sbr -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import com.typesafe.config.ConfigFactory import org.scalatest.concurrent.Eventually -class LeaseMajoritySpec extends AkkaSpec() with Eventually { +class LeaseMajoritySpec extends PekkoSpec() with Eventually { val default = ConfigFactory .parseString( diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/sbr/SplitBrainResolverSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/sbr/SplitBrainResolverSpec.scala index 218e8b3b2a..b5f29e357c 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/sbr/SplitBrainResolverSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/sbr/SplitBrainResolverSpec.scala @@ -31,7 +31,7 @@ import pekko.cluster._ import pekko.coordination.lease.LeaseSettings import pekko.coordination.lease.TestLease import pekko.coordination.lease.TimeoutSettings -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.EventFilter object SplitBrainResolverSpec { @@ -94,7 +94,7 @@ object SplitBrainResolverSpec { } class SplitBrainResolverSpec - extends AkkaSpec(""" + extends PekkoSpec(""" |pekko { | actor.provider = cluster | cluster.downing-provider-class = "org.apache.pekko.cluster.sbr.SplitBrainResolverProvider" diff --git a/akka-cluster/src/test/scala/org/apache/pekko/cluster/testkit/AutoDownSpec.scala b/akka-cluster/src/test/scala/org/apache/pekko/cluster/testkit/AutoDownSpec.scala index b965152bd3..b035fdc099 100644 --- a/akka-cluster/src/test/scala/org/apache/pekko/cluster/testkit/AutoDownSpec.scala +++ b/akka-cluster/src/test/scala/org/apache/pekko/cluster/testkit/AutoDownSpec.scala @@ -16,7 +16,7 @@ import pekko.cluster.Member import pekko.cluster.MemberStatus._ import pekko.cluster.TestMember import pekko.remote.RARP -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TimingTest object AutoDownSpec { @@ -39,7 +39,7 @@ object AutoDownSpec { } -class AutoDownSpec extends AkkaSpec(""" +class AutoDownSpec extends PekkoSpec(""" |pekko.actor.provider=remote |pekko.remote.warn-about-direct-use=off |""".stripMargin) { diff --git a/akka-coordination/src/test/java/org/apache/pekko/coordination/lease/javadsl/LeaseProviderTest.java b/akka-coordination/src/test/java/org/apache/pekko/coordination/lease/javadsl/LeaseProviderTest.java index 72895a746a..3958081d36 100644 --- a/akka-coordination/src/test/java/org/apache/pekko/coordination/lease/javadsl/LeaseProviderTest.java +++ b/akka-coordination/src/test/java/org/apache/pekko/coordination/lease/javadsl/LeaseProviderTest.java @@ -6,7 +6,7 @@ package org.apache.pekko.coordination.lease.javadsl; import org.apache.pekko.actor.ActorSystem; import org.apache.pekko.coordination.lease.scaladsl.LeaseProviderSpec; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -15,8 +15,8 @@ import static org.junit.Assert.assertEquals; public class LeaseProviderTest { @Rule - public AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("LoggingAdapterTest", LeaseProviderSpec.config()); + public PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("LoggingAdapterTest", LeaseProviderSpec.config()); private ActorSystem system = null; diff --git a/akka-coordination/src/test/scala/org/apache/pekko/coordination/lease/scaladsl/LeaseProviderSpec.scala b/akka-coordination/src/test/scala/org/apache/pekko/coordination/lease/scaladsl/LeaseProviderSpec.scala index 039f29c698..19cc3eae2a 100644 --- a/akka-coordination/src/test/scala/org/apache/pekko/coordination/lease/scaladsl/LeaseProviderSpec.scala +++ b/akka-coordination/src/test/scala/org/apache/pekko/coordination/lease/scaladsl/LeaseProviderSpec.scala @@ -12,7 +12,7 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.ExtendedActorSystem import pekko.coordination.lease.LeaseSettings -import pekko.testkit.{ AkkaSpec, EventFilter } +import pekko.testkit.{ EventFilter, PekkoSpec } object LeaseProviderSpec { class LeaseA(settings: LeaseSettings) extends Lease(settings) { @@ -64,7 +64,7 @@ object LeaseProviderSpec { """) } -class LeaseProviderSpec extends AkkaSpec(LeaseProviderSpec.config) { +class LeaseProviderSpec extends PekkoSpec(LeaseProviderSpec.config) { import LeaseProviderSpec._ "LeaseProvider" must { diff --git a/akka-discovery/src/test/scala/org/apache/pekko/discovery/dns/DnsServiceDiscoverySpec.scala b/akka-discovery/src/test/scala/org/apache/pekko/discovery/dns/DnsServiceDiscoverySpec.scala index a8702d4f6c..70e62003c9 100644 --- a/akka-discovery/src/test/scala/org/apache/pekko/discovery/dns/DnsServiceDiscoverySpec.scala +++ b/akka-discovery/src/test/scala/org/apache/pekko/discovery/dns/DnsServiceDiscoverySpec.scala @@ -22,10 +22,10 @@ import pekko.discovery.ServiceDiscovery.{ Resolved, ResolvedTarget } import pekko.discovery.ServiceDiscovery.DiscoveryTimeoutException import pekko.io.dns.{ AAAARecord, ARecord, DnsProtocol, SRVRecord } import pekko.io.dns.CachePolicy.Ttl -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.TestProbe -class DnsServiceDiscoverySpec extends AkkaSpec with AnyWordSpecLike with Matchers with ScalaFutures { +class DnsServiceDiscoverySpec extends PekkoSpec with AnyWordSpecLike with Matchers with ScalaFutures { "DnsServiceDiscovery" must { "fail future with DiscoveryTimeoutException if IP dns resolve does not respond" in { diff --git a/akka-distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala b/akka-distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala index 85998e0feb..a6164ac578 100644 --- a/akka-distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala +++ b/akka-distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala @@ -31,7 +31,7 @@ import pekko.remote.ByteStringUtils import pekko.serialization.BaseSerializer import pekko.serialization.Serialization import pekko.serialization.SerializerWithStringManifest -import pekko.util.{ ByteString => AkkaByteString } +import pekko.util.{ ByteString => PekkoByteString } import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ @@ -283,7 +283,7 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem) val fromSystemUid = if (status.hasFromSystemUid) Some(status.getFromSystemUid) else None Status( status.getEntriesList.asScala.iterator - .map(e => e.getKey -> AkkaByteString.fromArrayUnsafe(e.getDigest.toByteArray())) + .map(e => e.getKey -> PekkoByteString.fromArrayUnsafe(e.getDigest.toByteArray())) .toMap, status.getChunk, status.getTotChunks, diff --git a/akka-distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/ReplicatorSettingsSpec.scala b/akka-distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/ReplicatorSettingsSpec.scala index 4cec67498a..c724e4c02f 100644 --- a/akka-distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/ReplicatorSettingsSpec.scala +++ b/akka-distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/ReplicatorSettingsSpec.scala @@ -8,7 +8,7 @@ import com.typesafe.config.ConfigFactory import org.scalatest.BeforeAndAfterAll import org.scalatest.wordspec.AnyWordSpecLike -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec object ReplicatorSettingsSpec { @@ -20,7 +20,7 @@ object ReplicatorSettingsSpec { } class ReplicatorSettingsSpec - extends AkkaSpec(ReplicatorSettingsSpec.config) + extends PekkoSpec(ReplicatorSettingsSpec.config) with AnyWordSpecLike with BeforeAndAfterAll { diff --git a/akka-distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/WriteAggregatorSpec.scala b/akka-distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/WriteAggregatorSpec.scala index cb80abff76..040fefa10f 100644 --- a/akka-distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/WriteAggregatorSpec.scala +++ b/akka-distributed-data/src/test/scala/org/apache/pekko/cluster/ddata/WriteAggregatorSpec.scala @@ -129,7 +129,7 @@ object WriteAggregatorSpec { } } -class WriteAggregatorSpec extends AkkaSpec(s""" +class WriteAggregatorSpec extends PekkoSpec(s""" pekko.actor.provider = "cluster" pekko.remote.classic.netty.tcp.port = 0 pekko.remote.artery.canonical.port = 0 diff --git a/akka-multi-node-testkit/src/main/scala/org/apache/pekko/remote/testconductor/Conductor.scala b/akka-multi-node-testkit/src/main/scala/org/apache/pekko/remote/testconductor/Conductor.scala index ebd09d2b2b..2ab5b50c1e 100644 --- a/akka-multi-node-testkit/src/main/scala/org/apache/pekko/remote/testconductor/Conductor.scala +++ b/akka-multi-node-testkit/src/main/scala/org/apache/pekko/remote/testconductor/Conductor.scala @@ -24,7 +24,7 @@ import org.jboss.netty.channel.{ } import org.apache.pekko -import pekko.AkkaException +import pekko.PekkoException import pekko.ConfigurationException import pekko.actor.{ Actor, @@ -407,7 +407,7 @@ private[pekko] class ServerFSM(val controller: ActorRef, val channel: Channel) */ private[pekko] object Controller { final case class ClientDisconnected(name: RoleName) extends DeadLetterSuppression - class ClientDisconnectedException(msg: String) extends AkkaException(msg) with NoStackTrace + class ClientDisconnectedException(msg: String) extends PekkoException(msg) with NoStackTrace case object GetNodes case object GetSockAddr final case class CreateServerFSM(channel: Channel) extends NoSerializationVerificationNeeded diff --git a/akka-persistence-query/src/test/java/org/apache/pekko/persistence/query/PersistenceQueryTest.java b/akka-persistence-query/src/test/java/org/apache/pekko/persistence/query/PersistenceQueryTest.java index cb23456951..a554246f5e 100644 --- a/akka-persistence-query/src/test/java/org/apache/pekko/persistence/query/PersistenceQueryTest.java +++ b/akka-persistence-query/src/test/java/org/apache/pekko/persistence/query/PersistenceQueryTest.java @@ -6,14 +6,14 @@ package org.apache.pekko.persistence.query; import org.apache.pekko.NotUsed; import org.apache.pekko.actor.ActorSystem; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import org.junit.ClassRule; public class PersistenceQueryTest { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource(PersistenceQueryTest.class.getName()); + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource(PersistenceQueryTest.class.getName()); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/internal/QuerySerializerSpec.scala b/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/internal/QuerySerializerSpec.scala index d15fd156d8..779d32bf3e 100644 --- a/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/internal/QuerySerializerSpec.scala +++ b/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/internal/QuerySerializerSpec.scala @@ -15,9 +15,9 @@ import pekko.persistence.query.TimestampOffset import pekko.persistence.query.typed.EventEnvelope import pekko.serialization.SerializationExtension import pekko.serialization.SerializerWithStringManifest -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class QuerySerializerSpec extends AkkaSpec { +class QuerySerializerSpec extends PekkoSpec { private val serialization = SerializationExtension(system) diff --git a/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/AllPersistenceIdsSpec.scala b/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/AllPersistenceIdsSpec.scala index 1c47d3dbce..05905674b2 100644 --- a/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/AllPersistenceIdsSpec.scala +++ b/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/AllPersistenceIdsSpec.scala @@ -10,7 +10,7 @@ import pekko.persistence.query.PersistenceQuery import pekko.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal import pekko.persistence.query.scaladsl.PersistenceIdsQuery import pekko.stream.testkit.scaladsl.TestSink -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender import scala.annotation.nowarn @@ -27,7 +27,7 @@ object AllPersistenceIdsSpec { """ } -class AllPersistenceIdsSpec extends AkkaSpec(AllPersistenceIdsSpec.config) with Cleanup with ImplicitSender { +class AllPersistenceIdsSpec extends PekkoSpec(AllPersistenceIdsSpec.config) with Cleanup with ImplicitSender { @nowarn("msg=deprecated") val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier) diff --git a/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/Cleanup.scala b/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/Cleanup.scala index 782b671387..80f6237655 100644 --- a/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/Cleanup.scala +++ b/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/Cleanup.scala @@ -8,9 +8,9 @@ import java.io.File import org.apache.commons.io.FileUtils -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec -trait Cleanup { this: AkkaSpec => +trait Cleanup { this: PekkoSpec => val storageLocations = List( "pekko.persistence.journal.leveldb.dir", diff --git a/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/EventsByPersistenceIdSpec.scala b/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/EventsByPersistenceIdSpec.scala index 5daddb4275..d1aed1c0d6 100644 --- a/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/EventsByPersistenceIdSpec.scala +++ b/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/EventsByPersistenceIdSpec.scala @@ -12,7 +12,7 @@ import pekko.persistence.query.PersistenceQuery import pekko.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal import pekko.persistence.query.scaladsl.EventsByTagQuery import pekko.stream.testkit.scaladsl.TestSink -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender import scala.annotation.nowarn @@ -30,7 +30,7 @@ object EventsByPersistenceIdSpec { """ } -class EventsByPersistenceIdSpec extends AkkaSpec(EventsByPersistenceIdSpec.config) with Cleanup with ImplicitSender { +class EventsByPersistenceIdSpec extends PekkoSpec(EventsByPersistenceIdSpec.config) with Cleanup with ImplicitSender { @nowarn("msg=deprecated") val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier) diff --git a/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/EventsByTagSpec.scala b/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/EventsByTagSpec.scala index c29ffb063b..232d9f3af1 100644 --- a/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/EventsByTagSpec.scala +++ b/akka-persistence-query/src/test/scala/org/apache/pekko/persistence/query/journal/leveldb/EventsByTagSpec.scala @@ -15,7 +15,7 @@ import pekko.persistence.query.Sequence import pekko.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal import pekko.persistence.query.scaladsl.EventsByTagQuery import pekko.stream.testkit.scaladsl.TestSink -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.testkit.ImplicitSender import scala.annotation.nowarn @@ -61,7 +61,7 @@ class ColorTagger extends WriteEventAdapter { override def manifest(event: Any): String = "" } -class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config) with Cleanup with ImplicitSender { +class EventsByTagSpec extends PekkoSpec(EventsByTagSpec.config) with Cleanup with ImplicitSender { @nowarn("msg=deprecated") val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier) diff --git a/akka-persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/PersistencePluginProxySpec.scala b/akka-persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/PersistencePluginProxySpec.scala index b2da15e91c..5ba1ba2178 100644 --- a/akka-persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/PersistencePluginProxySpec.scala +++ b/akka-persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/PersistencePluginProxySpec.scala @@ -10,7 +10,7 @@ import org.apache.pekko import pekko.actor._ import pekko.persistence._ import pekko.persistence.journal.PersistencePluginProxy -import pekko.testkit.{ AkkaSpec, TestProbe } +import pekko.testkit.{ PekkoSpec, TestProbe } object PersistencePluginProxySpec { lazy val config = @@ -96,7 +96,7 @@ object PersistencePluginProxySpec { } class PersistencePluginProxySpec - extends AkkaSpec(PersistencePluginProxySpec.startTargetConfig.withFallback(PersistencePluginProxySpec.config)) + extends PekkoSpec(PersistencePluginProxySpec.startTargetConfig.withFallback(PersistencePluginProxySpec.config)) with Cleanup { import PersistencePluginProxySpec._ diff --git a/akka-persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala b/akka-persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala index 94d489b089..0dc6ee75ff 100644 --- a/akka-persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala +++ b/akka-persistence-shared/src/test/scala/org/apache/pekko/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala @@ -9,7 +9,7 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor._ import pekko.persistence._ -import pekko.testkit.{ AkkaSpec, TestProbe } +import pekko.testkit.{ PekkoSpec, TestProbe } object SharedLeveldbJournalSpec { val config = ConfigFactory.parseString(s""" @@ -72,7 +72,7 @@ object SharedLeveldbJournalSpec { } -class SharedLeveldbJournalSpec extends AkkaSpec(SharedLeveldbJournalSpec.config) with Cleanup { +class SharedLeveldbJournalSpec extends PekkoSpec(SharedLeveldbJournalSpec.config) with Cleanup { import SharedLeveldbJournalSpec._ val systemA = ActorSystem("SysA", system.settings.config) diff --git a/akka-persistence-shared/src/test/scala/org/apache/pekko/persistence/serialization/SerializerSpec.scala b/akka-persistence-shared/src/test/scala/org/apache/pekko/persistence/serialization/SerializerSpec.scala index 920dead0ce..4da32c5b44 100644 --- a/akka-persistence-shared/src/test/scala/org/apache/pekko/persistence/serialization/SerializerSpec.scala +++ b/akka-persistence-shared/src/test/scala/org/apache/pekko/persistence/serialization/SerializerSpec.scala @@ -73,7 +73,7 @@ object SerializerSpecConfigs { import pekko.persistence.serialization.SerializerSpecConfigs._ -class SnapshotSerializerPersistenceSpec extends AkkaSpec(customSerializers) { +class SnapshotSerializerPersistenceSpec extends PekkoSpec(customSerializers) { val serialization = SerializationExtension(system) "A snapshot serializer" must { @@ -167,7 +167,7 @@ class SnapshotSerializerPersistenceSpec extends AkkaSpec(customSerializers) { } } -class MessageSerializerPersistenceSpec extends AkkaSpec(customSerializers) { +class MessageSerializerPersistenceSpec extends PekkoSpec(customSerializers) { val serialization = SerializationExtension(system) "A message serializer" when { @@ -326,7 +326,7 @@ object MessageSerializerRemotingSpec { system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress } -class MessageSerializerRemotingSpec extends AkkaSpec(remote.withFallback(customSerializers)) with DefaultTimeout { +class MessageSerializerRemotingSpec extends PekkoSpec(remote.withFallback(customSerializers)) with DefaultTimeout { import MessageSerializerRemotingSpec._ val remoteSystem = ActorSystem("remote", remote.withFallback(customSerializers)) diff --git a/akka-persistence-typed-tests/src/test/scala/org/apache/pekko/persistence/typed/jackson/ReplicatedEventSourcingJacksonSpec.scala b/akka-persistence-typed-tests/src/test/scala/org/apache/pekko/persistence/typed/jackson/ReplicatedEventSourcingJacksonSpec.scala index a4f912d4a2..99ee853cc2 100644 --- a/akka-persistence-typed-tests/src/test/scala/org/apache/pekko/persistence/typed/jackson/ReplicatedEventSourcingJacksonSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/org/apache/pekko/persistence/typed/jackson/ReplicatedEventSourcingJacksonSpec.scala @@ -9,20 +9,20 @@ import pekko.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTest import pekko.persistence.typed.ReplicaId import pekko.persistence.typed.crdt.{ Counter, LwwTime, ORSet } import pekko.persistence.typed.jackson.ReplicatedEventSourcingJacksonSpec.{ WithCounter, WithLwwTime, WithOrSet } -import pekko.serialization.jackson.{ AkkaSerializationDeserializer, AkkaSerializationSerializer, JsonSerializable } +import pekko.serialization.jackson.{ JsonSerializable, PekkoSerializationDeserializer, PekkoSerializationSerializer } import com.fasterxml.jackson.databind.annotation.{ JsonDeserialize, JsonSerialize } import org.scalatest.wordspec.AnyWordSpecLike object ReplicatedEventSourcingJacksonSpec { final case class WithLwwTime(lwwTime: LwwTime) extends JsonSerializable final case class WithOrSet( - @JsonDeserialize(`using` = classOf[AkkaSerializationDeserializer]) - @JsonSerialize(`using` = classOf[AkkaSerializationSerializer]) + @JsonDeserialize(`using` = classOf[PekkoSerializationDeserializer]) + @JsonSerialize(`using` = classOf[PekkoSerializationSerializer]) orSet: ORSet[String]) extends JsonSerializable final case class WithCounter( - @JsonDeserialize(`using` = classOf[AkkaSerializationDeserializer]) - @JsonSerialize(`using` = classOf[AkkaSerializationSerializer]) + @JsonDeserialize(`using` = classOf[PekkoSerializationDeserializer]) + @JsonSerialize(`using` = classOf[PekkoSerializationSerializer]) counter: Counter) extends JsonSerializable diff --git a/akka-persistence/src/main/scala/org/apache/pekko/persistence/journal/AsyncWriteProxy.scala b/akka-persistence/src/main/scala/org/apache/pekko/persistence/journal/AsyncWriteProxy.scala index 5733267027..e19edb1c60 100644 --- a/akka-persistence/src/main/scala/org/apache/pekko/persistence/journal/AsyncWriteProxy.scala +++ b/akka-persistence/src/main/scala/org/apache/pekko/persistence/journal/AsyncWriteProxy.scala @@ -10,7 +10,7 @@ import scala.concurrent.duration.Duration import scala.util.Try import org.apache.pekko -import pekko.AkkaException +import pekko.PekkoException import pekko.actor._ import pekko.pattern.ask import pekko.persistence._ @@ -128,7 +128,7 @@ private[persistence] object AsyncWriteTarget { * Thrown if replay inactivity exceeds a specified timeout. */ @SerialVersionUID(1L) -class AsyncReplayTimeoutException(msg: String) extends AkkaException(msg) +class AsyncReplayTimeoutException(msg: String) extends PekkoException(msg) private class ReplayMediator( replayCallback: PersistentRepr => Unit, diff --git a/akka-persistence/src/test/scala/org/apache/pekko/persistence/AtLeastOnceDeliveryCrashSpec.scala b/akka-persistence/src/test/scala/org/apache/pekko/persistence/AtLeastOnceDeliveryCrashSpec.scala index 418e672982..906ceda2ac 100644 --- a/akka-persistence/src/test/scala/org/apache/pekko/persistence/AtLeastOnceDeliveryCrashSpec.scala +++ b/akka-persistence/src/test/scala/org/apache/pekko/persistence/AtLeastOnceDeliveryCrashSpec.scala @@ -10,7 +10,7 @@ import scala.util.control.NoStackTrace import org.apache.pekko import pekko.actor._ import pekko.actor.SupervisorStrategy.{ Escalate, Stop } -import pekko.testkit.{ AkkaSpec, ImplicitSender, TestProbe } +import pekko.testkit.{ ImplicitSender, PekkoSpec, TestProbe } object AtLeastOnceDeliveryCrashSpec { @@ -65,7 +65,7 @@ object AtLeastOnceDeliveryCrashSpec { } class AtLeastOnceDeliveryCrashSpec - extends AkkaSpec(PersistenceSpec.config("inmem", "AtLeastOnceDeliveryCrashSpec", serialization = "off")) + extends PekkoSpec(PersistenceSpec.config("inmem", "AtLeastOnceDeliveryCrashSpec", serialization = "off")) with ImplicitSender { import AtLeastOnceDeliveryCrashSpec._ "At least once delivery" should { diff --git a/akka-persistence/src/test/scala/org/apache/pekko/persistence/AtLeastOnceDeliveryFailureSpec.scala b/akka-persistence/src/test/scala/org/apache/pekko/persistence/AtLeastOnceDeliveryFailureSpec.scala index f35395a495..723ee37f66 100644 --- a/akka-persistence/src/test/scala/org/apache/pekko/persistence/AtLeastOnceDeliveryFailureSpec.scala +++ b/akka-persistence/src/test/scala/org/apache/pekko/persistence/AtLeastOnceDeliveryFailureSpec.scala @@ -170,7 +170,7 @@ object AtLeastOnceDeliveryFailureSpec { } class AtLeastOnceDeliveryFailureSpec - extends AkkaSpec(AtLeastOnceDeliveryFailureSpec.config) + extends PekkoSpec(AtLeastOnceDeliveryFailureSpec.config) with Cleanup with ImplicitSender { import AtLeastOnceDeliveryFailureSpec._ diff --git a/akka-persistence/src/test/scala/org/apache/pekko/persistence/PersistenceSpec.scala b/akka-persistence/src/test/scala/org/apache/pekko/persistence/PersistenceSpec.scala index b49f5e7b6b..bda6317764 100644 --- a/akka-persistence/src/test/scala/org/apache/pekko/persistence/PersistenceSpec.scala +++ b/akka-persistence/src/test/scala/org/apache/pekko/persistence/PersistenceSpec.scala @@ -18,13 +18,13 @@ import org.scalatest.matchers.{ MatchResult, Matcher } import org.apache.pekko import pekko.actor.Props -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec abstract class PersistenceSpec(config: Config) - extends AkkaSpec(config) + extends PekkoSpec(config) with BeforeAndAfterEach with Cleanup - with PersistenceMatchers { this: AkkaSpec => + with PersistenceMatchers { this: PekkoSpec => private var _name: String = _ lazy val extension = Persistence(system) @@ -78,7 +78,7 @@ object PersistenceSpec { """)) } -trait Cleanup { this: AkkaSpec => +trait Cleanup { this: PekkoSpec => val storageLocations = List("pekko.persistence.snapshot-store.local.dir").map(s => new File(system.settings.config.getString(s))) diff --git a/akka-persistence/src/test/scala/org/apache/pekko/persistence/PersistentActorJournalProtocolSpec.scala b/akka-persistence/src/test/scala/org/apache/pekko/persistence/PersistentActorJournalProtocolSpec.scala index 0bb737968e..6bb99c48a5 100644 --- a/akka-persistence/src/test/scala/org/apache/pekko/persistence/PersistentActorJournalProtocolSpec.scala +++ b/akka-persistence/src/test/scala/org/apache/pekko/persistence/PersistentActorJournalProtocolSpec.scala @@ -101,7 +101,7 @@ class JournalPuppet extends Actor { import PersistentActorJournalProtocolSpec._ -class PersistentActorJournalProtocolSpec extends AkkaSpec(config) with ImplicitSender { +class PersistentActorJournalProtocolSpec extends PekkoSpec(config) with ImplicitSender { val journal = JournalPuppet(system).probe diff --git a/akka-persistence/src/test/scala/org/apache/pekko/persistence/PersistentActorRecoveryTimeoutSpec.scala b/akka-persistence/src/test/scala/org/apache/pekko/persistence/PersistentActorRecoveryTimeoutSpec.scala index 715263ccbd..e8487b55b8 100644 --- a/akka-persistence/src/test/scala/org/apache/pekko/persistence/PersistentActorRecoveryTimeoutSpec.scala +++ b/akka-persistence/src/test/scala/org/apache/pekko/persistence/PersistentActorRecoveryTimeoutSpec.scala @@ -12,7 +12,7 @@ import org.apache.pekko import pekko.actor.{ Actor, ActorLogging, ActorRef, Props } import pekko.actor.Status.Failure import pekko.persistence.journal.SteppingInmemJournal -import pekko.testkit.{ AkkaSpec, ImplicitSender, TestProbe } +import pekko.testkit.{ ImplicitSender, PekkoSpec, TestProbe } object PersistentActorRecoveryTimeoutSpec { val journalId = "persistent-actor-recovery-timeout-spec" @@ -69,7 +69,7 @@ object PersistentActorRecoveryTimeoutSpec { } class PersistentActorRecoveryTimeoutSpec - extends AkkaSpec(PersistentActorRecoveryTimeoutSpec.config) + extends PekkoSpec(PersistentActorRecoveryTimeoutSpec.config) with ImplicitSender { import PersistentActorRecoveryTimeoutSpec.journalId diff --git a/akka-persistence/src/test/scala/org/apache/pekko/persistence/SnapshotDirectoryFailureSpec.scala b/akka-persistence/src/test/scala/org/apache/pekko/persistence/SnapshotDirectoryFailureSpec.scala index c4561f0892..93274e3c05 100644 --- a/akka-persistence/src/test/scala/org/apache/pekko/persistence/SnapshotDirectoryFailureSpec.scala +++ b/akka-persistence/src/test/scala/org/apache/pekko/persistence/SnapshotDirectoryFailureSpec.scala @@ -8,7 +8,7 @@ import java.io.{ File, IOException } import org.apache.pekko import pekko.actor.{ ActorInitializationException, ActorRef, Props } -import pekko.testkit.{ AkkaSpec, EventFilter, ImplicitSender } +import pekko.testkit.{ EventFilter, ImplicitSender, PekkoSpec } object SnapshotDirectoryFailureSpec { val inUseSnapshotPath = "target/inUseSnapshotPath" @@ -30,7 +30,7 @@ object SnapshotDirectoryFailureSpec { } class SnapshotDirectoryFailureSpec - extends AkkaSpec( + extends PekkoSpec( PersistenceSpec.config( "inmem", "SnapshotDirectoryFailureSpec", diff --git a/akka-persistence/src/test/scala/org/apache/pekko/persistence/journal/InmemEventAdaptersSpec.scala b/akka-persistence/src/test/scala/org/apache/pekko/persistence/journal/InmemEventAdaptersSpec.scala index 15c72189eb..dcd8f69626 100644 --- a/akka-persistence/src/test/scala/org/apache/pekko/persistence/journal/InmemEventAdaptersSpec.scala +++ b/akka-persistence/src/test/scala/org/apache/pekko/persistence/journal/InmemEventAdaptersSpec.scala @@ -7,12 +7,12 @@ package org.apache.pekko.persistence.journal import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.ExtendedActorSystem -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import scala.annotation.nowarn @nowarn("msg=Unused import") -class InmemEventAdaptersSpec extends AkkaSpec { +class InmemEventAdaptersSpec extends PekkoSpec { val config = ConfigFactory.parseString(s""" |pekko.persistence.journal { diff --git a/akka-persistence/src/test/scala/org/apache/pekko/persistence/journal/ReplayFilterSpec.scala b/akka-persistence/src/test/scala/org/apache/pekko/persistence/journal/ReplayFilterSpec.scala index 3fd87ac912..2fd3388573 100644 --- a/akka-persistence/src/test/scala/org/apache/pekko/persistence/journal/ReplayFilterSpec.scala +++ b/akka-persistence/src/test/scala/org/apache/pekko/persistence/journal/ReplayFilterSpec.scala @@ -9,7 +9,7 @@ import pekko.persistence.JournalProtocol import pekko.persistence.PersistentRepr import pekko.testkit._ -class ReplayFilterSpec extends AkkaSpec with ImplicitSender { +class ReplayFilterSpec extends PekkoSpec with ImplicitSender { import JournalProtocol._ import ReplayFilter.{ Fail, RepairByDiscardOld, Warn } diff --git a/akka-persistence/src/test/scala/org/apache/pekko/persistence/serialization/MessageSerializerSpec.scala b/akka-persistence/src/test/scala/org/apache/pekko/persistence/serialization/MessageSerializerSpec.scala index 8b7a8ae62d..aebea63667 100644 --- a/akka-persistence/src/test/scala/org/apache/pekko/persistence/serialization/MessageSerializerSpec.scala +++ b/akka-persistence/src/test/scala/org/apache/pekko/persistence/serialization/MessageSerializerSpec.scala @@ -7,9 +7,9 @@ package org.apache.pekko.persistence.serialization import org.apache.pekko import pekko.persistence.PersistentRepr import pekko.serialization.SerializationExtension -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec -class MessageSerializerSpec extends AkkaSpec { +class MessageSerializerSpec extends PekkoSpec { "Message serializer" should { "serialize metadata for persistent repr" in { diff --git a/akka-remote-tests/src/test/scala/org/apache/pekko/remote/testconductor/BarrierSpec.scala b/akka-remote-tests/src/test/scala/org/apache/pekko/remote/testconductor/BarrierSpec.scala index c136a33ad4..c78095ea6c 100644 --- a/akka-remote-tests/src/test/scala/org/apache/pekko/remote/testconductor/BarrierSpec.scala +++ b/akka-remote-tests/src/test/scala/org/apache/pekko/remote/testconductor/BarrierSpec.scala @@ -12,7 +12,7 @@ import language.postfixOps import org.apache.pekko import pekko.actor._ -import pekko.testkit.{ AkkaSpec, EventFilter, ImplicitSender, TestProbe, TimingTest } +import pekko.testkit.{ EventFilter, ImplicitSender, PekkoSpec, TestProbe, TimingTest } object BarrierSpec { final case class Failed(ref: ActorRef, thr: Throwable) @@ -24,7 +24,7 @@ object BarrierSpec { """ } -class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender { +class BarrierSpec extends PekkoSpec(BarrierSpec.config) with ImplicitSender { import BarrierCoordinator._ import BarrierSpec._ diff --git a/akka-remote-tests/src/test/scala/org/apache/pekko/remote/testconductor/ControllerSpec.scala b/akka-remote-tests/src/test/scala/org/apache/pekko/remote/testconductor/ControllerSpec.scala index 9d272e32b4..7b5b0cc872 100644 --- a/akka-remote-tests/src/test/scala/org/apache/pekko/remote/testconductor/ControllerSpec.scala +++ b/akka-remote-tests/src/test/scala/org/apache/pekko/remote/testconductor/ControllerSpec.scala @@ -9,7 +9,7 @@ import java.net.InetSocketAddress import org.apache.pekko.actor.{ AddressFromURIString, PoisonPill, Props } import org.apache.pekko.remote.testconductor.Controller.NodeInfo -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import org.apache.pekko.testkit.ImplicitSender object ControllerSpec { @@ -21,7 +21,7 @@ object ControllerSpec { """ } -class ControllerSpec extends AkkaSpec(ControllerSpec.config) with ImplicitSender { +class ControllerSpec extends PekkoSpec(ControllerSpec.config) with ImplicitSender { val A = RoleName("a") val B = RoleName("b") diff --git a/akka-remote/src/main/java/org/apache/pekko/remote/WireFormats.java b/akka-remote/src/main/java/org/apache/pekko/remote/WireFormats.java index 6022b13f88..3f16721bde 100644 --- a/akka-remote/src/main/java/org/apache/pekko/remote/WireFormats.java +++ b/akka-remote/src/main/java/org/apache/pekko/remote/WireFormats.java @@ -9678,7 +9678,7 @@ public final class WireFormats { } - public interface AkkaProtocolMessageOrBuilder extends + public interface PekkoProtocolMessageOrBuilder extends // @@protoc_insertion_point(interface_extends:AkkaProtocolMessage) org.apache.pekko.protobufv3.internal.MessageOrBuilder { @@ -9702,11 +9702,11 @@ public final class WireFormats { * optional .AkkaControlMessage instruction = 2; * @return The instruction. */ - org.apache.pekko.remote.WireFormats.AkkaControlMessage getInstruction(); + PekkoControlMessage getInstruction(); /** * optional .AkkaControlMessage instruction = 2; */ - org.apache.pekko.remote.WireFormats.AkkaControlMessageOrBuilder getInstructionOrBuilder(); + PekkoControlMessageOrBuilder getInstructionOrBuilder(); } /** *
@@ -9717,16 +9717,16 @@ public final class WireFormats {
    *
    * Protobuf type {@code AkkaProtocolMessage}
    */
-  public  static final class AkkaProtocolMessage extends
+  public  static final class PekkoProtocolMessage extends
       org.apache.pekko.protobufv3.internal.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:AkkaProtocolMessage)
-      AkkaProtocolMessageOrBuilder {
+          PekkoProtocolMessageOrBuilder {
   private static final long serialVersionUID = 0L;
     // Use AkkaProtocolMessage.newBuilder() to construct.
-    private AkkaProtocolMessage(org.apache.pekko.protobufv3.internal.GeneratedMessageV3.Builder builder) {
+    private PekkoProtocolMessage(org.apache.pekko.protobufv3.internal.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
-    private AkkaProtocolMessage() {
+    private PekkoProtocolMessage() {
       payload_ = org.apache.pekko.protobufv3.internal.ByteString.EMPTY;
     }
 
@@ -9734,7 +9734,7 @@ public final class WireFormats {
     @SuppressWarnings({"unused"})
     protected java.lang.Object newInstance(
         org.apache.pekko.protobufv3.internal.GeneratedMessageV3.UnusedPrivateParameter unused) {
-      return new AkkaProtocolMessage();
+      return new PekkoProtocolMessage();
     }
 
     @java.lang.Override
@@ -9742,7 +9742,7 @@ public final class WireFormats {
     getUnknownFields() {
       return this.unknownFields;
     }
-    private AkkaProtocolMessage(
+    private PekkoProtocolMessage(
         org.apache.pekko.protobufv3.internal.CodedInputStream input,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
@@ -9767,11 +9767,11 @@ public final class WireFormats {
               break;
             }
             case 18: {
-              org.apache.pekko.remote.WireFormats.AkkaControlMessage.Builder subBuilder = null;
+              PekkoControlMessage.Builder subBuilder = null;
               if (((bitField0_ & 0x00000002) != 0)) {
                 subBuilder = instruction_.toBuilder();
               }
-              instruction_ = input.readMessage(org.apache.pekko.remote.WireFormats.AkkaControlMessage.PARSER, extensionRegistry);
+              instruction_ = input.readMessage(PekkoControlMessage.PARSER, extensionRegistry);
               if (subBuilder != null) {
                 subBuilder.mergeFrom(instruction_);
                 instruction_ = subBuilder.buildPartial();
@@ -9808,7 +9808,7 @@ public final class WireFormats {
         internalGetFieldAccessorTable() {
       return org.apache.pekko.remote.WireFormats.internal_static_AkkaProtocolMessage_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.pekko.remote.WireFormats.AkkaProtocolMessage.class, org.apache.pekko.remote.WireFormats.AkkaProtocolMessage.Builder.class);
+              PekkoProtocolMessage.class, PekkoProtocolMessage.Builder.class);
     }
 
     private int bitField0_;
@@ -9830,7 +9830,7 @@ public final class WireFormats {
     }
 
     public static final int INSTRUCTION_FIELD_NUMBER = 2;
-    private org.apache.pekko.remote.WireFormats.AkkaControlMessage instruction_;
+    private PekkoControlMessage instruction_;
     /**
      * optional .AkkaControlMessage instruction = 2;
      * @return Whether the instruction field is set.
@@ -9842,14 +9842,14 @@ public final class WireFormats {
      * optional .AkkaControlMessage instruction = 2;
      * @return The instruction.
      */
-    public org.apache.pekko.remote.WireFormats.AkkaControlMessage getInstruction() {
-      return instruction_ == null ? org.apache.pekko.remote.WireFormats.AkkaControlMessage.getDefaultInstance() : instruction_;
+    public PekkoControlMessage getInstruction() {
+      return instruction_ == null ? PekkoControlMessage.getDefaultInstance() : instruction_;
     }
     /**
      * optional .AkkaControlMessage instruction = 2;
      */
-    public org.apache.pekko.remote.WireFormats.AkkaControlMessageOrBuilder getInstructionOrBuilder() {
-      return instruction_ == null ? org.apache.pekko.remote.WireFormats.AkkaControlMessage.getDefaultInstance() : instruction_;
+    public PekkoControlMessageOrBuilder getInstructionOrBuilder() {
+      return instruction_ == null ? PekkoControlMessage.getDefaultInstance() : instruction_;
     }
 
     private byte memoizedIsInitialized = -1;
@@ -9905,10 +9905,10 @@ public final class WireFormats {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.pekko.remote.WireFormats.AkkaProtocolMessage)) {
+      if (!(obj instanceof PekkoProtocolMessage)) {
         return super.equals(obj);
       }
-      org.apache.pekko.remote.WireFormats.AkkaProtocolMessage other = (org.apache.pekko.remote.WireFormats.AkkaProtocolMessage) obj;
+      PekkoProtocolMessage other = (PekkoProtocolMessage) obj;
 
       if (hasPayload() != other.hasPayload()) return false;
       if (hasPayload()) {
@@ -9944,69 +9944,69 @@ public final class WireFormats {
       return hash;
     }
 
-    public static org.apache.pekko.remote.WireFormats.AkkaProtocolMessage parseFrom(
+    public static PekkoProtocolMessage parseFrom(
         java.nio.ByteBuffer data)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaProtocolMessage parseFrom(
+    public static PekkoProtocolMessage parseFrom(
         java.nio.ByteBuffer data,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaProtocolMessage parseFrom(
+    public static PekkoProtocolMessage parseFrom(
         org.apache.pekko.protobufv3.internal.ByteString data)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaProtocolMessage parseFrom(
+    public static PekkoProtocolMessage parseFrom(
         org.apache.pekko.protobufv3.internal.ByteString data,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaProtocolMessage parseFrom(byte[] data)
+    public static PekkoProtocolMessage parseFrom(byte[] data)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaProtocolMessage parseFrom(
+    public static PekkoProtocolMessage parseFrom(
         byte[] data,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaProtocolMessage parseFrom(java.io.InputStream input)
+    public static PekkoProtocolMessage parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.pekko.protobufv3.internal.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaProtocolMessage parseFrom(
+    public static PekkoProtocolMessage parseFrom(
         java.io.InputStream input,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.pekko.protobufv3.internal.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaProtocolMessage parseDelimitedFrom(java.io.InputStream input)
+    public static PekkoProtocolMessage parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.pekko.protobufv3.internal.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaProtocolMessage parseDelimitedFrom(
+    public static PekkoProtocolMessage parseDelimitedFrom(
         java.io.InputStream input,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.pekko.protobufv3.internal.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaProtocolMessage parseFrom(
+    public static PekkoProtocolMessage parseFrom(
         org.apache.pekko.protobufv3.internal.CodedInputStream input)
         throws java.io.IOException {
       return org.apache.pekko.protobufv3.internal.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaProtocolMessage parseFrom(
+    public static PekkoProtocolMessage parseFrom(
         org.apache.pekko.protobufv3.internal.CodedInputStream input,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -10019,7 +10019,7 @@ public final class WireFormats {
     public static Builder newBuilder() {
       return DEFAULT_INSTANCE.toBuilder();
     }
-    public static Builder newBuilder(org.apache.pekko.remote.WireFormats.AkkaProtocolMessage prototype) {
+    public static Builder newBuilder(PekkoProtocolMessage prototype) {
       return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
     }
     @java.lang.Override
@@ -10046,7 +10046,7 @@ public final class WireFormats {
     public static final class Builder extends
         org.apache.pekko.protobufv3.internal.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:AkkaProtocolMessage)
-        org.apache.pekko.remote.WireFormats.AkkaProtocolMessageOrBuilder {
+            PekkoProtocolMessageOrBuilder {
       public static final org.apache.pekko.protobufv3.internal.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.pekko.remote.WireFormats.internal_static_AkkaProtocolMessage_descriptor;
@@ -10057,7 +10057,7 @@ public final class WireFormats {
           internalGetFieldAccessorTable() {
         return org.apache.pekko.remote.WireFormats.internal_static_AkkaProtocolMessage_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.pekko.remote.WireFormats.AkkaProtocolMessage.class, org.apache.pekko.remote.WireFormats.AkkaProtocolMessage.Builder.class);
+                PekkoProtocolMessage.class, PekkoProtocolMessage.Builder.class);
       }
 
       // Construct using org.apache.pekko.remote.WireFormats.AkkaProtocolMessage.newBuilder()
@@ -10097,13 +10097,13 @@ public final class WireFormats {
       }
 
       @java.lang.Override
-      public org.apache.pekko.remote.WireFormats.AkkaProtocolMessage getDefaultInstanceForType() {
-        return org.apache.pekko.remote.WireFormats.AkkaProtocolMessage.getDefaultInstance();
+      public PekkoProtocolMessage getDefaultInstanceForType() {
+        return PekkoProtocolMessage.getDefaultInstance();
       }
 
       @java.lang.Override
-      public org.apache.pekko.remote.WireFormats.AkkaProtocolMessage build() {
-        org.apache.pekko.remote.WireFormats.AkkaProtocolMessage result = buildPartial();
+      public PekkoProtocolMessage build() {
+        PekkoProtocolMessage result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
@@ -10111,8 +10111,8 @@ public final class WireFormats {
       }
 
       @java.lang.Override
-      public org.apache.pekko.remote.WireFormats.AkkaProtocolMessage buildPartial() {
-        org.apache.pekko.remote.WireFormats.AkkaProtocolMessage result = new org.apache.pekko.remote.WireFormats.AkkaProtocolMessage(this);
+      public PekkoProtocolMessage buildPartial() {
+        PekkoProtocolMessage result = new PekkoProtocolMessage(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) != 0)) {
@@ -10166,16 +10166,16 @@ public final class WireFormats {
       }
       @java.lang.Override
       public Builder mergeFrom(org.apache.pekko.protobufv3.internal.Message other) {
-        if (other instanceof org.apache.pekko.remote.WireFormats.AkkaProtocolMessage) {
-          return mergeFrom((org.apache.pekko.remote.WireFormats.AkkaProtocolMessage)other);
+        if (other instanceof PekkoProtocolMessage) {
+          return mergeFrom((PekkoProtocolMessage)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.pekko.remote.WireFormats.AkkaProtocolMessage other) {
-        if (other == org.apache.pekko.remote.WireFormats.AkkaProtocolMessage.getDefaultInstance()) return this;
+      public Builder mergeFrom(PekkoProtocolMessage other) {
+        if (other == PekkoProtocolMessage.getDefaultInstance()) return this;
         if (other.hasPayload()) {
           setPayload(other.getPayload());
         }
@@ -10202,11 +10202,11 @@ public final class WireFormats {
           org.apache.pekko.protobufv3.internal.CodedInputStream input,
           org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.pekko.remote.WireFormats.AkkaProtocolMessage parsedMessage = null;
+        PekkoProtocolMessage parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.pekko.remote.WireFormats.AkkaProtocolMessage) e.getUnfinishedMessage();
+          parsedMessage = (PekkoProtocolMessage) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
           if (parsedMessage != null) {
@@ -10257,9 +10257,9 @@ public final class WireFormats {
         return this;
       }
 
-      private org.apache.pekko.remote.WireFormats.AkkaControlMessage instruction_;
+      private PekkoControlMessage instruction_;
       private org.apache.pekko.protobufv3.internal.SingleFieldBuilderV3<
-          org.apache.pekko.remote.WireFormats.AkkaControlMessage, org.apache.pekko.remote.WireFormats.AkkaControlMessage.Builder, org.apache.pekko.remote.WireFormats.AkkaControlMessageOrBuilder> instructionBuilder_;
+              PekkoControlMessage, PekkoControlMessage.Builder, PekkoControlMessageOrBuilder> instructionBuilder_;
       /**
        * optional .AkkaControlMessage instruction = 2;
        * @return Whether the instruction field is set.
@@ -10271,9 +10271,9 @@ public final class WireFormats {
        * optional .AkkaControlMessage instruction = 2;
        * @return The instruction.
        */
-      public org.apache.pekko.remote.WireFormats.AkkaControlMessage getInstruction() {
+      public PekkoControlMessage getInstruction() {
         if (instructionBuilder_ == null) {
-          return instruction_ == null ? org.apache.pekko.remote.WireFormats.AkkaControlMessage.getDefaultInstance() : instruction_;
+          return instruction_ == null ? PekkoControlMessage.getDefaultInstance() : instruction_;
         } else {
           return instructionBuilder_.getMessage();
         }
@@ -10281,7 +10281,7 @@ public final class WireFormats {
       /**
        * optional .AkkaControlMessage instruction = 2;
        */
-      public Builder setInstruction(org.apache.pekko.remote.WireFormats.AkkaControlMessage value) {
+      public Builder setInstruction(PekkoControlMessage value) {
         if (instructionBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
@@ -10298,7 +10298,7 @@ public final class WireFormats {
        * optional .AkkaControlMessage instruction = 2;
        */
       public Builder setInstruction(
-          org.apache.pekko.remote.WireFormats.AkkaControlMessage.Builder builderForValue) {
+          PekkoControlMessage.Builder builderForValue) {
         if (instructionBuilder_ == null) {
           instruction_ = builderForValue.build();
           onChanged();
@@ -10311,13 +10311,13 @@ public final class WireFormats {
       /**
        * optional .AkkaControlMessage instruction = 2;
        */
-      public Builder mergeInstruction(org.apache.pekko.remote.WireFormats.AkkaControlMessage value) {
+      public Builder mergeInstruction(PekkoControlMessage value) {
         if (instructionBuilder_ == null) {
           if (((bitField0_ & 0x00000002) != 0) &&
               instruction_ != null &&
-              instruction_ != org.apache.pekko.remote.WireFormats.AkkaControlMessage.getDefaultInstance()) {
+              instruction_ != PekkoControlMessage.getDefaultInstance()) {
             instruction_ =
-              org.apache.pekko.remote.WireFormats.AkkaControlMessage.newBuilder(instruction_).mergeFrom(value).buildPartial();
+              PekkoControlMessage.newBuilder(instruction_).mergeFrom(value).buildPartial();
           } else {
             instruction_ = value;
           }
@@ -10344,7 +10344,7 @@ public final class WireFormats {
       /**
        * optional .AkkaControlMessage instruction = 2;
        */
-      public org.apache.pekko.remote.WireFormats.AkkaControlMessage.Builder getInstructionBuilder() {
+      public PekkoControlMessage.Builder getInstructionBuilder() {
         bitField0_ |= 0x00000002;
         onChanged();
         return getInstructionFieldBuilder().getBuilder();
@@ -10352,23 +10352,23 @@ public final class WireFormats {
       /**
        * optional .AkkaControlMessage instruction = 2;
        */
-      public org.apache.pekko.remote.WireFormats.AkkaControlMessageOrBuilder getInstructionOrBuilder() {
+      public PekkoControlMessageOrBuilder getInstructionOrBuilder() {
         if (instructionBuilder_ != null) {
           return instructionBuilder_.getMessageOrBuilder();
         } else {
           return instruction_ == null ?
-              org.apache.pekko.remote.WireFormats.AkkaControlMessage.getDefaultInstance() : instruction_;
+              PekkoControlMessage.getDefaultInstance() : instruction_;
         }
       }
       /**
        * optional .AkkaControlMessage instruction = 2;
        */
       private org.apache.pekko.protobufv3.internal.SingleFieldBuilderV3<
-          org.apache.pekko.remote.WireFormats.AkkaControlMessage, org.apache.pekko.remote.WireFormats.AkkaControlMessage.Builder, org.apache.pekko.remote.WireFormats.AkkaControlMessageOrBuilder> 
+              PekkoControlMessage, PekkoControlMessage.Builder, PekkoControlMessageOrBuilder>
           getInstructionFieldBuilder() {
         if (instructionBuilder_ == null) {
           instructionBuilder_ = new org.apache.pekko.protobufv3.internal.SingleFieldBuilderV3<
-              org.apache.pekko.remote.WireFormats.AkkaControlMessage, org.apache.pekko.remote.WireFormats.AkkaControlMessage.Builder, org.apache.pekko.remote.WireFormats.AkkaControlMessageOrBuilder>(
+                  PekkoControlMessage, PekkoControlMessage.Builder, PekkoControlMessageOrBuilder>(
                   getInstruction(),
                   getParentForChildren(),
                   isClean());
@@ -10393,43 +10393,43 @@ public final class WireFormats {
     }
 
     // @@protoc_insertion_point(class_scope:AkkaProtocolMessage)
-    private static final org.apache.pekko.remote.WireFormats.AkkaProtocolMessage DEFAULT_INSTANCE;
+    private static final PekkoProtocolMessage DEFAULT_INSTANCE;
     static {
-      DEFAULT_INSTANCE = new org.apache.pekko.remote.WireFormats.AkkaProtocolMessage();
+      DEFAULT_INSTANCE = new PekkoProtocolMessage();
     }
 
-    public static org.apache.pekko.remote.WireFormats.AkkaProtocolMessage getDefaultInstance() {
+    public static PekkoProtocolMessage getDefaultInstance() {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final org.apache.pekko.protobufv3.internal.Parser
-        PARSER = new org.apache.pekko.protobufv3.internal.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.pekko.protobufv3.internal.Parser
+        PARSER = new org.apache.pekko.protobufv3.internal.AbstractParser() {
       @java.lang.Override
-      public AkkaProtocolMessage parsePartialFrom(
+      public PekkoProtocolMessage parsePartialFrom(
           org.apache.pekko.protobufv3.internal.CodedInputStream input,
           org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
           throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
-        return new AkkaProtocolMessage(input, extensionRegistry);
+        return new PekkoProtocolMessage(input, extensionRegistry);
       }
     };
 
-    public static org.apache.pekko.protobufv3.internal.Parser parser() {
+    public static org.apache.pekko.protobufv3.internal.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public org.apache.pekko.protobufv3.internal.Parser getParserForType() {
+    public org.apache.pekko.protobufv3.internal.Parser getParserForType() {
       return PARSER;
     }
 
     @java.lang.Override
-    public org.apache.pekko.remote.WireFormats.AkkaProtocolMessage getDefaultInstanceForType() {
+    public PekkoProtocolMessage getDefaultInstanceForType() {
       return DEFAULT_INSTANCE;
     }
 
   }
 
-  public interface AkkaControlMessageOrBuilder extends
+  public interface PekkoControlMessageOrBuilder extends
       // @@protoc_insertion_point(interface_extends:AkkaControlMessage)
       org.apache.pekko.protobufv3.internal.MessageOrBuilder {
 
@@ -10453,11 +10453,11 @@ public final class WireFormats {
      * optional .AkkaHandshakeInfo handshakeInfo = 2;
      * @return The handshakeInfo.
      */
-    org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo getHandshakeInfo();
+    PekkoHandshakeInfo getHandshakeInfo();
     /**
      * optional .AkkaHandshakeInfo handshakeInfo = 2;
      */
-    org.apache.pekko.remote.WireFormats.AkkaHandshakeInfoOrBuilder getHandshakeInfoOrBuilder();
+    PekkoHandshakeInfoOrBuilder getHandshakeInfoOrBuilder();
   }
   /**
    * 
@@ -10467,16 +10467,16 @@ public final class WireFormats {
    *
    * Protobuf type {@code AkkaControlMessage}
    */
-  public  static final class AkkaControlMessage extends
+  public  static final class PekkoControlMessage extends
       org.apache.pekko.protobufv3.internal.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:AkkaControlMessage)
-      AkkaControlMessageOrBuilder {
+          PekkoControlMessageOrBuilder {
   private static final long serialVersionUID = 0L;
     // Use AkkaControlMessage.newBuilder() to construct.
-    private AkkaControlMessage(org.apache.pekko.protobufv3.internal.GeneratedMessageV3.Builder builder) {
+    private PekkoControlMessage(org.apache.pekko.protobufv3.internal.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
-    private AkkaControlMessage() {
+    private PekkoControlMessage() {
       commandType_ = 1;
     }
 
@@ -10484,7 +10484,7 @@ public final class WireFormats {
     @SuppressWarnings({"unused"})
     protected java.lang.Object newInstance(
         org.apache.pekko.protobufv3.internal.GeneratedMessageV3.UnusedPrivateParameter unused) {
-      return new AkkaControlMessage();
+      return new PekkoControlMessage();
     }
 
     @java.lang.Override
@@ -10492,7 +10492,7 @@ public final class WireFormats {
     getUnknownFields() {
       return this.unknownFields;
     }
-    private AkkaControlMessage(
+    private PekkoControlMessage(
         org.apache.pekko.protobufv3.internal.CodedInputStream input,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
@@ -10524,11 +10524,11 @@ public final class WireFormats {
               break;
             }
             case 18: {
-              org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.Builder subBuilder = null;
+              PekkoHandshakeInfo.Builder subBuilder = null;
               if (((bitField0_ & 0x00000002) != 0)) {
                 subBuilder = handshakeInfo_.toBuilder();
               }
-              handshakeInfo_ = input.readMessage(org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.PARSER, extensionRegistry);
+              handshakeInfo_ = input.readMessage(PekkoHandshakeInfo.PARSER, extensionRegistry);
               if (subBuilder != null) {
                 subBuilder.mergeFrom(handshakeInfo_);
                 handshakeInfo_ = subBuilder.buildPartial();
@@ -10565,7 +10565,7 @@ public final class WireFormats {
         internalGetFieldAccessorTable() {
       return org.apache.pekko.remote.WireFormats.internal_static_AkkaControlMessage_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.pekko.remote.WireFormats.AkkaControlMessage.class, org.apache.pekko.remote.WireFormats.AkkaControlMessage.Builder.class);
+              PekkoControlMessage.class, PekkoControlMessage.Builder.class);
     }
 
     private int bitField0_;
@@ -10589,7 +10589,7 @@ public final class WireFormats {
     }
 
     public static final int HANDSHAKEINFO_FIELD_NUMBER = 2;
-    private org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo handshakeInfo_;
+    private PekkoHandshakeInfo handshakeInfo_;
     /**
      * optional .AkkaHandshakeInfo handshakeInfo = 2;
      * @return Whether the handshakeInfo field is set.
@@ -10601,14 +10601,14 @@ public final class WireFormats {
      * optional .AkkaHandshakeInfo handshakeInfo = 2;
      * @return The handshakeInfo.
      */
-    public org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo getHandshakeInfo() {
-      return handshakeInfo_ == null ? org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.getDefaultInstance() : handshakeInfo_;
+    public PekkoHandshakeInfo getHandshakeInfo() {
+      return handshakeInfo_ == null ? PekkoHandshakeInfo.getDefaultInstance() : handshakeInfo_;
     }
     /**
      * optional .AkkaHandshakeInfo handshakeInfo = 2;
      */
-    public org.apache.pekko.remote.WireFormats.AkkaHandshakeInfoOrBuilder getHandshakeInfoOrBuilder() {
-      return handshakeInfo_ == null ? org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.getDefaultInstance() : handshakeInfo_;
+    public PekkoHandshakeInfoOrBuilder getHandshakeInfoOrBuilder() {
+      return handshakeInfo_ == null ? PekkoHandshakeInfo.getDefaultInstance() : handshakeInfo_;
     }
 
     private byte memoizedIsInitialized = -1;
@@ -10668,10 +10668,10 @@ public final class WireFormats {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.pekko.remote.WireFormats.AkkaControlMessage)) {
+      if (!(obj instanceof PekkoControlMessage)) {
         return super.equals(obj);
       }
-      org.apache.pekko.remote.WireFormats.AkkaControlMessage other = (org.apache.pekko.remote.WireFormats.AkkaControlMessage) obj;
+      PekkoControlMessage other = (PekkoControlMessage) obj;
 
       if (hasCommandType() != other.hasCommandType()) return false;
       if (hasCommandType()) {
@@ -10706,69 +10706,69 @@ public final class WireFormats {
       return hash;
     }
 
-    public static org.apache.pekko.remote.WireFormats.AkkaControlMessage parseFrom(
+    public static PekkoControlMessage parseFrom(
         java.nio.ByteBuffer data)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaControlMessage parseFrom(
+    public static PekkoControlMessage parseFrom(
         java.nio.ByteBuffer data,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaControlMessage parseFrom(
+    public static PekkoControlMessage parseFrom(
         org.apache.pekko.protobufv3.internal.ByteString data)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaControlMessage parseFrom(
+    public static PekkoControlMessage parseFrom(
         org.apache.pekko.protobufv3.internal.ByteString data,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaControlMessage parseFrom(byte[] data)
+    public static PekkoControlMessage parseFrom(byte[] data)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaControlMessage parseFrom(
+    public static PekkoControlMessage parseFrom(
         byte[] data,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaControlMessage parseFrom(java.io.InputStream input)
+    public static PekkoControlMessage parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.pekko.protobufv3.internal.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaControlMessage parseFrom(
+    public static PekkoControlMessage parseFrom(
         java.io.InputStream input,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.pekko.protobufv3.internal.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaControlMessage parseDelimitedFrom(java.io.InputStream input)
+    public static PekkoControlMessage parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.pekko.protobufv3.internal.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaControlMessage parseDelimitedFrom(
+    public static PekkoControlMessage parseDelimitedFrom(
         java.io.InputStream input,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.pekko.protobufv3.internal.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaControlMessage parseFrom(
+    public static PekkoControlMessage parseFrom(
         org.apache.pekko.protobufv3.internal.CodedInputStream input)
         throws java.io.IOException {
       return org.apache.pekko.protobufv3.internal.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaControlMessage parseFrom(
+    public static PekkoControlMessage parseFrom(
         org.apache.pekko.protobufv3.internal.CodedInputStream input,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -10781,7 +10781,7 @@ public final class WireFormats {
     public static Builder newBuilder() {
       return DEFAULT_INSTANCE.toBuilder();
     }
-    public static Builder newBuilder(org.apache.pekko.remote.WireFormats.AkkaControlMessage prototype) {
+    public static Builder newBuilder(PekkoControlMessage prototype) {
       return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
     }
     @java.lang.Override
@@ -10807,7 +10807,7 @@ public final class WireFormats {
     public static final class Builder extends
         org.apache.pekko.protobufv3.internal.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:AkkaControlMessage)
-        org.apache.pekko.remote.WireFormats.AkkaControlMessageOrBuilder {
+            PekkoControlMessageOrBuilder {
       public static final org.apache.pekko.protobufv3.internal.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.pekko.remote.WireFormats.internal_static_AkkaControlMessage_descriptor;
@@ -10818,7 +10818,7 @@ public final class WireFormats {
           internalGetFieldAccessorTable() {
         return org.apache.pekko.remote.WireFormats.internal_static_AkkaControlMessage_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.pekko.remote.WireFormats.AkkaControlMessage.class, org.apache.pekko.remote.WireFormats.AkkaControlMessage.Builder.class);
+                PekkoControlMessage.class, PekkoControlMessage.Builder.class);
       }
 
       // Construct using org.apache.pekko.remote.WireFormats.AkkaControlMessage.newBuilder()
@@ -10858,13 +10858,13 @@ public final class WireFormats {
       }
 
       @java.lang.Override
-      public org.apache.pekko.remote.WireFormats.AkkaControlMessage getDefaultInstanceForType() {
-        return org.apache.pekko.remote.WireFormats.AkkaControlMessage.getDefaultInstance();
+      public PekkoControlMessage getDefaultInstanceForType() {
+        return PekkoControlMessage.getDefaultInstance();
       }
 
       @java.lang.Override
-      public org.apache.pekko.remote.WireFormats.AkkaControlMessage build() {
-        org.apache.pekko.remote.WireFormats.AkkaControlMessage result = buildPartial();
+      public PekkoControlMessage build() {
+        PekkoControlMessage result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
@@ -10872,8 +10872,8 @@ public final class WireFormats {
       }
 
       @java.lang.Override
-      public org.apache.pekko.remote.WireFormats.AkkaControlMessage buildPartial() {
-        org.apache.pekko.remote.WireFormats.AkkaControlMessage result = new org.apache.pekko.remote.WireFormats.AkkaControlMessage(this);
+      public PekkoControlMessage buildPartial() {
+        PekkoControlMessage result = new PekkoControlMessage(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) != 0)) {
@@ -10927,16 +10927,16 @@ public final class WireFormats {
       }
       @java.lang.Override
       public Builder mergeFrom(org.apache.pekko.protobufv3.internal.Message other) {
-        if (other instanceof org.apache.pekko.remote.WireFormats.AkkaControlMessage) {
-          return mergeFrom((org.apache.pekko.remote.WireFormats.AkkaControlMessage)other);
+        if (other instanceof PekkoControlMessage) {
+          return mergeFrom((PekkoControlMessage)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.pekko.remote.WireFormats.AkkaControlMessage other) {
-        if (other == org.apache.pekko.remote.WireFormats.AkkaControlMessage.getDefaultInstance()) return this;
+      public Builder mergeFrom(PekkoControlMessage other) {
+        if (other == PekkoControlMessage.getDefaultInstance()) return this;
         if (other.hasCommandType()) {
           setCommandType(other.getCommandType());
         }
@@ -10966,11 +10966,11 @@ public final class WireFormats {
           org.apache.pekko.protobufv3.internal.CodedInputStream input,
           org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.pekko.remote.WireFormats.AkkaControlMessage parsedMessage = null;
+        PekkoControlMessage parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.pekko.remote.WireFormats.AkkaControlMessage) e.getUnfinishedMessage();
+          parsedMessage = (PekkoControlMessage) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
           if (parsedMessage != null) {
@@ -11023,9 +11023,9 @@ public final class WireFormats {
         return this;
       }
 
-      private org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo handshakeInfo_;
+      private PekkoHandshakeInfo handshakeInfo_;
       private org.apache.pekko.protobufv3.internal.SingleFieldBuilderV3<
-          org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo, org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.Builder, org.apache.pekko.remote.WireFormats.AkkaHandshakeInfoOrBuilder> handshakeInfoBuilder_;
+              PekkoHandshakeInfo, PekkoHandshakeInfo.Builder, PekkoHandshakeInfoOrBuilder> handshakeInfoBuilder_;
       /**
        * optional .AkkaHandshakeInfo handshakeInfo = 2;
        * @return Whether the handshakeInfo field is set.
@@ -11037,9 +11037,9 @@ public final class WireFormats {
        * optional .AkkaHandshakeInfo handshakeInfo = 2;
        * @return The handshakeInfo.
        */
-      public org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo getHandshakeInfo() {
+      public PekkoHandshakeInfo getHandshakeInfo() {
         if (handshakeInfoBuilder_ == null) {
-          return handshakeInfo_ == null ? org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.getDefaultInstance() : handshakeInfo_;
+          return handshakeInfo_ == null ? PekkoHandshakeInfo.getDefaultInstance() : handshakeInfo_;
         } else {
           return handshakeInfoBuilder_.getMessage();
         }
@@ -11047,7 +11047,7 @@ public final class WireFormats {
       /**
        * optional .AkkaHandshakeInfo handshakeInfo = 2;
        */
-      public Builder setHandshakeInfo(org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo value) {
+      public Builder setHandshakeInfo(PekkoHandshakeInfo value) {
         if (handshakeInfoBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
@@ -11064,7 +11064,7 @@ public final class WireFormats {
        * optional .AkkaHandshakeInfo handshakeInfo = 2;
        */
       public Builder setHandshakeInfo(
-          org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.Builder builderForValue) {
+          PekkoHandshakeInfo.Builder builderForValue) {
         if (handshakeInfoBuilder_ == null) {
           handshakeInfo_ = builderForValue.build();
           onChanged();
@@ -11077,13 +11077,13 @@ public final class WireFormats {
       /**
        * optional .AkkaHandshakeInfo handshakeInfo = 2;
        */
-      public Builder mergeHandshakeInfo(org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo value) {
+      public Builder mergeHandshakeInfo(PekkoHandshakeInfo value) {
         if (handshakeInfoBuilder_ == null) {
           if (((bitField0_ & 0x00000002) != 0) &&
               handshakeInfo_ != null &&
-              handshakeInfo_ != org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.getDefaultInstance()) {
+              handshakeInfo_ != PekkoHandshakeInfo.getDefaultInstance()) {
             handshakeInfo_ =
-              org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.newBuilder(handshakeInfo_).mergeFrom(value).buildPartial();
+              PekkoHandshakeInfo.newBuilder(handshakeInfo_).mergeFrom(value).buildPartial();
           } else {
             handshakeInfo_ = value;
           }
@@ -11110,7 +11110,7 @@ public final class WireFormats {
       /**
        * optional .AkkaHandshakeInfo handshakeInfo = 2;
        */
-      public org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.Builder getHandshakeInfoBuilder() {
+      public PekkoHandshakeInfo.Builder getHandshakeInfoBuilder() {
         bitField0_ |= 0x00000002;
         onChanged();
         return getHandshakeInfoFieldBuilder().getBuilder();
@@ -11118,23 +11118,23 @@ public final class WireFormats {
       /**
        * optional .AkkaHandshakeInfo handshakeInfo = 2;
        */
-      public org.apache.pekko.remote.WireFormats.AkkaHandshakeInfoOrBuilder getHandshakeInfoOrBuilder() {
+      public PekkoHandshakeInfoOrBuilder getHandshakeInfoOrBuilder() {
         if (handshakeInfoBuilder_ != null) {
           return handshakeInfoBuilder_.getMessageOrBuilder();
         } else {
           return handshakeInfo_ == null ?
-              org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.getDefaultInstance() : handshakeInfo_;
+              PekkoHandshakeInfo.getDefaultInstance() : handshakeInfo_;
         }
       }
       /**
        * optional .AkkaHandshakeInfo handshakeInfo = 2;
        */
       private org.apache.pekko.protobufv3.internal.SingleFieldBuilderV3<
-          org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo, org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.Builder, org.apache.pekko.remote.WireFormats.AkkaHandshakeInfoOrBuilder> 
+              PekkoHandshakeInfo, PekkoHandshakeInfo.Builder, PekkoHandshakeInfoOrBuilder>
           getHandshakeInfoFieldBuilder() {
         if (handshakeInfoBuilder_ == null) {
           handshakeInfoBuilder_ = new org.apache.pekko.protobufv3.internal.SingleFieldBuilderV3<
-              org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo, org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.Builder, org.apache.pekko.remote.WireFormats.AkkaHandshakeInfoOrBuilder>(
+                  PekkoHandshakeInfo, PekkoHandshakeInfo.Builder, PekkoHandshakeInfoOrBuilder>(
                   getHandshakeInfo(),
                   getParentForChildren(),
                   isClean());
@@ -11159,43 +11159,43 @@ public final class WireFormats {
     }
 
     // @@protoc_insertion_point(class_scope:AkkaControlMessage)
-    private static final org.apache.pekko.remote.WireFormats.AkkaControlMessage DEFAULT_INSTANCE;
+    private static final PekkoControlMessage DEFAULT_INSTANCE;
     static {
-      DEFAULT_INSTANCE = new org.apache.pekko.remote.WireFormats.AkkaControlMessage();
+      DEFAULT_INSTANCE = new PekkoControlMessage();
     }
 
-    public static org.apache.pekko.remote.WireFormats.AkkaControlMessage getDefaultInstance() {
+    public static PekkoControlMessage getDefaultInstance() {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final org.apache.pekko.protobufv3.internal.Parser
-        PARSER = new org.apache.pekko.protobufv3.internal.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.pekko.protobufv3.internal.Parser
+        PARSER = new org.apache.pekko.protobufv3.internal.AbstractParser() {
       @java.lang.Override
-      public AkkaControlMessage parsePartialFrom(
+      public PekkoControlMessage parsePartialFrom(
           org.apache.pekko.protobufv3.internal.CodedInputStream input,
           org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
           throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
-        return new AkkaControlMessage(input, extensionRegistry);
+        return new PekkoControlMessage(input, extensionRegistry);
       }
     };
 
-    public static org.apache.pekko.protobufv3.internal.Parser parser() {
+    public static org.apache.pekko.protobufv3.internal.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public org.apache.pekko.protobufv3.internal.Parser getParserForType() {
+    public org.apache.pekko.protobufv3.internal.Parser getParserForType() {
       return PARSER;
     }
 
     @java.lang.Override
-    public org.apache.pekko.remote.WireFormats.AkkaControlMessage getDefaultInstanceForType() {
+    public PekkoControlMessage getDefaultInstanceForType() {
       return DEFAULT_INSTANCE;
     }
 
   }
 
-  public interface AkkaHandshakeInfoOrBuilder extends
+  public interface PekkoHandshakeInfoOrBuilder extends
       // @@protoc_insertion_point(interface_extends:AkkaHandshakeInfo)
       org.apache.pekko.protobufv3.internal.MessageOrBuilder {
 
@@ -11245,16 +11245,16 @@ public final class WireFormats {
   /**
    * Protobuf type {@code AkkaHandshakeInfo}
    */
-  public  static final class AkkaHandshakeInfo extends
+  public  static final class PekkoHandshakeInfo extends
       org.apache.pekko.protobufv3.internal.GeneratedMessageV3 implements
       // @@protoc_insertion_point(message_implements:AkkaHandshakeInfo)
-      AkkaHandshakeInfoOrBuilder {
+          PekkoHandshakeInfoOrBuilder {
   private static final long serialVersionUID = 0L;
     // Use AkkaHandshakeInfo.newBuilder() to construct.
-    private AkkaHandshakeInfo(org.apache.pekko.protobufv3.internal.GeneratedMessageV3.Builder builder) {
+    private PekkoHandshakeInfo(org.apache.pekko.protobufv3.internal.GeneratedMessageV3.Builder builder) {
       super(builder);
     }
-    private AkkaHandshakeInfo() {
+    private PekkoHandshakeInfo() {
       cookie_ = "";
     }
 
@@ -11262,7 +11262,7 @@ public final class WireFormats {
     @SuppressWarnings({"unused"})
     protected java.lang.Object newInstance(
         org.apache.pekko.protobufv3.internal.GeneratedMessageV3.UnusedPrivateParameter unused) {
-      return new AkkaHandshakeInfo();
+      return new PekkoHandshakeInfo();
     }
 
     @java.lang.Override
@@ -11270,7 +11270,7 @@ public final class WireFormats {
     getUnknownFields() {
       return this.unknownFields;
     }
-    private AkkaHandshakeInfo(
+    private PekkoHandshakeInfo(
         org.apache.pekko.protobufv3.internal.CodedInputStream input,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
@@ -11342,7 +11342,7 @@ public final class WireFormats {
         internalGetFieldAccessorTable() {
       return org.apache.pekko.remote.WireFormats.internal_static_AkkaHandshakeInfo_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.class, org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.Builder.class);
+              PekkoHandshakeInfo.class, PekkoHandshakeInfo.Builder.class);
     }
 
     private int bitField0_;
@@ -11496,10 +11496,10 @@ public final class WireFormats {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo)) {
+      if (!(obj instanceof PekkoHandshakeInfo)) {
         return super.equals(obj);
       }
-      org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo other = (org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo) obj;
+      PekkoHandshakeInfo other = (PekkoHandshakeInfo) obj;
 
       if (hasOrigin() != other.hasOrigin()) return false;
       if (hasOrigin()) {
@@ -11545,69 +11545,69 @@ public final class WireFormats {
       return hash;
     }
 
-    public static org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo parseFrom(
+    public static PekkoHandshakeInfo parseFrom(
         java.nio.ByteBuffer data)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo parseFrom(
+    public static PekkoHandshakeInfo parseFrom(
         java.nio.ByteBuffer data,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo parseFrom(
+    public static PekkoHandshakeInfo parseFrom(
         org.apache.pekko.protobufv3.internal.ByteString data)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo parseFrom(
+    public static PekkoHandshakeInfo parseFrom(
         org.apache.pekko.protobufv3.internal.ByteString data,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo parseFrom(byte[] data)
+    public static PekkoHandshakeInfo parseFrom(byte[] data)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo parseFrom(
+    public static PekkoHandshakeInfo parseFrom(
         byte[] data,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo parseFrom(java.io.InputStream input)
+    public static PekkoHandshakeInfo parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.pekko.protobufv3.internal.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo parseFrom(
+    public static PekkoHandshakeInfo parseFrom(
         java.io.InputStream input,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.pekko.protobufv3.internal.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo parseDelimitedFrom(java.io.InputStream input)
+    public static PekkoHandshakeInfo parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.pekko.protobufv3.internal.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo parseDelimitedFrom(
+    public static PekkoHandshakeInfo parseDelimitedFrom(
         java.io.InputStream input,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.pekko.protobufv3.internal.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo parseFrom(
+    public static PekkoHandshakeInfo parseFrom(
         org.apache.pekko.protobufv3.internal.CodedInputStream input)
         throws java.io.IOException {
       return org.apache.pekko.protobufv3.internal.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo parseFrom(
+    public static PekkoHandshakeInfo parseFrom(
         org.apache.pekko.protobufv3.internal.CodedInputStream input,
         org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -11620,7 +11620,7 @@ public final class WireFormats {
     public static Builder newBuilder() {
       return DEFAULT_INSTANCE.toBuilder();
     }
-    public static Builder newBuilder(org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo prototype) {
+    public static Builder newBuilder(PekkoHandshakeInfo prototype) {
       return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
     }
     @java.lang.Override
@@ -11641,7 +11641,7 @@ public final class WireFormats {
     public static final class Builder extends
         org.apache.pekko.protobufv3.internal.GeneratedMessageV3.Builder implements
         // @@protoc_insertion_point(builder_implements:AkkaHandshakeInfo)
-        org.apache.pekko.remote.WireFormats.AkkaHandshakeInfoOrBuilder {
+            PekkoHandshakeInfoOrBuilder {
       public static final org.apache.pekko.protobufv3.internal.Descriptors.Descriptor
           getDescriptor() {
         return org.apache.pekko.remote.WireFormats.internal_static_AkkaHandshakeInfo_descriptor;
@@ -11652,7 +11652,7 @@ public final class WireFormats {
           internalGetFieldAccessorTable() {
         return org.apache.pekko.remote.WireFormats.internal_static_AkkaHandshakeInfo_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.class, org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.Builder.class);
+                PekkoHandshakeInfo.class, PekkoHandshakeInfo.Builder.class);
       }
 
       // Construct using org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.newBuilder()
@@ -11694,13 +11694,13 @@ public final class WireFormats {
       }
 
       @java.lang.Override
-      public org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo getDefaultInstanceForType() {
-        return org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.getDefaultInstance();
+      public PekkoHandshakeInfo getDefaultInstanceForType() {
+        return PekkoHandshakeInfo.getDefaultInstance();
       }
 
       @java.lang.Override
-      public org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo build() {
-        org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo result = buildPartial();
+      public PekkoHandshakeInfo build() {
+        PekkoHandshakeInfo result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
@@ -11708,8 +11708,8 @@ public final class WireFormats {
       }
 
       @java.lang.Override
-      public org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo buildPartial() {
-        org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo result = new org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo(this);
+      public PekkoHandshakeInfo buildPartial() {
+        PekkoHandshakeInfo result = new PekkoHandshakeInfo(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) != 0)) {
@@ -11767,16 +11767,16 @@ public final class WireFormats {
       }
       @java.lang.Override
       public Builder mergeFrom(org.apache.pekko.protobufv3.internal.Message other) {
-        if (other instanceof org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo) {
-          return mergeFrom((org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo)other);
+        if (other instanceof PekkoHandshakeInfo) {
+          return mergeFrom((PekkoHandshakeInfo)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo other) {
-        if (other == org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo.getDefaultInstance()) return this;
+      public Builder mergeFrom(PekkoHandshakeInfo other) {
+        if (other == PekkoHandshakeInfo.getDefaultInstance()) return this;
         if (other.hasOrigin()) {
           mergeOrigin(other.getOrigin());
         }
@@ -11812,11 +11812,11 @@ public final class WireFormats {
           org.apache.pekko.protobufv3.internal.CodedInputStream input,
           org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo parsedMessage = null;
+        PekkoHandshakeInfo parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo) e.getUnfinishedMessage();
+          parsedMessage = (PekkoHandshakeInfo) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
           if (parsedMessage != null) {
@@ -12084,37 +12084,37 @@ public final class WireFormats {
     }
 
     // @@protoc_insertion_point(class_scope:AkkaHandshakeInfo)
-    private static final org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo DEFAULT_INSTANCE;
+    private static final PekkoHandshakeInfo DEFAULT_INSTANCE;
     static {
-      DEFAULT_INSTANCE = new org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo();
+      DEFAULT_INSTANCE = new PekkoHandshakeInfo();
     }
 
-    public static org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo getDefaultInstance() {
+    public static PekkoHandshakeInfo getDefaultInstance() {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final org.apache.pekko.protobufv3.internal.Parser
-        PARSER = new org.apache.pekko.protobufv3.internal.AbstractParser() {
+    @java.lang.Deprecated public static final org.apache.pekko.protobufv3.internal.Parser
+        PARSER = new org.apache.pekko.protobufv3.internal.AbstractParser() {
       @java.lang.Override
-      public AkkaHandshakeInfo parsePartialFrom(
+      public PekkoHandshakeInfo parsePartialFrom(
           org.apache.pekko.protobufv3.internal.CodedInputStream input,
           org.apache.pekko.protobufv3.internal.ExtensionRegistryLite extensionRegistry)
           throws org.apache.pekko.protobufv3.internal.InvalidProtocolBufferException {
-        return new AkkaHandshakeInfo(input, extensionRegistry);
+        return new PekkoHandshakeInfo(input, extensionRegistry);
       }
     };
 
-    public static org.apache.pekko.protobufv3.internal.Parser parser() {
+    public static org.apache.pekko.protobufv3.internal.Parser parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public org.apache.pekko.protobufv3.internal.Parser getParserForType() {
+    public org.apache.pekko.protobufv3.internal.Parser getParserForType() {
       return PARSER;
     }
 
     @java.lang.Override
-    public org.apache.pekko.remote.WireFormats.AkkaHandshakeInfo getDefaultInstanceForType() {
+    public PekkoHandshakeInfo getDefaultInstanceForType() {
       return DEFAULT_INSTANCE;
     }
 
@@ -20577,42 +20577,42 @@ public final class WireFormats {
       "(\005\022\025\n\rscopeManifest\030\007 \001(\t\022\032\n\022configSeria" +
       "lizerId\030\010 \001(\005\022\026\n\016configManifest\030\t \001(\t\022 \n" +
       "\030routerConfigSerializerId\030\n \001(\005\022\034\n\024route" +
-      "rConfigManifest\030\013 \001(\t\022\014\n\004tags\030\014 \003(\t\"P\n\023A" +
-      "kkaProtocolMessage\022\017\n\007payload\030\001 \001(\014\022(\n\013i" +
-      "nstruction\030\002 \001(\0132\023.AkkaControlMessage\"b\n" +
-      "\022AkkaControlMessage\022!\n\013commandType\030\001 \002(\016" +
-      "2\014.CommandType\022)\n\rhandshakeInfo\030\002 \001(\0132\022." +
-      "AkkaHandshakeInfo\"N\n\021AkkaHandshakeInfo\022\034" +
-      "\n\006origin\030\001 \002(\0132\014.AddressData\022\013\n\003uid\030\002 \002(" +
-      "\006\022\016\n\006cookie\030\003 \001(\t\"8\n\016FiniteDuration\022\r\n\005v" +
-      "alue\030\001 \002(\003\022\027\n\004unit\030\002 \002(\0162\t.TimeUnit\")\n\013R" +
-      "emoteScope\022\032\n\004node\030\001 \002(\0132\014.AddressData\"\261" +
-      "\001\n\016DefaultResizer\022\022\n\nlowerBound\030\001 \002(\r\022\022\n" +
-      "\nupperBound\030\002 \002(\r\022\031\n\021pressureThreshold\030\003" +
-      " \002(\r\022\022\n\nrampupRate\030\004 \002(\001\022\030\n\020backoffThres" +
-      "hold\030\005 \002(\001\022\023\n\013backoffRate\030\006 \002(\001\022\031\n\021messa" +
-      "gesPerResize\030\007 \002(\r\"A\n\nFromConfig\022\031\n\007resi" +
-      "zer\030\001 \001(\0132\010.Payload\022\030\n\020routerDispatcher\030" +
-      "\002 \001(\t\"{\n\022GenericRoutingPool\022\025\n\rnrOfInsta" +
-      "nces\030\001 \002(\r\022\030\n\020routerDispatcher\030\002 \001(\t\022\031\n\021" +
-      "usePoolDispatcher\030\003 \002(\010\022\031\n\007resizer\030\004 \001(\013" +
-      "2\010.Payload\"Z\n\021ScatterGatherPool\022$\n\007gener" +
-      "ic\030\001 \002(\0132\023.GenericRoutingPool\022\037\n\006within\030" +
-      "\002 \002(\0132\017.FiniteDuration\"|\n\020TailChoppingPo" +
-      "ol\022$\n\007generic\030\001 \002(\0132\023.GenericRoutingPool" +
-      "\022\037\n\006within\030\002 \002(\0132\017.FiniteDuration\022!\n\010int" +
-      "erval\030\003 \002(\0132\017.FiniteDuration\"O\n\013AddressD" +
-      "ata\022\016\n\006system\030\001 \002(\t\022\020\n\010hostname\030\002 \002(\t\022\014\n" +
-      "\004port\030\003 \002(\r\022\020\n\010protocol\030\004 \001(\t\"J\n\022RemoteR" +
-      "outerConfig\022\027\n\005local\030\001 \002(\0132\010.Payload\022\033\n\005" +
-      "nodes\030\002 \003(\0132\014.AddressData*{\n\013CommandType" +
-      "\022\r\n\tASSOCIATE\020\001\022\020\n\014DISASSOCIATE\020\002\022\r\n\tHEA" +
-      "RTBEAT\020\003\022\036\n\032DISASSOCIATE_SHUTTING_DOWN\020\004" +
-      "\022\034\n\030DISASSOCIATE_QUARANTINED\020\005*n\n\010TimeUn" +
-      "it\022\017\n\013NANOSECONDS\020\001\022\020\n\014MICROSECONDS\020\002\022\020\n" +
-      "\014MILLISECONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MINUTES\020" +
-      "\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007B\033\n\027org.apache.pek" +
-      "ko.remoteH\001"
+      "rConfigManifest\030\013 \001(\t\022\014\n\004tags\030\014 \003(\t\"R\n\024P" +
+      "ekkoProtocolMessage\022\017\n\007payload\030\001 \001(\014\022)\n\013" +
+      "instruction\030\002 \001(\0132\024.PekkoControlMessage\"" +
+      "d\n\023PekkoControlMessage\022!\n\013commandType\030\001 " +
+      "\002(\0162\014.CommandType\022*\n\rhandshakeInfo\030\002 \001(\013" +
+      "2\023.PekkoHandshakeInfo\"O\n\022PekkoHandshakeI" +
+      "nfo\022\034\n\006origin\030\001 \002(\0132\014.AddressData\022\013\n\003uid" +
+      "\030\002 \002(\006\022\016\n\006cookie\030\003 \001(\t\"8\n\016FiniteDuration" +
+      "\022\r\n\005value\030\001 \002(\003\022\027\n\004unit\030\002 \002(\0162\t.TimeUnit" +
+      "\")\n\013RemoteScope\022\032\n\004node\030\001 \002(\0132\014.AddressD" +
+      "ata\"\261\001\n\016DefaultResizer\022\022\n\nlowerBound\030\001 \002" +
+      "(\r\022\022\n\nupperBound\030\002 \002(\r\022\031\n\021pressureThresh" +
+      "old\030\003 \002(\r\022\022\n\nrampupRate\030\004 \002(\001\022\030\n\020backoff" +
+      "Threshold\030\005 \002(\001\022\023\n\013backoffRate\030\006 \002(\001\022\031\n\021" +
+      "messagesPerResize\030\007 \002(\r\"A\n\nFromConfig\022\031\n" +
+      "\007resizer\030\001 \001(\0132\010.Payload\022\030\n\020routerDispat" +
+      "cher\030\002 \001(\t\"{\n\022GenericRoutingPool\022\025\n\rnrOf" +
+      "Instances\030\001 \002(\r\022\030\n\020routerDispatcher\030\002 \001(" +
+      "\t\022\031\n\021usePoolDispatcher\030\003 \002(\010\022\031\n\007resizer\030" +
+      "\004 \001(\0132\010.Payload\"Z\n\021ScatterGatherPool\022$\n\007" +
+      "generic\030\001 \002(\0132\023.GenericRoutingPool\022\037\n\006wi" +
+      "thin\030\002 \002(\0132\017.FiniteDuration\"|\n\020TailChopp" +
+      "ingPool\022$\n\007generic\030\001 \002(\0132\023.GenericRoutin" +
+      "gPool\022\037\n\006within\030\002 \002(\0132\017.FiniteDuration\022!" +
+      "\n\010interval\030\003 \002(\0132\017.FiniteDuration\"O\n\013Add" +
+      "ressData\022\016\n\006system\030\001 \002(\t\022\020\n\010hostname\030\002 \002" +
+      "(\t\022\014\n\004port\030\003 \002(\r\022\020\n\010protocol\030\004 \001(\t\"J\n\022Re" +
+      "moteRouterConfig\022\027\n\005local\030\001 \002(\0132\010.Payloa" +
+      "d\022\033\n\005nodes\030\002 \003(\0132\014.AddressData*{\n\013Comman" +
+      "dType\022\r\n\tASSOCIATE\020\001\022\020\n\014DISASSOCIATE\020\002\022\r" +
+      "\n\tHEARTBEAT\020\003\022\036\n\032DISASSOCIATE_SHUTTING_D" +
+      "OWN\020\004\022\034\n\030DISASSOCIATE_QUARANTINED\020\005*n\n\010T" +
+      "imeUnit\022\017\n\013NANOSECONDS\020\001\022\020\n\014MICROSECONDS" +
+      "\020\002\022\020\n\014MILLISECONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MIN" +
+      "UTES\020\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007B\033\n\027org.apach" +
+      "e.pekko.remoteH\001"
     };
     descriptor = org.apache.pekko.protobufv3.internal.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
diff --git a/akka-remote/src/main/protobuf/WireFormats.proto b/akka-remote/src/main/protobuf/WireFormats.proto
index 73a0077c7a..6560b54b4d 100644
--- a/akka-remote/src/main/protobuf/WireFormats.proto
+++ b/akka-remote/src/main/protobuf/WireFormats.proto
@@ -108,20 +108,20 @@ message DeployData {
  * Message format of Akka Protocol.
  * Message contains either a payload or an instruction.
  */
-message AkkaProtocolMessage {
+message PekkoProtocolMessage {
   optional bytes payload = 1;
-  optional AkkaControlMessage instruction = 2;
+  optional PekkoControlMessage instruction = 2;
 }
 
 /**
  * Defines some control messages for the remoting
  */
-message AkkaControlMessage {
+message PekkoControlMessage {
   required CommandType commandType = 1;
-  optional AkkaHandshakeInfo handshakeInfo = 2;
+  optional PekkoHandshakeInfo handshakeInfo = 2;
 }
 
-message AkkaHandshakeInfo {
+message PekkoHandshakeInfo {
   required AddressData origin = 1;
   required fixed64 uid = 2;
   optional string cookie = 3;
@@ -129,7 +129,7 @@ message AkkaHandshakeInfo {
 }
 
 /**
- * Defines the type of the AkkaControlMessage command type
+ * Defines the type of the PekkoControlMessage command type
  */
 enum CommandType {
   ASSOCIATE = 1;
diff --git a/akka-remote/src/main/scala/org/apache/pekko/remote/AckedDelivery.scala b/akka-remote/src/main/scala/org/apache/pekko/remote/AckedDelivery.scala
index 78222921ce..2f69be6770 100644
--- a/akka-remote/src/main/scala/org/apache/pekko/remote/AckedDelivery.scala
+++ b/akka-remote/src/main/scala/org/apache/pekko/remote/AckedDelivery.scala
@@ -7,7 +7,7 @@ package org.apache.pekko.remote
 import scala.collection.immutable._
 
 import org.apache.pekko
-import pekko.AkkaException
+import pekko.PekkoException
 
 @deprecated("Classic remoting is deprecated, use Artery", "2.6.0")
 object SeqNo {
@@ -79,11 +79,11 @@ final case class Ack(cumulativeAck: SeqNo, nacks: Set[SeqNo] = Set.empty) {
 
 @deprecated("Classic remoting is deprecated, use Artery", "2.6.0")
 class ResendBufferCapacityReachedException(c: Int)
-    extends AkkaException(s"Resend buffer capacity of [$c] has been reached.")
+    extends PekkoException(s"Resend buffer capacity of [$c] has been reached.")
 
 @deprecated("Classic remoting is deprecated, use Artery", "2.6.0")
 class ResendUnfulfillableException
-    extends AkkaException(
+    extends PekkoException(
       "Unable to fulfill resend request since negatively acknowledged payload is no longer in buffer. " +
       "The resend states between two systems are compromised and cannot be recovered.")
 
diff --git a/akka-remote/src/main/scala/org/apache/pekko/remote/Endpoint.scala b/akka-remote/src/main/scala/org/apache/pekko/remote/Endpoint.scala
index f34545a77d..4c615a03d0 100644
--- a/akka-remote/src/main/scala/org/apache/pekko/remote/Endpoint.scala
+++ b/akka-remote/src/main/scala/org/apache/pekko/remote/Endpoint.scala
@@ -17,7 +17,7 @@ import scala.util.control.NonFatal
 import scala.annotation.nowarn
 
 import org.apache.pekko
-import pekko.{ AkkaException, OnlyCauseStackTrace }
+import pekko.{ OnlyCauseStackTrace, PekkoException }
 import pekko.actor._
 import pekko.actor.OneForOneStrategy
 import pekko.actor.SupervisorStrategy._
@@ -29,7 +29,7 @@ import pekko.remote.EndpointManager.{ Link, ResendState, Send }
 import pekko.remote.EndpointWriter.{ FlushAndStop, StoppedReading }
 import pekko.remote.WireFormats.SerializedMessage
 import pekko.remote.transport._
-import pekko.remote.transport.AkkaPduCodec.Message
+import pekko.remote.transport.PekkoPduCodec.Message
 import pekko.remote.transport.AssociationHandle.{
   ActorHandleEventListener,
   DisassociateInfo,
@@ -150,7 +150,7 @@ private[remote] class DefaultMessageDispatcher(
  */
 @SerialVersionUID(1L)
 private[remote] class EndpointException(msg: String, cause: Throwable)
-    extends AkkaException(msg, cause)
+    extends PekkoException(msg, cause)
     with OnlyCauseStackTrace {
   def this(msg: String) = this(msg, null)
 }
@@ -223,13 +223,13 @@ private[remote] object ReliableDeliverySupervisor {
   case object TooLongIdle
 
   def props(
-      handleOrActive: Option[AkkaProtocolHandle],
+      handleOrActive: Option[PekkoProtocolHandle],
       localAddress: Address,
       remoteAddress: Address,
       refuseUid: Option[Int],
-      transport: AkkaProtocolTransport,
+      transport: PekkoProtocolTransport,
       settings: RemoteSettings,
-      codec: AkkaPduCodec,
+      codec: PekkoPduCodec,
       receiveBuffers: ConcurrentHashMap[Link, ResendState]): Props =
     Props(
       classOf[ReliableDeliverySupervisor],
@@ -248,13 +248,13 @@ private[remote] object ReliableDeliverySupervisor {
  */
 @nowarn("msg=deprecated")
 private[remote] class ReliableDeliverySupervisor(
-    handleOrActive: Option[AkkaProtocolHandle],
+    handleOrActive: Option[PekkoProtocolHandle],
     val localAddress: Address,
     val remoteAddress: Address,
     val refuseUid: Option[Int],
-    val transport: AkkaProtocolTransport,
+    val transport: PekkoProtocolTransport,
     val settings: RemoteSettings,
-    val codec: AkkaPduCodec,
+    val codec: PekkoPduCodec,
     val receiveBuffers: ConcurrentHashMap[Link, ResendState])
     extends Actor
     with ActorLogging {
@@ -286,7 +286,7 @@ private[remote] class ReliableDeliverySupervisor(
       Stop
   }
 
-  var currentHandle: Option[AkkaProtocolHandle] = handleOrActive
+  var currentHandle: Option[PekkoProtocolHandle] = handleOrActive
 
   var resendBuffer: AckedSendBuffer[Send] = _
   var seqCounter: Long = _
@@ -526,7 +526,7 @@ private[remote] class ReliableDeliverySupervisor(
             refuseUid,
             transport = transport,
             settings = settings,
-            AkkaPduProtobufCodec,
+            PekkoPduProtobufCodec$,
             receiveBuffers = receiveBuffers,
             reliableDeliverySupervisor = Some(self)))
           .withDeploy(Deploy.local),
@@ -543,7 +543,7 @@ private[remote] abstract class EndpointActor(
     val remoteAddress: Address,
     val transport: Transport,
     val settings: RemoteSettings,
-    val codec: AkkaPduCodec)
+    val codec: PekkoPduCodec)
     extends Actor
     with ActorLogging {
 
@@ -568,13 +568,13 @@ private[remote] abstract class EndpointActor(
 private[remote] object EndpointWriter {
 
   def props(
-      handleOrActive: Option[AkkaProtocolHandle],
+      handleOrActive: Option[PekkoProtocolHandle],
       localAddress: Address,
       remoteAddress: Address,
       refuseUid: Option[Int],
-      transport: AkkaProtocolTransport,
+      transport: PekkoProtocolTransport,
       settings: RemoteSettings,
-      codec: AkkaPduCodec,
+      codec: PekkoPduCodec,
       receiveBuffers: ConcurrentHashMap[Link, ResendState],
       reliableDeliverySupervisor: Option[ActorRef]): Props =
     Props(
@@ -596,8 +596,8 @@ private[remote] object EndpointWriter {
    * used instead.
    * @param handle Handle of the new inbound association.
    */
-  final case class TakeOver(handle: AkkaProtocolHandle, replyTo: ActorRef) extends NoSerializationVerificationNeeded
-  final case class TookOver(writer: ActorRef, handle: AkkaProtocolHandle) extends NoSerializationVerificationNeeded
+  final case class TakeOver(handle: PekkoProtocolHandle, replyTo: ActorRef) extends NoSerializationVerificationNeeded
+  final case class TookOver(writer: ActorRef, handle: PekkoProtocolHandle) extends NoSerializationVerificationNeeded
   case object BackoffTimer
   case object FlushAndStop
   private case object FlushAndStopTimeout
@@ -605,7 +605,7 @@ private[remote] object EndpointWriter {
   final case class StopReading(writer: ActorRef, replyTo: ActorRef)
   final case class StoppedReading(writer: ActorRef)
 
-  final case class Handle(handle: AkkaProtocolHandle) extends NoSerializationVerificationNeeded
+  final case class Handle(handle: PekkoProtocolHandle) extends NoSerializationVerificationNeeded
 
   final case class OutboundAck(ack: Ack)
 
@@ -623,13 +623,13 @@ private[remote] object EndpointWriter {
  */
 @nowarn("msg=deprecated")
 private[remote] class EndpointWriter(
-    handleOrActive: Option[AkkaProtocolHandle],
+    handleOrActive: Option[PekkoProtocolHandle],
     localAddress: Address,
     remoteAddress: Address,
     refuseUid: Option[Int],
-    transport: AkkaProtocolTransport,
+    transport: PekkoProtocolTransport,
     settings: RemoteSettings,
-    codec: AkkaPduCodec,
+    codec: PekkoPduCodec,
     val receiveBuffers: ConcurrentHashMap[Link, ResendState],
     val reliableDeliverySupervisor: Option[ActorRef])
     extends EndpointActor(localAddress, remoteAddress, transport, settings, codec) {
@@ -643,7 +643,7 @@ private[remote] class EndpointWriter(
   val backoffDispatcher = context.system.dispatchers.lookup("pekko.remote.classic.backoff-remote-dispatcher")
 
   var reader: Option[ActorRef] = None
-  var handle: Option[AkkaProtocolHandle] = handleOrActive
+  var handle: Option[PekkoProtocolHandle] = handleOrActive
   val readerId = Iterator.from(0)
 
   def newAckDeadline: Deadline = Deadline.now + settings.SysMsgAckTimeout
@@ -1007,7 +1007,7 @@ private[remote] class EndpointWriter(
       }
     }
 
-  private def startReadEndpoint(handle: AkkaProtocolHandle): Some[ActorRef] = {
+  private def startReadEndpoint(handle: PekkoProtocolHandle): Some[ActorRef] = {
     val newReader =
       context.watch(
         context.actorOf(
@@ -1052,7 +1052,7 @@ private[remote] object EndpointReader {
       remoteAddress: Address,
       transport: Transport,
       settings: RemoteSettings,
-      codec: AkkaPduCodec,
+      codec: PekkoPduCodec,
       msgDispatch: InboundMessageDispatcher,
       inbound: Boolean,
       uid: Int,
@@ -1082,7 +1082,7 @@ private[remote] class EndpointReader(
     remoteAddress: Address,
     transport: Transport,
     settings: RemoteSettings,
-    codec: AkkaPduCodec,
+    codec: PekkoPduCodec,
     msgDispatch: InboundMessageDispatcher,
     val inbound: Boolean,
     val uid: Int,
@@ -1165,7 +1165,7 @@ private[remote] class EndpointReader(
 
   }
 
-  private def logTransientSerializationError(msg: AkkaPduCodec.Message, error: Exception): Unit = {
+  private def logTransientSerializationError(msg: PekkoPduCodec.Message, error: Exception): Unit = {
     val sm = msg.serializedMessage
     log.warning(
       "Serializer not defined for message with serializer id [{}] and manifest [{}]. " +
diff --git a/akka-remote/src/main/scala/org/apache/pekko/remote/RemoteTransport.scala b/akka-remote/src/main/scala/org/apache/pekko/remote/RemoteTransport.scala
index a50398ab03..0cc8fd41aa 100644
--- a/akka-remote/src/main/scala/org/apache/pekko/remote/RemoteTransport.scala
+++ b/akka-remote/src/main/scala/org/apache/pekko/remote/RemoteTransport.scala
@@ -9,7 +9,7 @@ import scala.concurrent.Future
 import scala.util.control.NoStackTrace
 
 import org.apache.pekko
-import pekko.AkkaException
+import pekko.PekkoException
 import pekko.Done
 import pekko.actor._
 import pekko.annotation.InternalStableApi
@@ -21,7 +21,7 @@ import pekko.util.{ unused, OptionVal }
  * such as inability to start, wrong configuration etc.
  */
 @SerialVersionUID(1L)
-class RemoteTransportException(message: String, cause: Throwable) extends AkkaException(message, cause) {
+class RemoteTransportException(message: String, cause: Throwable) extends PekkoException(message, cause) {
   def this(msg: String) = this(msg, null)
 }
 
diff --git a/akka-remote/src/main/scala/org/apache/pekko/remote/Remoting.scala b/akka-remote/src/main/scala/org/apache/pekko/remote/Remoting.scala
index 90e37d3a58..5734ab7f02 100644
--- a/akka-remote/src/main/scala/org/apache/pekko/remote/Remoting.scala
+++ b/akka-remote/src/main/scala/org/apache/pekko/remote/Remoting.scala
@@ -31,7 +31,7 @@ import pekko.pattern.{ ask, gracefulStop, pipe }
 import pekko.remote.EndpointManager._
 import pekko.remote.Remoting.TransportSupervisor
 import pekko.remote.transport._
-import pekko.remote.transport.AkkaPduCodec.Message
+import pekko.remote.transport.PekkoPduCodec.Message
 import pekko.remote.transport.Transport.{ ActorAssociationEventListener, AssociationEventListener, InboundAssociation }
 import pekko.util.ByteString.UTF_8
 import pekko.util.OptionVal
@@ -85,7 +85,7 @@ private[remote] object Remoting {
   final val EndpointManagerName = "endpointManager"
 
   def localAddressForRemote(
-      transportMapping: Map[String, Set[(AkkaProtocolTransport, Address)]],
+      transportMapping: Map[String, Set[(PekkoProtocolTransport, Address)]],
       remote: Address): Address = {
 
     transportMapping.get(remote.protocol) match {
@@ -141,7 +141,7 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc
     extends RemoteTransport(_system, _provider) {
 
   @volatile private var endpointManager: Option[ActorRef] = None
-  @volatile private var transportMapping: Map[String, Set[(AkkaProtocolTransport, Address)]] = _
+  @volatile private var transportMapping: Map[String, Set[(PekkoProtocolTransport, Address)]] = _
   // This is effectively a write-once variable similar to a lazy val. The reason for not using a lazy val is exception
   // handling.
   @volatile var addresses: Set[Address] = _
@@ -209,10 +209,10 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc
         endpointManager = Some(manager)
 
         try {
-          val addressesPromise: Promise[Seq[(AkkaProtocolTransport, Address)]] = Promise()
+          val addressesPromise: Promise[Seq[(PekkoProtocolTransport, Address)]] = Promise()
           manager ! Listen(addressesPromise)
 
-          val transports: Seq[(AkkaProtocolTransport, Address)] =
+          val transports: Seq[(PekkoProtocolTransport, Address)] =
             Await.result(addressesPromise.future, StartupTimeout.duration)
           if (transports.isEmpty) throw new RemoteTransportException("No transport drivers were loaded.", null)
 
@@ -297,7 +297,7 @@ private[remote] object EndpointManager {
 
   // Messages between Remoting and EndpointManager
   sealed trait RemotingCommand extends NoSerializationVerificationNeeded
-  final case class Listen(addressesPromise: Promise[Seq[(AkkaProtocolTransport, Address)]]) extends RemotingCommand
+  final case class Listen(addressesPromise: Promise[Seq[(PekkoProtocolTransport, Address)]]) extends RemotingCommand
   case object StartupFinished extends RemotingCommand
   case object ShutdownAndFlush extends RemotingCommand
   @InternalStableApi
@@ -321,10 +321,10 @@ private[remote] object EndpointManager {
   // Messages internal to EndpointManager
   case object Prune extends NoSerializationVerificationNeeded
   final case class ListensResult(
-      addressesPromise: Promise[Seq[(AkkaProtocolTransport, Address)]],
-      results: Seq[(AkkaProtocolTransport, Address, Promise[AssociationEventListener])])
+      addressesPromise: Promise[Seq[(PekkoProtocolTransport, Address)]],
+      results: Seq[(PekkoProtocolTransport, Address, Promise[AssociationEventListener])])
       extends NoSerializationVerificationNeeded
-  final case class ListensFailure(addressesPromise: Promise[Seq[(AkkaProtocolTransport, Address)]], cause: Throwable)
+  final case class ListensFailure(addressesPromise: Promise[Seq[(PekkoProtocolTransport, Address)]], cause: Throwable)
       extends NoSerializationVerificationNeeded
 
   // Helper class to store address pairs
@@ -503,14 +503,14 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter)
   // will be not part of this map!
   val endpoints = new EndpointRegistry
   // Mapping between transports and the local addresses they listen to
-  var transportMapping: Map[Address, AkkaProtocolTransport] = Map()
+  var transportMapping: Map[Address, PekkoProtocolTransport] = Map()
 
   val pruneInterval: FiniteDuration = (settings.RetryGateClosedFor * 2).max(1.second).min(10.seconds)
 
   val pruneTimerCancellable: Cancellable =
     context.system.scheduler.scheduleWithFixedDelay(pruneInterval, pruneInterval, self, Prune)
 
-  var pendingReadHandoffs = Map[ActorRef, AkkaProtocolHandle]()
+  var pendingReadHandoffs = Map[ActorRef, PekkoProtocolHandle]()
   var stashedInbound = Map[ActorRef, Vector[InboundAssociation]]()
 
   def handleStashedInbound(endpoint: ActorRef, writerIsIdle: Boolean): Unit = {
@@ -696,7 +696,7 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter)
         case _ => // nothing to stop
       }
 
-      def matchesQuarantine(handle: AkkaProtocolHandle): Boolean = {
+      def matchesQuarantine(handle: PekkoProtocolHandle): Boolean = {
         handle.remoteAddress == address &&
         uidToQuarantineOption.forall(_ == handle.handshakeInfo.uid)
       }
@@ -717,7 +717,7 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter)
       stashedInbound = stashedInbound.map {
         case (writer, associations) =>
           writer -> associations.filter { assoc =>
-            val handle = assoc.association.asInstanceOf[AkkaProtocolHandle]
+            val handle = assoc.association.asInstanceOf[PekkoProtocolHandle]
             val drop = matchesQuarantine(handle)
             if (drop) handle.disassociate("the stashed inbound handle was quarantined", log)
             !drop
@@ -755,7 +755,7 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter)
 
       }
 
-    case ia @ InboundAssociation(_: AkkaProtocolHandle) =>
+    case ia @ InboundAssociation(_: PekkoProtocolHandle) =>
       handleInboundAssociation(ia, writerIsIdle = false)
     case EndpointWriter.StoppedReading(endpoint) =>
       acceptPendingReader(takingOverFrom = endpoint)
@@ -807,13 +807,13 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter)
   }
 
   def flushing: Receive = {
-    case s: Send                                   => extendedSystem.deadLetters ! s
-    case InboundAssociation(h: AkkaProtocolHandle) => h.disassociate(AssociationHandle.Shutdown)
-    case Terminated(_)                             => // why should we care now?
+    case s: Send                                    => extendedSystem.deadLetters ! s
+    case InboundAssociation(h: PekkoProtocolHandle) => h.disassociate(AssociationHandle.Shutdown)
+    case Terminated(_)                              => // why should we care now?
   }
 
   def handleInboundAssociation(ia: InboundAssociation, writerIsIdle: Boolean): Unit = ia match {
-    case ia @ InboundAssociation(handle: AkkaProtocolHandle) =>
+    case ia @ InboundAssociation(handle: PekkoProtocolHandle) =>
       endpoints.readOnlyEndpointFor(handle.remoteAddress) match {
         case Some((endpoint, _)) =>
           pendingReadHandoffs
@@ -861,7 +861,7 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter)
     case _ => // ignore
   }
 
-  private def createAndRegisterEndpoint(handle: AkkaProtocolHandle): Unit = {
+  private def createAndRegisterEndpoint(handle: PekkoProtocolHandle): Unit = {
     val writing = settings.UsePassiveConnections && !endpoints.hasWritableEndpointFor(handle.remoteAddress)
     eventPublisher.notifyListeners(AssociatedEvent(handle.localAddress, handle.remoteAddress, inbound = true))
 
@@ -882,15 +882,15 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter)
     }
   }
 
-  private def listens: Future[Seq[(AkkaProtocolTransport, Address, Promise[AssociationEventListener])]] = {
+  private def listens: Future[Seq[(PekkoProtocolTransport, Address, Promise[AssociationEventListener])]] = {
     /*
      * Constructs chains of adapters on top of each driver as given in configuration. The resulting structure looks
      * like the following:
-     *   AkkaProtocolTransport <- Adapter <- ... <- Adapter <- Driver
+     *   PekkoProtocolTransport <- Adapter <- ... <- Adapter <- Driver
      *
-     * The transports variable contains only the heads of each chains (the AkkaProtocolTransport instances).
+     * The transports variable contains only the heads of each chains (the PekkoProtocolTransport instances).
      */
-    val transports: Seq[AkkaProtocolTransport] = for ((fqn, adapters, config) <- settings.Transports) yield {
+    val transports: Seq[PekkoProtocolTransport] = for ((fqn, adapters, config) <- settings.Transports) yield {
 
       val args = Seq(classOf[ExtendedActorSystem] -> context.system, classOf[Config] -> config)
 
@@ -921,10 +921,11 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter)
             provider.create(t, context.system.asInstanceOf[ExtendedActorSystem])
         }
 
-      // Apply AkkaProtocolTransport wrapper to the end of the chain
+      // Apply PekkoProtocolTransport wrapper to the end of the chain
       // The chain at this point:
-      //   AkkaProtocolTransport <- Adapter <- ... <- Adapter <- Driver
-      new AkkaProtocolTransport(wrappedTransport, context.system, new AkkaProtocolSettings(conf), AkkaPduProtobufCodec)
+      //   PekkoProtocolTransport <- Adapter <- ... <- Adapter <- Driver
+      new PekkoProtocolTransport(wrappedTransport, context.system, new PekkoProtocolSettings(conf),
+        PekkoPduProtobufCodec$)
     }
 
     // Collect all transports, listen addresses and listener promises in one future
@@ -950,7 +951,7 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter)
     }
   }
 
-  private def removePendingReader(takingOverFrom: ActorRef, withHandle: AkkaProtocolHandle): Unit = {
+  private def removePendingReader(takingOverFrom: ActorRef, withHandle: PekkoProtocolHandle): Unit = {
     if (pendingReadHandoffs.get(takingOverFrom).exists(handle => handle == withHandle))
       pendingReadHandoffs -= takingOverFrom
   }
@@ -958,9 +959,9 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter)
   private def createEndpoint(
       remoteAddress: Address,
       localAddress: Address,
-      transport: AkkaProtocolTransport,
+      transport: PekkoProtocolTransport,
       endpointSettings: RemoteSettings,
-      handleOption: Option[AkkaProtocolHandle],
+      handleOption: Option[PekkoProtocolHandle],
       writing: Boolean): ActorRef = {
     require(transportMapping contains localAddress, "Transport mapping is not defined for the address")
     // refuseUid is ignored for read-only endpoints since the UID of the remote system is already known and has passed
@@ -979,7 +980,7 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter)
                 refuseUid,
                 transport,
                 endpointSettings,
-                AkkaPduProtobufCodec,
+                PekkoPduProtobufCodec$,
                 receiveBuffers))
             .withDeploy(Deploy.local),
           "reliableEndpointWriter-" + AddressUrlEncoder(remoteAddress) + "-" + endpointId.next()))
@@ -995,7 +996,7 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter)
                 refuseUid,
                 transport,
                 endpointSettings,
-                AkkaPduProtobufCodec,
+                PekkoPduProtobufCodec$,
                 receiveBuffers,
                 reliableDeliverySupervisor = None))
             .withDeploy(Deploy.local),
diff --git a/akka-remote/src/main/scala/org/apache/pekko/remote/transport/FailureInjectorTransportAdapter.scala b/akka-remote/src/main/scala/org/apache/pekko/remote/transport/FailureInjectorTransportAdapter.scala
index 6ef67f43f6..2b941d1b1e 100644
--- a/akka-remote/src/main/scala/org/apache/pekko/remote/transport/FailureInjectorTransportAdapter.scala
+++ b/akka-remote/src/main/scala/org/apache/pekko/remote/transport/FailureInjectorTransportAdapter.scala
@@ -14,7 +14,7 @@ import FailureInjectorTransportAdapter._
 import scala.annotation.nowarn
 
 import org.apache.pekko
-import pekko.AkkaException
+import pekko.PekkoException
 import pekko.actor.{ Address, ExtendedActorSystem }
 import pekko.event.{ Logging, LoggingAdapter }
 import pekko.remote.transport.AssociationHandle.{ HandleEvent, HandleEventListener }
@@ -23,7 +23,7 @@ import pekko.util.ByteString
 
 @SerialVersionUID(1L)
 @deprecated("Classic remoting is deprecated, use Artery", "2.6.0")
-final case class FailureInjectorException(msg: String) extends AkkaException(msg) with NoStackTrace
+final case class FailureInjectorException(msg: String) extends PekkoException(msg) with NoStackTrace
 
 @deprecated("Classic remoting is deprecated, use Artery", "2.6.0")
 class FailureInjectorProvider extends TransportAdapterProvider {
diff --git a/akka-remote/src/main/scala/org/apache/pekko/remote/transport/AkkaPduCodec.scala b/akka-remote/src/main/scala/org/apache/pekko/remote/transport/PekkoPduCodec.scala
similarity index 84%
rename from akka-remote/src/main/scala/org/apache/pekko/remote/transport/AkkaPduCodec.scala
rename to akka-remote/src/main/scala/org/apache/pekko/remote/transport/PekkoPduCodec.scala
index 135b0fe768..03590912b6 100644
--- a/akka-remote/src/main/scala/org/apache/pekko/remote/transport/AkkaPduCodec.scala
+++ b/akka-remote/src/main/scala/org/apache/pekko/remote/transport/PekkoPduCodec.scala
@@ -6,7 +6,7 @@ package org.apache.pekko.remote.transport
 
 import scala.annotation.nowarn
 import org.apache.pekko
-import pekko.AkkaException
+import pekko.PekkoException
 import pekko.actor.{ ActorRef, Address, AddressFromURIString, InternalActorRef }
 import pekko.protobufv3.internal.InvalidProtocolBufferException
 import pekko.remote._
@@ -18,25 +18,25 @@ import pekko.util.OptionVal
  * INTERNAL API
  */
 @SerialVersionUID(1L)
-private[remote] class PduCodecException(msg: String, cause: Throwable) extends AkkaException(msg, cause)
+private[remote] class PduCodecException(msg: String, cause: Throwable) extends PekkoException(msg, cause)
 
 /**
  * INTERNAL API
  *
- * Companion object of the [[pekko.remote.transport.AkkaPduCodec]] trait. Contains the representation case classes
+ * Companion object of the [[pekko.remote.transport.PekkoPduCodec]] trait. Contains the representation case classes
  * of decoded Akka Protocol Data Units (PDUs).
  */
 @nowarn("msg=deprecated")
-private[remote] object AkkaPduCodec {
+private[remote] object PekkoPduCodec {
 
   /**
    * Trait that represents decoded Akka PDUs (Protocol Data Units)
    */
-  sealed trait AkkaPdu
-  final case class Associate(info: HandshakeInfo) extends AkkaPdu
-  final case class Disassociate(reason: AssociationHandle.DisassociateInfo) extends AkkaPdu
-  case object Heartbeat extends AkkaPdu
-  final case class Payload(bytes: ByteString) extends AkkaPdu
+  sealed trait PekkoPdu
+  final case class Associate(info: HandshakeInfo) extends PekkoPdu
+  final case class Disassociate(reason: AssociationHandle.DisassociateInfo) extends PekkoPdu
+  case object Heartbeat extends PekkoPdu
+  final case class Payload(bytes: ByteString) extends PekkoPdu
 
   final case class Message(
       recipient: InternalActorRef,
@@ -58,32 +58,33 @@ private[remote] object AkkaPduCodec {
  * A Codec that is able to convert Akka PDUs (Protocol Data Units) from and to [[pekko.util.ByteString]]s.
  */
 @nowarn("msg=deprecated")
-private[remote] trait AkkaPduCodec {
-  import AkkaPduCodec._
+private[remote] trait PekkoPduCodec {
+  import PekkoPduCodec._
 
   /**
-   * Returns an [[pekko.remote.transport.AkkaPduCodec.AkkaPdu]] instance that represents the PDU contained in the raw
+   * Returns an [[pekko.remote.transport.PekkoPduCodec.PekkoPdu]] instance that represents the PDU contained in the raw
    * ByteString.
+   *
    * @param raw
    *   Encoded raw byte representation of an Akka PDU
    * @return
    *   Case class representation of the decoded PDU that can be used in a match statement
    */
-  def decodePdu(raw: ByteString): AkkaPdu
+  def decodePdu(raw: ByteString): PekkoPdu
 
   /**
-   * Takes an [[pekko.remote.transport.AkkaPduCodec.AkkaPdu]] representation of an Akka PDU and returns its encoded
+   * Takes an [[pekko.remote.transport.PekkoPduCodec.PekkoPdu]] representation of an Akka PDU and returns its encoded
    * form as a [[pekko.util.ByteString]].
    *
    * For the same effect the constructXXX methods might be called directly, taking method parameters instead of the
-   * [[pekko.remote.transport.AkkaPduCodec.AkkaPdu]] final case classes.
+   * [[pekko.remote.transport.PekkoPduCodec.PekkoPdu]] final case classes.
    *
    * @param pdu
    *   The Akka Protocol Data Unit to be encoded
    * @return
    *   Encoded form as raw bytes
    */
-  def encodePdu(pdu: AkkaPdu): ByteString = pdu match {
+  def encodePdu(pdu: PekkoPdu): ByteString = pdu match {
     case Associate(info)      => constructAssociate(info)
     case Payload(bytes)       => constructPayload(bytes)
     case Disassociate(reason) => constructDisassociate(reason)
@@ -118,8 +119,8 @@ private[remote] trait AkkaPduCodec {
  * INTERNAL API
  */
 @nowarn("msg=deprecated")
-private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec {
-  import AkkaPduCodec._
+private[remote] object PekkoPduProtobufCodec$ extends PekkoPduCodec {
+  import PekkoPduCodec._
 
   private def ackBuilder(ack: Ack): AcknowledgementInfo.Builder = {
     val ackBuilder = AcknowledgementInfo.newBuilder()
@@ -165,10 +166,10 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec {
 
   override def constructPayload(payload: ByteString): ByteString =
     ByteString.fromArrayUnsafe(
-      AkkaProtocolMessage.newBuilder().setPayload(ByteStringUtils.toProtoByteStringUnsafe(payload)).build.toByteArray)
+      PekkoProtocolMessage.newBuilder().setPayload(ByteStringUtils.toProtoByteStringUnsafe(payload)).build.toByteArray)
 
   override def constructAssociate(info: HandshakeInfo): ByteString = {
-    val handshakeInfo = AkkaHandshakeInfo.newBuilder.setOrigin(serializeAddress(info.origin)).setUid(info.uid.toLong)
+    val handshakeInfo = PekkoHandshakeInfo.newBuilder.setOrigin(serializeAddress(info.origin)).setUid(info.uid.toLong)
     constructControlMessagePdu(WireFormats.CommandType.ASSOCIATE, Some(handshakeInfo))
   }
 
@@ -187,9 +188,9 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec {
   override val constructHeartbeat: ByteString =
     constructControlMessagePdu(WireFormats.CommandType.HEARTBEAT, None)
 
-  override def decodePdu(raw: ByteString): AkkaPdu = {
+  override def decodePdu(raw: ByteString): PekkoPdu = {
     try {
-      val pdu = AkkaProtocolMessage.parseFrom(raw.toArrayUnsafe())
+      val pdu = PekkoProtocolMessage.parseFrom(raw.toArrayUnsafe())
       if (pdu.hasPayload) Payload(ByteString.fromByteBuffer(pdu.getPayload.asReadOnlyByteBuffer()))
       else if (pdu.hasInstruction) decodeControlPdu(pdu.getInstruction)
       else
@@ -230,7 +231,7 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec {
     (ackOption, messageOption)
   }
 
-  private def decodeControlPdu(controlPdu: AkkaControlMessage): AkkaPdu = {
+  private def decodeControlPdu(controlPdu: PekkoControlMessage): PekkoPdu = {
 
     controlPdu.getCommandType match {
       case CommandType.ASSOCIATE if controlPdu.hasHandshakeInfo =>
@@ -254,14 +255,14 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec {
 
   private def constructControlMessagePdu(
       code: WireFormats.CommandType,
-      handshakeInfo: Option[AkkaHandshakeInfo.Builder]): ByteString = {
+      handshakeInfo: Option[PekkoHandshakeInfo.Builder]): ByteString = {
 
-    val controlMessageBuilder = AkkaControlMessage.newBuilder()
+    val controlMessageBuilder = PekkoControlMessage.newBuilder()
     controlMessageBuilder.setCommandType(code)
     handshakeInfo.foreach(controlMessageBuilder.setHandshakeInfo)
 
     ByteString.ByteString1C(
-      AkkaProtocolMessage
+      PekkoProtocolMessage
         .newBuilder()
         .setInstruction(controlMessageBuilder.build)
         .build
diff --git a/akka-remote/src/main/scala/org/apache/pekko/remote/transport/AkkaProtocolTransport.scala b/akka-remote/src/main/scala/org/apache/pekko/remote/transport/PekkoProtocolTransport.scala
similarity index 91%
rename from akka-remote/src/main/scala/org/apache/pekko/remote/transport/AkkaProtocolTransport.scala
rename to akka-remote/src/main/scala/org/apache/pekko/remote/transport/PekkoProtocolTransport.scala
index 02d544a39e..eb500fe716 100644
--- a/akka-remote/src/main/scala/org/apache/pekko/remote/transport/AkkaProtocolTransport.scala
+++ b/akka-remote/src/main/scala/org/apache/pekko/remote/transport/PekkoProtocolTransport.scala
@@ -15,15 +15,15 @@ import scala.annotation.nowarn
 import com.typesafe.config.Config
 
 import org.apache.pekko
-import pekko.{ AkkaException, OnlyCauseStackTrace }
+import pekko.{ OnlyCauseStackTrace, PekkoException }
 import pekko.actor._
 import pekko.actor.SupervisorStrategy.Stop
 import pekko.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
 import pekko.pattern.pipe
 import pekko.remote._
 import pekko.remote.transport.ActorTransportAdapter._
-import pekko.remote.transport.AkkaPduCodec._
-import pekko.remote.transport.AkkaProtocolTransport._
+import pekko.remote.transport.PekkoPduCodec._
+import pekko.remote.transport.PekkoProtocolTransport._
 import pekko.remote.transport.AssociationHandle._
 import pekko.remote.transport.ProtocolStateActor._
 import pekko.remote.transport.Transport._
@@ -31,11 +31,12 @@ import pekko.util.ByteString
 import pekko.util.Helpers.Requiring
 
 @SerialVersionUID(1L)
-class AkkaProtocolException(msg: String, cause: Throwable) extends AkkaException(msg, cause) with OnlyCauseStackTrace {
+class PekkoProtocolException(
+    msg: String, cause: Throwable) extends PekkoException(msg, cause) with OnlyCauseStackTrace {
   def this(msg: String) = this(msg, null)
 }
 
-private[remote] class AkkaProtocolSettings(config: Config) {
+private[remote] class PekkoProtocolSettings(config: Config) {
 
   import config._
 
@@ -62,7 +63,7 @@ private[remote] class AkkaProtocolSettings(config: Config) {
 }
 
 @nowarn("msg=deprecated")
-private[remote] object AkkaProtocolTransport { // Couldn't these go into the Remoting Extension/ RemoteSettings instead?
+private[remote] object PekkoProtocolTransport { // Couldn't these go into the Remoting Extension/ RemoteSettings instead?
   val AkkaScheme: String = "akka"
   val AkkaOverhead: Int = 0 // Don't know yet
   val UniqueId = new java.util.concurrent.atomic.AtomicInteger(0)
@@ -105,42 +106,42 @@ final case class HandshakeInfo(origin: Address, uid: Int, cookie: Option[String]
  *   the codec that will be used to encode/decode Akka PDUs
  */
 @nowarn("msg=deprecated")
-private[remote] class AkkaProtocolTransport(
+private[remote] class PekkoProtocolTransport(
     wrappedTransport: Transport,
     private val system: ActorSystem,
-    private val settings: AkkaProtocolSettings,
-    private val codec: AkkaPduCodec)
+    private val settings: PekkoProtocolSettings,
+    private val codec: PekkoPduCodec)
     extends ActorTransportAdapter(wrappedTransport, system) {
 
   override val addedSchemeIdentifier: String = AkkaScheme
 
   override def managementCommand(cmd: Any): Future[Boolean] = wrappedTransport.managementCommand(cmd)
 
-  def associate(remoteAddress: Address, refuseUid: Option[Int]): Future[AkkaProtocolHandle] = {
+  def associate(remoteAddress: Address, refuseUid: Option[Int]): Future[PekkoProtocolHandle] = {
     // Prepare a future, and pass its promise to the manager
     val statusPromise: Promise[AssociationHandle] = Promise()
 
     manager ! AssociateUnderlyingRefuseUid(removeScheme(remoteAddress), statusPromise, refuseUid)
 
-    statusPromise.future.mapTo[AkkaProtocolHandle]
+    statusPromise.future.mapTo[PekkoProtocolHandle]
   }
 
-  override val maximumOverhead: Int = AkkaProtocolTransport.AkkaOverhead
+  override val maximumOverhead: Int = PekkoProtocolTransport.AkkaOverhead
   protected def managerName = s"akkaprotocolmanager.${wrappedTransport.schemeIdentifier}${UniqueId.getAndIncrement}"
   protected def managerProps = {
     val wt = wrappedTransport
     val s = settings
-    Props(classOf[AkkaProtocolManager], wt, s).withDeploy(Deploy.local)
+    Props(classOf[PekkoProtocolManager], wt, s).withDeploy(Deploy.local)
   }
 }
 
 @nowarn("msg=deprecated")
-private[transport] class AkkaProtocolManager(
+private[transport] class PekkoProtocolManager(
     private val wrappedTransport: Transport,
-    private val settings: AkkaProtocolSettings)
+    private val settings: PekkoProtocolSettings)
     extends ActorTransportAdapterManager {
 
-  // The AkkaProtocolTransport does not handle the recovery of associations, this task is implemented in the
+  // The PekkoProtocolTransport does not handle the recovery of associations, this task is implemented in the
   // remoting itself. Hence the strategy Stop.
   override val supervisorStrategy = OneForOneStrategy() {
     case NonFatal(_) => Stop
@@ -167,7 +168,7 @@ private[transport] class AkkaProtocolManager(
             handle,
             stateActorAssociationHandler,
             stateActorSettings,
-            AkkaPduProtobufCodec,
+            PekkoPduProtobufCodec$,
             failureDetector)),
         actorNameFor(handle.remoteAddress))
 
@@ -200,7 +201,7 @@ private[transport] class AkkaProtocolManager(
           statusPromise,
           stateActorWrappedTransport,
           stateActorSettings,
-          AkkaPduProtobufCodec,
+          PekkoPduProtobufCodec$,
           failureDetector,
           refuseUid)),
       actorNameFor(remoteAddress))
@@ -212,14 +213,14 @@ private[transport] class AkkaProtocolManager(
 }
 
 @nowarn("msg=deprecated")
-private[remote] class AkkaProtocolHandle(
+private[remote] class PekkoProtocolHandle(
     _localAddress: Address,
     _remoteAddress: Address,
     val readHandlerPromise: Promise[HandleEventListener],
     _wrappedHandle: AssociationHandle,
     val handshakeInfo: HandshakeInfo,
     private val stateActor: ActorRef,
-    private val codec: AkkaPduCodec)
+    private val codec: PekkoPduCodec)
     extends AbstractTransportAdapterHandle(_localAddress, _remoteAddress, _wrappedHandle, AkkaScheme) {
 
   override def write(payload: ByteString): Boolean = wrappedHandle.write(codec.constructPayload(payload))
@@ -300,8 +301,8 @@ private[remote] object ProtocolStateActor {
       remoteAddress: Address,
       statusPromise: Promise[AssociationHandle],
       transport: Transport,
-      settings: AkkaProtocolSettings,
-      codec: AkkaPduCodec,
+      settings: PekkoProtocolSettings,
+      codec: PekkoPduCodec,
       failureDetector: FailureDetector,
       refuseUid: Option[Int]): Props =
     Props(
@@ -319,8 +320,8 @@ private[remote] object ProtocolStateActor {
       handshakeInfo: HandshakeInfo,
       wrappedHandle: AssociationHandle,
       associationListener: AssociationEventListener,
-      settings: AkkaProtocolSettings,
-      codec: AkkaPduCodec,
+      settings: PekkoProtocolSettings,
+      codec: PekkoPduCodec,
       failureDetector: FailureDetector): Props =
     Props(
       classOf[ProtocolStateActor],
@@ -337,8 +338,8 @@ private[remote] class ProtocolStateActor(
     initialData: InitialProtocolStateData,
     private val localHandshakeInfo: HandshakeInfo,
     private val refuseUid: Option[Int],
-    private val settings: AkkaProtocolSettings,
-    private val codec: AkkaPduCodec,
+    private val settings: PekkoProtocolSettings,
+    private val codec: PekkoPduCodec,
     private val failureDetector: FailureDetector)
     extends Actor
     with FSM[AssociationState, ProtocolStateData]
@@ -353,8 +354,8 @@ private[remote] class ProtocolStateActor(
       remoteAddress: Address,
       statusPromise: Promise[AssociationHandle],
       transport: Transport,
-      settings: AkkaProtocolSettings,
-      codec: AkkaPduCodec,
+      settings: PekkoProtocolSettings,
+      codec: PekkoPduCodec,
       failureDetector: FailureDetector,
       refuseUid: Option[Int]) = {
     this(
@@ -371,8 +372,8 @@ private[remote] class ProtocolStateActor(
       handshakeInfo: HandshakeInfo,
       wrappedHandle: AssociationHandle,
       associationListener: AssociationEventListener,
-      settings: AkkaProtocolSettings,
-      codec: AkkaPduCodec,
+      settings: PekkoProtocolSettings,
+      codec: PekkoPduCodec,
       failureDetector: FailureDetector) = {
     this(
       InboundUnassociated(associationListener, wrappedHandle),
@@ -555,7 +556,7 @@ private[remote] class ProtocolStateActor(
               listener.notify(InboundPayload(payload))
               stay()
             case msg =>
-              throw new AkkaProtocolException(
+              throw new PekkoProtocolException(
                 s"unhandled message in state Open(InboundPayload) with type [${safeClassName(msg)}]")
           }
 
@@ -570,7 +571,7 @@ private[remote] class ProtocolStateActor(
         case ListenerReady(_, wrappedHandle)            => wrappedHandle
         case AssociatedWaitHandler(_, wrappedHandle, _) => wrappedHandle
         case msg =>
-          throw new AkkaProtocolException(
+          throw new PekkoProtocolException(
             s"unhandled message in state Open(DisassociateUnderlying) with type [${safeClassName(msg)}]")
       }
       // No debug logging here as sending DisassociateUnderlying(Unknown) should have been logged from where
@@ -627,19 +628,19 @@ private[remote] class ProtocolStateActor(
     case StopEvent(reason, _, OutboundUnassociated(_, statusPromise, _)) =>
       statusPromise.tryFailure(reason match {
         case FSM.Failure(info: DisassociateInfo) => disassociateException(info)
-        case _                                   => new AkkaProtocolException("Transport disassociated before handshake finished")
+        case _                                   => new PekkoProtocolException("Transport disassociated before handshake finished")
       })
 
     case StopEvent(reason, _, OutboundUnderlyingAssociated(statusPromise, wrappedHandle)) =>
       statusPromise.tryFailure(reason match {
         case FSM.Failure(TimeoutReason(errorMessage)) =>
-          new AkkaProtocolException(errorMessage)
+          new PekkoProtocolException(errorMessage)
         case FSM.Failure(info: DisassociateInfo) =>
           disassociateException(info)
         case FSM.Failure(ForbiddenUidReason) =>
           InvalidAssociationException("The remote system has a UID that has been quarantined. Association aborted.")
         case _ =>
-          new AkkaProtocolException("Transport disassociated before handshake finished")
+          new PekkoProtocolException("Transport disassociated before handshake finished")
       })
       wrappedHandle.disassociate(disassociationReason(reason), log)
 
@@ -668,7 +669,7 @@ private[remote] class ProtocolStateActor(
 
   private def disassociateException(info: DisassociateInfo): Exception = info match {
     case Unknown =>
-      new AkkaProtocolException("The remote system explicitly disassociated (reason unknown).")
+      new PekkoProtocolException("The remote system explicitly disassociated (reason unknown).")
     case Shutdown =>
       InvalidAssociationException("The remote system refused the association because it is shutting down.")
     case Quarantined =>
@@ -696,7 +697,7 @@ private[remote] class ProtocolStateActor(
     listenForListenerRegistration(readHandlerPromise)
 
     statusPromise.success(
-      new AkkaProtocolHandle(
+      new PekkoProtocolHandle(
         localAddress,
         wrappedHandle.remoteAddress,
         readHandlerPromise,
@@ -716,7 +717,7 @@ private[remote] class ProtocolStateActor(
 
     associationListener.notify(
       InboundAssociation(
-        new AkkaProtocolHandle(
+        new PekkoProtocolHandle(
           localAddress,
           handshakeInfo.origin,
           readHandlerPromise,
@@ -727,11 +728,11 @@ private[remote] class ProtocolStateActor(
     readHandlerPromise.future
   }
 
-  private def decodePdu(pdu: ByteString): AkkaPdu =
+  private def decodePdu(pdu: ByteString): PekkoPdu =
     try codec.decodePdu(pdu)
     catch {
       case NonFatal(e) =>
-        throw new AkkaProtocolException("Error while decoding incoming Akka PDU of length: " + pdu.length, e)
+        throw new PekkoProtocolException("Error while decoding incoming Akka PDU of length: " + pdu.length, e)
     }
 
   // Neither heartbeats neither disassociate cares about backing off if write fails:
@@ -740,20 +741,20 @@ private[remote] class ProtocolStateActor(
   private def sendHeartbeat(wrappedHandle: AssociationHandle): Boolean =
     try wrappedHandle.write(codec.constructHeartbeat)
     catch {
-      case NonFatal(e) => throw new AkkaProtocolException("Error writing HEARTBEAT to transport", e)
+      case NonFatal(e) => throw new PekkoProtocolException("Error writing HEARTBEAT to transport", e)
     }
 
   private def sendDisassociate(wrappedHandle: AssociationHandle, info: DisassociateInfo): Unit =
     try wrappedHandle.write(codec.constructDisassociate(info))
     catch {
-      case NonFatal(e) => throw new AkkaProtocolException("Error writing DISASSOCIATE to transport", e)
+      case NonFatal(e) => throw new PekkoProtocolException("Error writing DISASSOCIATE to transport", e)
     }
 
   private def sendAssociate(wrappedHandle: AssociationHandle, info: HandshakeInfo): Boolean =
     try {
       wrappedHandle.write(codec.constructAssociate(info))
     } catch {
-      case NonFatal(e) => throw new AkkaProtocolException("Error writing ASSOCIATE to transport", e)
+      case NonFatal(e) => throw new PekkoProtocolException("Error writing ASSOCIATE to transport", e)
     }
 
   private def disassociationReason(reason: FSM.Reason): String = reason match {
diff --git a/akka-remote/src/main/scala/org/apache/pekko/remote/transport/ThrottlerTransportAdapter.scala b/akka-remote/src/main/scala/org/apache/pekko/remote/transport/ThrottlerTransportAdapter.scala
index 9ed2b249a0..84474a0438 100644
--- a/akka-remote/src/main/scala/org/apache/pekko/remote/transport/ThrottlerTransportAdapter.scala
+++ b/akka-remote/src/main/scala/org/apache/pekko/remote/transport/ThrottlerTransportAdapter.scala
@@ -25,7 +25,7 @@ import pekko.dispatch.sysmsg.{ Unwatch, Watch }
 import pekko.pattern.{ ask, pipe, PromiseActorRef }
 import pekko.remote.RARP
 import pekko.remote.transport.ActorTransportAdapter.AssociateUnderlying
-import pekko.remote.transport.AkkaPduCodec.Associate
+import pekko.remote.transport.PekkoPduCodec.Associate
 import pekko.remote.transport.AssociationHandle.{
   ActorHandleEventListener,
   DisassociateInfo,
@@ -565,7 +565,7 @@ private[transport] class ThrottledAssociation(
   // This method captures ASSOCIATE packets and extracts the origin address
   private def peekOrigin(b: ByteString): Option[Address] = {
     try {
-      AkkaPduProtobufCodec.decodePdu(b) match {
+      PekkoPduProtobufCodec$.decodePdu(b) match {
         case Associate(info) => Some(info.origin)
         case _               => None
       }
diff --git a/akka-remote/src/main/scala/org/apache/pekko/remote/transport/Transport.scala b/akka-remote/src/main/scala/org/apache/pekko/remote/transport/Transport.scala
index c70d161da8..804aecfcd9 100644
--- a/akka-remote/src/main/scala/org/apache/pekko/remote/transport/Transport.scala
+++ b/akka-remote/src/main/scala/org/apache/pekko/remote/transport/Transport.scala
@@ -10,7 +10,7 @@ import scala.util.control.NoStackTrace
 import scala.annotation.nowarn
 
 import org.apache.pekko
-import pekko.AkkaException
+import pekko.PekkoException
 import pekko.actor.{ ActorRef, Address, NoSerializationVerificationNeeded }
 import pekko.actor.DeadLetterSuppression
 import pekko.event.LoggingAdapter
@@ -28,7 +28,7 @@ object Transport {
    */
   @SerialVersionUID(1L)
   final case class InvalidAssociationException(msg: String, cause: Throwable = null)
-      extends AkkaException(msg, cause)
+      extends PekkoException(msg, cause)
       with NoStackTrace
 
   /**
diff --git a/akka-remote/src/main/scala/org/apache/pekko/remote/transport/netty/NettyHelpers.scala b/akka-remote/src/main/scala/org/apache/pekko/remote/transport/netty/NettyHelpers.scala
index e2a3a74ea7..a62bfb5bb1 100644
--- a/akka-remote/src/main/scala/org/apache/pekko/remote/transport/netty/NettyHelpers.scala
+++ b/akka-remote/src/main/scala/org/apache/pekko/remote/transport/netty/NettyHelpers.scala
@@ -11,7 +11,7 @@ import scala.util.control.NonFatal
 import org.jboss.netty.channel._
 
 import org.apache.pekko
-import pekko.AkkaException
+import pekko.PekkoException
 import pekko.util.unused
 
 /**
@@ -30,7 +30,7 @@ private[netty] trait NettyHelpers {
   protected def onException(@unused ctx: ChannelHandlerContext, @unused e: ExceptionEvent): Unit = ()
 
   final protected def transformException(ctx: ChannelHandlerContext, ev: ExceptionEvent): Unit = {
-    val cause = if (ev.getCause ne null) ev.getCause else new AkkaException("Unknown cause")
+    val cause = if (ev.getCause ne null) ev.getCause else new PekkoException("Unknown cause")
     cause match {
       case _: ClosedChannelException => // Ignore
       case null | NonFatal(_)        => onException(ctx, ev)
diff --git a/akka-remote/src/test/scala-jdk9-only/org/apache/pekko/remote/artery/jfr/JFRRemotingFlightRecorderSpec.scala b/akka-remote/src/test/scala-jdk9-only/org/apache/pekko/remote/artery/jfr/JFRRemotingFlightRecorderSpec.scala
index 74f3a63a01..ee91d48aac 100644
--- a/akka-remote/src/test/scala-jdk9-only/org/apache/pekko/remote/artery/jfr/JFRRemotingFlightRecorderSpec.scala
+++ b/akka-remote/src/test/scala-jdk9-only/org/apache/pekko/remote/artery/jfr/JFRRemotingFlightRecorderSpec.scala
@@ -10,10 +10,10 @@ import org.apache.pekko
 import pekko.actor.ActorSystem
 import pekko.remote.artery.NoOpRemotingFlightRecorder
 import pekko.remote.artery.RemotingFlightRecorder
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.TestKit
 
-class JFRRemotingFlightRecorderSpec extends AkkaSpec {
+class JFRRemotingFlightRecorderSpec extends PekkoSpec {
 
   "The RemotingFlightRecorder" must {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/AccrualFailureDetectorSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/AccrualFailureDetectorSpec.scala
index 71b620dd85..291f9e09f5 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/AccrualFailureDetectorSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/AccrualFailureDetectorSpec.scala
@@ -11,10 +11,10 @@ import scala.annotation.nowarn
 
 import org.apache.pekko
 import pekko.remote.FailureDetector.Clock
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
 @nowarn
-class AccrualFailureDetectorSpec extends AkkaSpec("pekko.loglevel = INFO") {
+class AccrualFailureDetectorSpec extends PekkoSpec("pekko.loglevel = INFO") {
 
   "An AccrualFailureDetector" must {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/AckedDeliverySpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/AckedDeliverySpec.scala
index 3b6bd2e003..9fd476d95e 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/AckedDeliverySpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/AckedDeliverySpec.scala
@@ -10,7 +10,7 @@ import scala.annotation.tailrec
 
 import scala.annotation.nowarn
 
-import org.apache.pekko.testkit.AkkaSpec
+import org.apache.pekko.testkit.PekkoSpec
 
 @nowarn("msg=deprecated")
 object AckedDeliverySpec {
@@ -22,7 +22,7 @@ object AckedDeliverySpec {
 }
 
 @nowarn("msg=deprecated")
-class AckedDeliverySpec extends AkkaSpec {
+class AckedDeliverySpec extends PekkoSpec {
   import AckedDeliverySpec._
 
   def msg(seq: Long) = Sequenced(SeqNo(seq), "msg" + seq)
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/DaemonicSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/DaemonicSpec.scala
index 05d5a997e6..94ace6ee93 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/DaemonicSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/DaemonicSpec.scala
@@ -15,7 +15,7 @@ import pekko.util.ccompat._
 import pekko.util.ccompat.JavaConverters._
 
 @ccompatUsedUntil213
-class DaemonicSpec extends AkkaSpec {
+class DaemonicSpec extends PekkoSpec {
 
   "Remoting configured with daemonic = on" must {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/DeadlineFailureDetectorSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/DeadlineFailureDetectorSpec.scala
index 285bce3373..082b0d5ad6 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/DeadlineFailureDetectorSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/DeadlineFailureDetectorSpec.scala
@@ -8,9 +8,9 @@ import scala.concurrent.duration._
 
 import org.apache.pekko
 import pekko.remote.FailureDetector.Clock
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
-class DeadlineFailureDetectorSpec extends AkkaSpec {
+class DeadlineFailureDetectorSpec extends PekkoSpec {
 
   "A DeadlineFailureDetector" must {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/EndpointRegistrySpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/EndpointRegistrySpec.scala
index 70c0f3e27a..34358a3839 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/EndpointRegistrySpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/EndpointRegistrySpec.scala
@@ -9,9 +9,9 @@ import scala.concurrent.duration._
 import org.apache.pekko
 import pekko.actor.{ Address, Props }
 import pekko.remote.EndpointManager._
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
-class EndpointRegistrySpec extends AkkaSpec {
+class EndpointRegistrySpec extends PekkoSpec {
 
   val actorA = system.actorOf(Props.empty, "actorA")
   val actorB = system.actorOf(Props.empty, "actorB")
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/FailureDetectorRegistrySpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/FailureDetectorRegistrySpec.scala
index a907a148fc..0f30dc5476 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/FailureDetectorRegistrySpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/FailureDetectorRegistrySpec.scala
@@ -8,9 +8,9 @@ import scala.concurrent.duration._
 
 import org.apache.pekko
 import pekko.remote.FailureDetector.Clock
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
-class FailureDetectorRegistrySpec extends AkkaSpec("pekko.loglevel = INFO") {
+class FailureDetectorRegistrySpec extends PekkoSpec("pekko.loglevel = INFO") {
 
   def fakeTimeGenerator(timeIntervals: Seq[Long]): Clock = new Clock {
     @volatile var times =
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/LogSourceSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/LogSourceSpec.scala
index 24cbe11ab3..97d7e39afa 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/LogSourceSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/LogSourceSpec.scala
@@ -12,7 +12,7 @@ import pekko.actor.ExtendedActorSystem
 import pekko.actor.Props
 import pekko.event.Logging
 import pekko.event.Logging.Info
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.TestProbe
 
 object LogSourceSpec {
@@ -24,7 +24,7 @@ object LogSourceSpec {
   }
 }
 
-class LogSourceSpec extends AkkaSpec("""
+class LogSourceSpec extends PekkoSpec("""
     pekko.loglevel = INFO
     pekko.actor.provider = remote
     pekko.remote.classic.netty.tcp.port = 0
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/MessageLoggingSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/MessageLoggingSpec.scala
index 06b7da421d..d1add4a05c 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/MessageLoggingSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/MessageLoggingSpec.scala
@@ -12,7 +12,7 @@ import pekko.actor.{ Actor, ActorIdentity, ActorSystem, ExtendedActorSystem, Ide
 import pekko.serialization.jackson.CborSerializable
 import pekko.testkit.EventFilter
 import pekko.testkit.TestActors
-import pekko.testkit.{ AkkaSpec, ImplicitSender, TestKit }
+import pekko.testkit.{ ImplicitSender, PekkoSpec, TestKit }
 
 object MessageLoggingSpec {
   def config(artery: Boolean) = ConfigFactory.parseString(s"""
@@ -59,7 +59,7 @@ object MessageLoggingSpec {
 class ArteryMessageLoggingSpec extends MessageLoggingSpec(config(true))
 class ClassicMessageLoggingSpec extends MessageLoggingSpec(config(false))
 
-abstract class MessageLoggingSpec(config: Config) extends AkkaSpec(config) with ImplicitSender {
+abstract class MessageLoggingSpec(config: Config) extends PekkoSpec(config) with ImplicitSender {
 
   val remoteSystem = ActorSystem("remote-sys", ConfigFactory.load(config))
   val remoteAddress = remoteSystem.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/NetworkFailureSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/NetworkFailureSpec.scala
index f7dd936e90..1a31233cb6 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/NetworkFailureSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/NetworkFailureSpec.scala
@@ -9,10 +9,10 @@ import java.util.concurrent.atomic.AtomicBoolean
 import scala.concurrent.Future
 
 import org.apache.pekko
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.DefaultTimeout
 
-trait NetworkFailureSpec extends DefaultTimeout { self: AkkaSpec =>
+trait NetworkFailureSpec extends DefaultTimeout { self: PekkoSpec =>
   import scala.concurrent.duration.Duration
 
   import system.dispatcher
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteConfigSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteConfigSpec.scala
index a2071eb27e..c05572ee69 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteConfigSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteConfigSpec.scala
@@ -10,14 +10,14 @@ import scala.annotation.nowarn
 import language.postfixOps
 
 import org.apache.pekko
-import pekko.remote.transport.AkkaProtocolSettings
+import pekko.remote.transport.PekkoProtocolSettings
 import pekko.remote.transport.netty.{ NettyTransportSettings, SSLSettings }
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.util.Helpers
 import pekko.util.Helpers.ConfigOps
 
 @nowarn // classic deprecated
-class RemoteConfigSpec extends AkkaSpec("""
+class RemoteConfigSpec extends PekkoSpec("""
     pekko.actor.provider = remote
     pekko.remote.classic.netty.tcp.port = 0
   """) {
@@ -67,8 +67,8 @@ class RemoteConfigSpec extends AkkaSpec("""
       remoteSettings.config.getString("pekko.remote.classic.log-frame-size-exceeding") should ===("off")
     }
 
-    "be able to parse AkkaProtocol related config elements" in {
-      val settings = new AkkaProtocolSettings(RARP(system).provider.remoteSettings.config)
+    "be able to parse PekkoProtocol related config elements" in {
+      val settings = new PekkoProtocolSettings(RARP(system).provider.remoteSettings.config)
       import settings._
 
       TransportFailureDetectorImplementationClass should ===(classOf[DeadlineFailureDetector].getName)
@@ -84,7 +84,7 @@ class RemoteConfigSpec extends AkkaSpec("""
 
       ConnectionTimeout should ===(15.seconds)
       ConnectionTimeout should ===(
-        new AkkaProtocolSettings(RARP(system).provider.remoteSettings.config).HandshakeTimeout)
+        new PekkoProtocolSettings(RARP(system).provider.remoteSettings.config).HandshakeTimeout)
       WriteBufferHighWaterMark should ===(None)
       WriteBufferLowWaterMark should ===(None)
       SendBufferSize should ===(Some(256000))
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteConsistentHashingRouterSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteConsistentHashingRouterSpec.scala
index e105468742..1e3fed3b91 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteConsistentHashingRouterSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteConsistentHashingRouterSpec.scala
@@ -9,10 +9,10 @@ import pekko.actor.Address
 import pekko.routing.ActorSelectionRoutee
 import pekko.routing.ConsistentHash
 import pekko.routing.ConsistentRoutee
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
 class RemoteConsistentHashingRouterSpec
-    extends AkkaSpec("""
+    extends PekkoSpec("""
     pekko.remote.artery.canonical.port = 0                                                         
     pekko.actor.provider = remote """) {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteDeployerSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteDeployerSpec.scala
index 1e5b7e9502..51f2d49cc4 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteDeployerSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteDeployerSpec.scala
@@ -34,7 +34,7 @@ object RemoteDeployerSpec {
 
 }
 
-class RemoteDeployerSpec extends AkkaSpec(RemoteDeployerSpec.deployerConf) {
+class RemoteDeployerSpec extends PekkoSpec(RemoteDeployerSpec.deployerConf) {
 
   "A RemoteDeployer" must {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteRouterSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteRouterSpec.scala
index 6c7fd4e266..f222af6014 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteRouterSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/RemoteRouterSpec.scala
@@ -24,7 +24,7 @@ object RemoteRouterSpec {
   }
 }
 
-class RemoteRouterSpec extends AkkaSpec(s"""
+class RemoteRouterSpec extends PekkoSpec(s"""
     pekko.actor.provider = remote
     pekko.remote.use-unsafe-remote-features-outside-cluster = on
     pekko.remote.classic.netty.tcp {
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/Ticket1978CommunicationSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/Ticket1978CommunicationSpec.scala
index 72b3ceff76..aca4686071 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/Ticket1978CommunicationSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/Ticket1978CommunicationSpec.scala
@@ -82,7 +82,7 @@ object Configuration {
       val config =
         ConfigFactory.parseString(conf.format(localPort, trustStore, keyStore, cipher, enabled.mkString(", ")))
       val fullConfig = config
-        .withFallback(AkkaSpec.testConf)
+        .withFallback(PekkoSpec.testConf)
         .withFallback(ConfigFactory.load)
         .getConfig("pekko.remote.classic.netty.ssl.security")
       val settings = new SSLSettings(fullConfig)
@@ -106,7 +106,7 @@ object Configuration {
       CipherConfig(true, config, cipher, localPort, remotePort, Some(sslEngineProvider))
     } catch {
       case _: IllegalArgumentException | _: NoSuchAlgorithmException =>
-        CipherConfig(false, AkkaSpec.testConf, cipher, localPort, remotePort, None) // Cannot match against the message since the message might be localized :S
+        CipherConfig(false, PekkoSpec.testConf, cipher, localPort, remotePort, None) // Cannot match against the message since the message might be localized :S
     }
   }
 }
@@ -123,11 +123,11 @@ class Ticket1978CrappyRSAWithMD5OnlyHereToMakeSureThingsWorkSpec
     extends Ticket1978CommunicationSpec(getCipherConfig("", "SSL_RSA_WITH_NULL_MD5"))
 
 class Ticket1978NonExistingRNGSecureSpec
-    extends Ticket1978CommunicationSpec(CipherConfig(false, AkkaSpec.testConf, "NonExistingRNG", 12345, 12346, None))
+    extends Ticket1978CommunicationSpec(CipherConfig(false, PekkoSpec.testConf, "NonExistingRNG", 12345, 12346, None))
 
 @nowarn("msg=deprecated")
 abstract class Ticket1978CommunicationSpec(val cipherConfig: CipherConfig)
-    extends AkkaSpec(cipherConfig.config)
+    extends PekkoSpec(cipherConfig.config)
     with ImplicitSender {
 
   implicit val timeout: Timeout = Timeout(10.seconds)
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/Ticket1978ConfigSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/Ticket1978ConfigSpec.scala
index 45ed11dc4f..113ec200d2 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/Ticket1978ConfigSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/Ticket1978ConfigSpec.scala
@@ -8,7 +8,7 @@ import org.apache.pekko
 import pekko.remote.transport.netty.SSLSettings
 import pekko.testkit._
 
-class Ticket1978ConfigSpec extends AkkaSpec("""
+class Ticket1978ConfigSpec extends PekkoSpec("""
     pekko.remote.classic.netty.ssl.security {
       random-number-generator = "SecureRandom"
     }
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/TransientSerializationErrorSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/TransientSerializationErrorSpec.scala
index 0ea9e59160..b13cb01fbb 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/TransientSerializationErrorSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/TransientSerializationErrorSpec.scala
@@ -11,7 +11,7 @@ import com.typesafe.config.{ Config, ConfigFactory }
 import org.apache.pekko
 import pekko.actor.{ ActorSystem, ExtendedActorSystem, RootActorPath }
 import pekko.serialization.SerializerWithStringManifest
-import pekko.testkit.{ AkkaSpec, TestActors, TestKit }
+import pekko.testkit.{ PekkoSpec, TestActors, TestKit }
 import pekko.util.unused
 
 object TransientSerializationErrorSpec {
@@ -49,7 +49,7 @@ object TransientSerializationErrorSpec {
 }
 
 abstract class AbstractTransientSerializationErrorSpec(config: Config)
-    extends AkkaSpec(
+    extends PekkoSpec(
       config.withFallback(
         ConfigFactory.parseString("""
     pekko {
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/TypedActorRemoteDeploySpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/TypedActorRemoteDeploySpec.scala
index 3bf2dcd792..26e06e48d1 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/TypedActorRemoteDeploySpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/TypedActorRemoteDeploySpec.scala
@@ -13,7 +13,7 @@ import com.typesafe.config._
 
 import org.apache.pekko
 import pekko.actor.{ ActorSystem, Deploy, TypedActor, TypedProps }
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
 object TypedActorRemoteDeploySpec {
   val conf = ConfigFactory.parseString("""
@@ -39,7 +39,7 @@ object TypedActorRemoteDeploySpec {
 
 }
 
-class TypedActorRemoteDeploySpec extends AkkaSpec(conf) {
+class TypedActorRemoteDeploySpec extends PekkoSpec(conf) {
   val remoteName = "remote-sys"
   val remoteSystem = ActorSystem(remoteName, conf)
   val remoteAddress = RARP(remoteSystem).provider.getDefaultAddress
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/ArteryMultiNodeSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/ArteryMultiNodeSpec.scala
index 9ec76857d0..b2e144e14b 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/ArteryMultiNodeSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/ArteryMultiNodeSpec.scala
@@ -11,14 +11,14 @@ import org.apache.pekko
 import pekko.actor.{ ActorSystem, Address, BootstrapSetup, RootActorPath }
 import pekko.actor.setup.ActorSystemSetup
 import pekko.remote.RARP
-import pekko.testkit.{ AkkaSpec, SocketUtil }
+import pekko.testkit.{ PekkoSpec, SocketUtil }
 
 /**
  * Base class for remoting tests what needs to test interaction between a "local" actor system
- * which is always created (the usual AkkaSpec system), and multiple additional actor systems over artery
+ * which is always created (the usual PekkoSpec system), and multiple additional actor systems over artery
  */
 abstract class ArteryMultiNodeSpec(config: Config)
-    extends AkkaSpec(config.withFallback(ArterySpecSupport.defaultConfig)) {
+    extends PekkoSpec(config.withFallback(ArterySpecSupport.defaultConfig)) {
 
   def this() = this(ConfigFactory.empty())
   def this(extraConfig: String) = this(ConfigFactory.parseString(extraConfig))
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/DuplicateFlushSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/DuplicateFlushSpec.scala
index a8f3232fc2..5ebd4a43de 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/DuplicateFlushSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/DuplicateFlushSpec.scala
@@ -15,11 +15,11 @@ import pekko.stream.testkit.TestPublisher
 import pekko.stream.testkit.TestSubscriber
 import pekko.stream.testkit.scaladsl.TestSink
 import pekko.stream.testkit.scaladsl.TestSource
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.ImplicitSender
 import pekko.util.OptionVal
 
-class DuplicateFlushSpec extends AkkaSpec("""
+class DuplicateFlushSpec extends PekkoSpec("""
       pekko.stream.materializer.debug.fuzzing-mode = on
   """) with ImplicitSender {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/DuplicateHandshakeSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/DuplicateHandshakeSpec.scala
index 85f9717a91..c74c0a3cee 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/DuplicateHandshakeSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/DuplicateHandshakeSpec.scala
@@ -16,11 +16,11 @@ import pekko.stream.testkit.TestPublisher
 import pekko.stream.testkit.TestSubscriber
 import pekko.stream.testkit.scaladsl.TestSink
 import pekko.stream.testkit.scaladsl.TestSource
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.ImplicitSender
 import pekko.util.OptionVal
 
-class DuplicateHandshakeSpec extends AkkaSpec("""
+class DuplicateHandshakeSpec extends PekkoSpec("""
       pekko.stream.materializer.debug.fuzzing-mode = on
   """) with ImplicitSender {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/EnvelopeBufferSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/EnvelopeBufferSpec.scala
index 4eee50d728..264ab200b6 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/EnvelopeBufferSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/EnvelopeBufferSpec.scala
@@ -9,10 +9,10 @@ import org.apache.pekko
 import pekko.actor._
 import pekko.remote.artery.compress.{ CompressionTable, CompressionTestUtils, InboundCompressions }
 import pekko.serialization.Serialization
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.util.{ ByteString, OptionVal }
 
-class EnvelopeBufferSpec extends AkkaSpec {
+class EnvelopeBufferSpec extends PekkoSpec {
   import CompressionTestUtils._
 
   object TestCompressor extends InboundCompressions {
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/InboundControlJunctionSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/InboundControlJunctionSpec.scala
index b34ac2ef1b..782c219b2c 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/InboundControlJunctionSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/InboundControlJunctionSpec.scala
@@ -14,7 +14,7 @@ import pekko.remote.artery.InboundControlJunction.ControlMessageObserver
 import pekko.stream.scaladsl.Keep
 import pekko.stream.testkit.scaladsl.TestSink
 import pekko.stream.testkit.scaladsl.TestSource
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.ImplicitSender
 import pekko.testkit.TestProbe
 import pekko.util.OptionVal
@@ -28,7 +28,7 @@ object InboundControlJunctionSpec {
 }
 
 class InboundControlJunctionSpec
-    extends AkkaSpec("""
+    extends PekkoSpec("""
                    pekko.actor.serialization-bindings {
                      "org.apache.pekko.remote.artery.InboundControlJunctionSpec$TestControlMessage" = java
                    }
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/InboundHandshakeSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/InboundHandshakeSpec.scala
index d33c18216b..b328864066 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/InboundHandshakeSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/InboundHandshakeSpec.scala
@@ -19,7 +19,7 @@ import pekko.stream.testkit.TestPublisher
 import pekko.stream.testkit.TestSubscriber
 import pekko.stream.testkit.scaladsl.TestSink
 import pekko.stream.testkit.scaladsl.TestSource
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.ImplicitSender
 import pekko.testkit.TestProbe
 import pekko.util.OptionVal
@@ -30,7 +30,7 @@ object InboundHandshakeSpec {
   case object Control3 extends ControlMessage
 }
 
-class InboundHandshakeSpec extends AkkaSpec("""
+class InboundHandshakeSpec extends PekkoSpec("""
     pekko.stream.materializer.debug.fuzzing-mode = on
   """) with ImplicitSender {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/LruBoundedCacheSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/LruBoundedCacheSpec.scala
index f2e8f972dd..9f2d841d2b 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/LruBoundedCacheSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/LruBoundedCacheSpec.scala
@@ -9,11 +9,11 @@ import scala.util.Random
 import scala.annotation.nowarn
 
 import org.apache.pekko
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.util.Unsafe
 
 @nowarn
-class LruBoundedCacheSpec extends AkkaSpec {
+class LruBoundedCacheSpec extends PekkoSpec {
 
   class TestCache(_capacity: Int, threshold: Int, hashSeed: String = "")
       extends LruBoundedCache[String, String](_capacity, threshold) {
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/OutboundControlJunctionSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/OutboundControlJunctionSpec.scala
index 2444ce3e53..d7acce0464 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/OutboundControlJunctionSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/OutboundControlJunctionSpec.scala
@@ -10,7 +10,7 @@ import pekko.remote.UniqueAddress
 import pekko.stream.scaladsl.Keep
 import pekko.stream.testkit.scaladsl.TestSink
 import pekko.stream.testkit.scaladsl.TestSource
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.ImplicitSender
 import pekko.util.OptionVal
 
@@ -20,7 +20,7 @@ object OutboundControlJunctionSpec {
   case object Control3 extends ControlMessage
 }
 
-class OutboundControlJunctionSpec extends AkkaSpec("""
+class OutboundControlJunctionSpec extends PekkoSpec("""
     pekko.stream.materializer.debug.fuzzing-mode = on
   """) with ImplicitSender {
   import OutboundControlJunctionSpec._
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/OutboundHandshakeSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/OutboundHandshakeSpec.scala
index b6b2013a9f..4ee7915d73 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/OutboundHandshakeSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/OutboundHandshakeSpec.scala
@@ -16,11 +16,11 @@ import pekko.stream.testkit.TestPublisher
 import pekko.stream.testkit.TestSubscriber
 import pekko.stream.testkit.scaladsl.TestSink
 import pekko.stream.testkit.scaladsl.TestSource
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.ImplicitSender
 import pekko.util.OptionVal
 
-class OutboundHandshakeSpec extends AkkaSpec("""
+class OutboundHandshakeSpec extends PekkoSpec("""
     pekko.stream.materializer.debug.fuzzing-mode = on
   """) with ImplicitSender {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemoteDeployerSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemoteDeployerSpec.scala
index 0cf5320212..d059804175 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemoteDeployerSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemoteDeployerSpec.scala
@@ -31,7 +31,7 @@ object RemoteDeployerSpec {
 
 }
 
-class RemoteDeployerSpec extends AkkaSpec(RemoteDeployerSpec.deployerConf) {
+class RemoteDeployerSpec extends PekkoSpec(RemoteDeployerSpec.deployerConf) {
 
   "A RemoteDeployer" must {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemoteInstrumentsSerializationSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemoteInstrumentsSerializationSpec.scala
index 69aef2d7ab..ce04827499 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemoteInstrumentsSerializationSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemoteInstrumentsSerializationSpec.scala
@@ -10,13 +10,13 @@ import scala.concurrent.duration._
 import org.apache.pekko
 import pekko.actor.{ ActorRef, ActorSystem, ExtendedActorSystem, InternalActorRef }
 import pekko.event._
-import pekko.testkit.{ AkkaSpec, EventFilter, TestProbe }
+import pekko.testkit.{ EventFilter, PekkoSpec, TestProbe }
 import pekko.testkit.TestEvent.Mute
 import pekko.util.{ unused, OptionVal }
 
 import java.nio.ByteOrder
 
-class RemoteInstrumentsSerializationSpec extends AkkaSpec("pekko.loglevel = DEBUG") {
+class RemoteInstrumentsSerializationSpec extends PekkoSpec("pekko.loglevel = DEBUG") {
   import RemoteInstrumentsSerializationSpec._
 
   def remoteInstruments(instruments: RemoteInstrument*): RemoteInstruments = {
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemoteRouterSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemoteRouterSpec.scala
index 73ec7c9140..6ff4525505 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemoteRouterSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemoteRouterSpec.scala
@@ -26,7 +26,7 @@ object RemoteRouterSpec {
 }
 
 class RemoteRouterSpec
-    extends AkkaSpec(ConfigFactory.parseString("""
+    extends PekkoSpec(ConfigFactory.parseString("""
     pekko.remote.use-unsafe-remote-features-outside-cluster = on
     pekko.actor.deployment {
       /remote-override {
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemotingFlightRecorderSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemotingFlightRecorderSpec.scala
index 89755407f6..f6cdcbeaf4 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemotingFlightRecorderSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RemotingFlightRecorderSpec.scala
@@ -7,10 +7,10 @@ package org.apache.pekko.remote.artery
 import org.scalatest.matchers.should.Matchers
 
 import org.apache.pekko
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.util.JavaVersion
 
-class RemotingFlightRecorderSpec extends AkkaSpec with Matchers {
+class RemotingFlightRecorderSpec extends PekkoSpec with Matchers {
 
   "The RemotingFlightRecorder" must {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RollingEventLogSimulationSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RollingEventLogSimulationSpec.scala
index 2706b8449f..dedbc6dc3a 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RollingEventLogSimulationSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/RollingEventLogSimulationSpec.scala
@@ -8,7 +8,7 @@ import scala.annotation.tailrec
 import scala.util.Random
 import scala.util.control.NonFatal
 
-import org.apache.pekko.testkit.AkkaSpec
+import org.apache.pekko.testkit.PekkoSpec
 
 /*
  * This test is a simulation of the actual concurrent rolling log implemented in SnapshottableRollingEventLog. It
@@ -19,7 +19,7 @@ import org.apache.pekko.testkit.AkkaSpec
  * results when debugging using the logs. This simulation tries to uncover many race scenarios by simulating the
  * algorithm down to the individual byte write level.
  */
-class RollingEventLogSimulationSpec extends AkkaSpec {
+class RollingEventLogSimulationSpec extends PekkoSpec {
 
   val Committed: Byte = 0.toByte
   val Dirty: Byte = 1.toByte
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/SendQueueSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/SendQueueSpec.scala
index da0556a0c1..5ce8426ed4 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/SendQueueSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/SendQueueSpec.scala
@@ -17,7 +17,7 @@ import pekko.stream.scaladsl.Keep
 import pekko.stream.scaladsl.Source
 import pekko.stream.testkit.TestSubscriber
 import pekko.stream.testkit.scaladsl.TestSink
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.ImplicitSender
 
 object SendQueueSpec {
@@ -50,7 +50,7 @@ object SendQueueSpec {
   }
 }
 
-class SendQueueSpec extends AkkaSpec("""
+class SendQueueSpec extends PekkoSpec("""
     pekko.stream.materializer.debug.fuzzing-mode = on
     pekko.stream.secret-test-fuzzing-warning-disable = yep
   """) with ImplicitSender {
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/SystemMessageAckerSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/SystemMessageAckerSpec.scala
index 33f25c46aa..ba617ad162 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/SystemMessageAckerSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/SystemMessageAckerSpec.scala
@@ -13,12 +13,12 @@ import pekko.stream.testkit.TestPublisher
 import pekko.stream.testkit.TestSubscriber
 import pekko.stream.testkit.scaladsl.TestSink
 import pekko.stream.testkit.scaladsl.TestSource
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.ImplicitSender
 import pekko.testkit.TestProbe
 import pekko.util.OptionVal
 
-class SystemMessageAckerSpec extends AkkaSpec("""
+class SystemMessageAckerSpec extends PekkoSpec("""
     pekko.stream.materializer.debug.fuzzing-mode = on
   """) with ImplicitSender {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/aeron/AeronSinkSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/aeron/AeronSinkSpec.scala
index d0a2d6ea69..d44f5057c2 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/aeron/AeronSinkSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/aeron/AeronSinkSpec.scala
@@ -20,11 +20,11 @@ import pekko.actor.ExtendedActorSystem
 import pekko.remote.artery.aeron.AeronSink.GaveUpMessageException
 import pekko.stream.scaladsl.Sink
 import pekko.stream.scaladsl.Source
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.ImplicitSender
 import pekko.testkit.SocketUtil
 
-class AeronSinkSpec extends AkkaSpec("""
+class AeronSinkSpec extends PekkoSpec("""
     pekko.stream.materializer.debug.fuzzing-mode = on
   """) with ImplicitSender {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/compress/CompressionTableSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/compress/CompressionTableSpec.scala
index 0df5783d3a..46f70ebf61 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/compress/CompressionTableSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/compress/CompressionTableSpec.scala
@@ -4,9 +4,9 @@
 
 package org.apache.pekko.remote.artery.compress
 
-import org.apache.pekko.testkit.AkkaSpec
+import org.apache.pekko.testkit.PekkoSpec
 
-class CompressionTableSpec extends AkkaSpec {
+class CompressionTableSpec extends PekkoSpec {
 
   "CompressionTable" must {
     "should invert" in {
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/compress/OutboundCompressionSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/compress/OutboundCompressionSpec.scala
index 19979e08a8..513b37c937 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/compress/OutboundCompressionSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/compress/OutboundCompressionSpec.scala
@@ -6,9 +6,9 @@ package org.apache.pekko.remote.artery.compress
 
 import org.apache.pekko
 import pekko.actor._
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
-class OutboundCompressionSpec extends AkkaSpec {
+class OutboundCompressionSpec extends PekkoSpec {
   import CompressionTestUtils._
 
   "Outbound ActorRef compression" must {
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/tcp/SecureRandomFactorySpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/tcp/SecureRandomFactorySpec.scala
index f6d1ec9909..18ce8702f8 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/tcp/SecureRandomFactorySpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/tcp/SecureRandomFactorySpec.scala
@@ -11,7 +11,7 @@ import java.util.zip.GZIPOutputStream
 
 import org.apache.pekko
 import pekko.event.NoMarkerLogging
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
 class SecureRandomFactorySHA1Spec extends SecureRandomFactorySpec("SHA1PRNG")
 class SecureRandomFactoryNativePRNGSpec extends SecureRandomFactorySpec("NativePRNG")
@@ -19,7 +19,7 @@ class SecureRandomFactoryJVMChoiceSpec extends SecureRandomFactorySpec("SecureRa
 class SecureRandomFactoryBlankSpec extends SecureRandomFactorySpec("")
 class SecureRandomFactoryInvalidPRNGSpec extends SecureRandomFactorySpec("InvalidPRNG")
 
-abstract class SecureRandomFactorySpec(alg: String) extends AkkaSpec {
+abstract class SecureRandomFactorySpec(alg: String) extends PekkoSpec {
   var prng: SecureRandom = null
 
   def isSupported: Boolean = {
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/tcp/TcpFramingSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/tcp/TcpFramingSpec.scala
index fba2369690..55c24ef6ee 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/artery/tcp/TcpFramingSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/artery/tcp/TcpFramingSpec.scala
@@ -12,11 +12,11 @@ import pekko.stream.scaladsl.Flow
 import pekko.stream.scaladsl.Framing.FramingException
 import pekko.stream.scaladsl.Sink
 import pekko.stream.scaladsl.Source
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.ImplicitSender
 import pekko.util.ByteString
 
-class TcpFramingSpec extends AkkaSpec("""
+class TcpFramingSpec extends PekkoSpec("""
     pekko.stream.materializer.debug.fuzzing-mode = on
   """) with ImplicitSender {
   import TcpFraming.encodeFrameHeader
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/ActorsLeakSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/ActorsLeakSpec.scala
index 7ca9358edd..1015c84aa8 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/ActorsLeakSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/ActorsLeakSpec.scala
@@ -70,7 +70,7 @@ object ActorsLeakSpec {
 
 }
 
-class ActorsLeakSpec extends AkkaSpec(ActorsLeakSpec.config) with ImplicitSender {
+class ActorsLeakSpec extends PekkoSpec(ActorsLeakSpec.config) with ImplicitSender {
   import ActorsLeakSpec._
 
   "Remoting" must {
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemoteDeathWatchSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemoteDeathWatchSpec.scala
index 121931c624..9b3264ad52 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemoteDeathWatchSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemoteDeathWatchSpec.scala
@@ -17,7 +17,7 @@ import pekko.testkit.{ SocketUtil, _ }
 
 @nowarn // classic deprecated
 class RemoteDeathWatchSpec
-    extends AkkaSpec(ConfigFactory.parseString("""
+    extends PekkoSpec(ConfigFactory.parseString("""
 pekko {
     actor {
         provider = remote
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemoteDeploymentAllowListSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemoteDeploymentAllowListSpec.scala
index e910f4f025..d577a2b01d 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemoteDeploymentAllowListSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemoteDeploymentAllowListSpec.scala
@@ -107,7 +107,7 @@ object RemoteDeploymentAllowListSpec {
 
 @nowarn("msg=deprecated")
 class RemoteDeploymentAllowListSpec
-    extends AkkaSpec(RemoteDeploymentAllowListSpec.cfg)
+    extends PekkoSpec(RemoteDeploymentAllowListSpec.cfg)
     with ImplicitSender
     with DefaultTimeout {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemoteWatcherSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemoteWatcherSpec.scala
index bc8f68b878..e8b64872e7 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemoteWatcherSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemoteWatcherSpec.scala
@@ -70,7 +70,7 @@ object RemoteWatcherSpec {
 }
 
 @nowarn("msg=deprecated")
-class RemoteWatcherSpec extends AkkaSpec("""
+class RemoteWatcherSpec extends PekkoSpec("""
      pekko {
        loglevel = INFO
        log-dead-letters-during-shutdown = false
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemotingSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemotingSpec.scala
index 01acc892d9..5a297fc19e 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemotingSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/RemotingSpec.scala
@@ -136,7 +136,7 @@ object RemotingSpec {
 }
 
 @nowarn
-class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with DefaultTimeout {
+class RemotingSpec extends PekkoSpec(RemotingSpec.cfg) with ImplicitSender with DefaultTimeout {
 
   import RemotingSpec._
 
@@ -658,8 +658,8 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
         }
 
         val handshakePacket =
-          AkkaPduProtobufCodec.constructAssociate(HandshakeInfo(rawRemoteAddress, uid = 0))
-        val brokenPacket = AkkaPduProtobufCodec.constructPayload(ByteString(0, 1, 2, 3, 4, 5, 6))
+          PekkoPduProtobufCodec$.constructAssociate(HandshakeInfo(rawRemoteAddress, uid = 0))
+        val brokenPacket = PekkoPduProtobufCodec$.constructPayload(ByteString(0, 1, 2, 3, 4, 5, 6))
 
         // Finish the inbound handshake so now it is handed up to Remoting
         inboundHandle.write(handshakePacket)
@@ -740,7 +740,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
         }
 
         val handshakePacket =
-          AkkaPduProtobufCodec.constructAssociate(HandshakeInfo(rawRemoteAddress, uid = remoteUID))
+          PekkoPduProtobufCodec$.constructAssociate(HandshakeInfo(rawRemoteAddress, uid = remoteUID))
 
         // Finish the inbound handshake so now it is handed up to Remoting
         inboundHandle.write(handshakePacket)
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/UntrustedSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/UntrustedSpec.scala
index 83ecd4e5a5..fd1751524c 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/UntrustedSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/UntrustedSpec.scala
@@ -22,7 +22,7 @@ import pekko.actor.Props
 import pekko.actor.RootActorPath
 import pekko.actor.Terminated
 import pekko.event.Logging
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.EventFilter
 import pekko.testkit.ImplicitSender
 import pekko.testkit.JavaSerializable
@@ -63,7 +63,7 @@ object UntrustedSpec {
 
 }
 
-class UntrustedSpec extends AkkaSpec("""
+class UntrustedSpec extends PekkoSpec("""
 pekko.loglevel = DEBUG
 pekko.actor.provider = remote
 pekko.remote.artery.enabled = off
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/GenericTransportSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/GenericTransportSpec.scala
index 7ad2ce9525..4c4592d8e3 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/GenericTransportSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/GenericTransportSpec.scala
@@ -13,14 +13,14 @@ import pekko.actor.{ Address, ExtendedActorSystem }
 import pekko.remote.RemoteActorRefProvider
 import pekko.remote.transport.{ AssociationRegistry => _, _ }
 import pekko.remote.transport.AssociationHandle.{ ActorHandleEventListener, Disassociated, InboundPayload }
-import pekko.remote.transport.TestTransport.{ AssociateAttempt, DisassociateAttempt, ListenAttempt, WriteAttempt, _ }
+import pekko.remote.transport.TestTransport._
 import pekko.remote.transport.Transport._
-import pekko.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender }
+import pekko.testkit.{ DefaultTimeout, ImplicitSender, PekkoSpec }
 import pekko.util.ByteString
 
 @nowarn("msg=deprecated")
 abstract class GenericTransportSpec(withAkkaProtocol: Boolean = false)
-    extends AkkaSpec("""
+    extends PekkoSpec("""
          pekko.remote.artery.enabled = false
          pekko.actor.provider = remote
          # test is using Java serialization and not priority to rewrite
@@ -44,11 +44,11 @@ abstract class GenericTransportSpec(withAkkaProtocol: Boolean = false)
   def wrapTransport(transport: Transport): Transport =
     if (withAkkaProtocol) {
       val provider = system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider]
-      new AkkaProtocolTransport(
+      new PekkoProtocolTransport(
         transport,
         system,
-        new AkkaProtocolSettings(provider.remoteSettings.config),
-        AkkaPduProtobufCodec)
+        new PekkoProtocolSettings(provider.remoteSettings.config),
+        PekkoPduProtobufCodec$)
     } else transport
 
   def newTransportA(registry: AssociationRegistry): Transport =
@@ -128,7 +128,7 @@ abstract class GenericTransportSpec(withAkkaProtocol: Boolean = false)
       handleB.readHandlerPromise.success(ActorHandleEventListener(self))
 
       val payload = ByteString("PDU")
-      val pdu = if (withAkkaProtocol) AkkaPduProtobufCodec.constructPayload(payload) else payload
+      val pdu = if (withAkkaProtocol) PekkoPduProtobufCodec$.constructPayload(payload) else payload
 
       awaitCond(registry.existsAssociation(addressATest, addressBTest))
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/AkkaProtocolSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/PekkoProtocolSpec.scala
similarity index 93%
rename from akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/AkkaProtocolSpec.scala
rename to akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/PekkoProtocolSpec.scala
index b9e169ed77..eac71427dc 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/AkkaProtocolSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/PekkoProtocolSpec.scala
@@ -16,9 +16,9 @@ import org.apache.pekko
 import pekko.actor.Address
 import pekko.protobufv3.internal.{ ByteString => PByteString }
 import pekko.remote.{ FailureDetector, WireFormats }
-import pekko.remote.classic.transport.AkkaProtocolSpec.TestFailureDetector
+import pekko.remote.classic.transport.PekkoProtocolSpec.TestFailureDetector
 import pekko.remote.transport.{ AssociationRegistry => _, _ }
-import pekko.remote.transport.AkkaPduCodec.{ Associate, Disassociate, Heartbeat }
+import pekko.remote.transport.PekkoPduCodec.{ Associate, Disassociate, Heartbeat }
 import pekko.remote.transport.AssociationHandle.{
   ActorHandleEventListener,
   DisassociateInfo,
@@ -28,10 +28,10 @@ import pekko.remote.transport.AssociationHandle.{
 import pekko.remote.transport.ProtocolStateActor
 import pekko.remote.transport.TestTransport._
 import pekko.remote.transport.Transport._
-import pekko.testkit.{ AkkaSpec, ImplicitSender }
+import pekko.testkit.{ ImplicitSender, PekkoSpec }
 import pekko.util.{ ByteString, OptionVal }
 
-object AkkaProtocolSpec {
+object PekkoProtocolSpec {
 
   class TestFailureDetector extends FailureDetector {
     @volatile var isAvailable: Boolean = true
@@ -46,7 +46,7 @@ object AkkaProtocolSpec {
 }
 
 @nowarn("msg=deprecated")
-class AkkaProtocolSpec extends AkkaSpec("""pekko.actor.provider = remote """) with ImplicitSender {
+class PekkoProtocolSpec extends PekkoSpec("""pekko.actor.provider = remote """) with ImplicitSender {
 
   val conf = ConfigFactory.parseString("""
       pekko.remote {
@@ -79,7 +79,7 @@ class AkkaProtocolSpec extends AkkaSpec("""pekko.actor.provider = remote """) wi
   val remoteAddress = Address("test", "testsystem2", "testhost2", 1234)
   val remoteAkkaAddress = Address("akka.test", "testsystem2", "testhost2", 1234)
 
-  val codec = AkkaPduProtobufCodec
+  val codec = PekkoPduProtobufCodec$
 
   val testMsg =
     WireFormats.SerializedMessage.newBuilder().setSerializerId(0).setMessage(PByteString.copyFromUtf8("foo")).build
@@ -150,7 +150,7 @@ class AkkaProtocolSpec extends AkkaSpec("""pekko.actor.provider = remote """) wi
           HandshakeInfo(origin = localAddress, uid = 42),
           handle,
           ActorAssociationEventListener(testActor),
-          new AkkaProtocolSettings(conf),
+          new PekkoProtocolSettings(conf),
           codec,
           failureDetector))
 
@@ -165,7 +165,7 @@ class AkkaProtocolSpec extends AkkaSpec("""pekko.actor.provider = remote """) wi
           HandshakeInfo(origin = localAddress, uid = 42),
           handle,
           ActorAssociationEventListener(testActor),
-          new AkkaProtocolSettings(conf),
+          new PekkoProtocolSettings(conf),
           codec,
           failureDetector))
 
@@ -174,7 +174,7 @@ class AkkaProtocolSpec extends AkkaSpec("""pekko.actor.provider = remote """) wi
       awaitCond(failureDetector.called)
 
       val wrappedHandle = expectMsgPF() {
-        case InboundAssociation(h: AkkaProtocolHandle) =>
+        case InboundAssociation(h: PekkoProtocolHandle) =>
           h.handshakeInfo.uid should ===(33)
           h
       }
@@ -201,7 +201,7 @@ class AkkaProtocolSpec extends AkkaSpec("""pekko.actor.provider = remote """) wi
           HandshakeInfo(origin = localAddress, uid = 42),
           handle,
           ActorAssociationEventListener(testActor),
-          new AkkaProtocolSettings(conf),
+          new PekkoProtocolSettings(conf),
           codec,
           failureDetector))
 
@@ -229,7 +229,7 @@ class AkkaProtocolSpec extends AkkaSpec("""pekko.actor.provider = remote """) wi
           remoteAddress,
           statusPromise,
           transport,
-          new AkkaProtocolSettings(conf),
+          new PekkoProtocolSettings(conf),
           codec,
           failureDetector,
           refuseUid = None))
@@ -246,7 +246,7 @@ class AkkaProtocolSpec extends AkkaSpec("""pekko.actor.provider = remote """) wi
       reader ! testAssociate(33)
 
       Await.result(statusPromise.future, 3.seconds) match {
-        case h: AkkaProtocolHandle =>
+        case h: PekkoProtocolHandle =>
           h.remoteAddress should ===(remoteAkkaAddress)
           h.localAddress should ===(localAkkaAddress)
           h.handshakeInfo.uid should ===(33)
@@ -268,7 +268,7 @@ class AkkaProtocolSpec extends AkkaSpec("""pekko.actor.provider = remote """) wi
           remoteAddress,
           statusPromise,
           transport,
-          new AkkaProtocolSettings(conf),
+          new PekkoProtocolSettings(conf),
           codec,
           failureDetector,
           refuseUid = None))
@@ -303,7 +303,7 @@ class AkkaProtocolSpec extends AkkaSpec("""pekko.actor.provider = remote """) wi
           remoteAddress,
           statusPromise,
           transport,
-          new AkkaProtocolSettings(conf),
+          new PekkoProtocolSettings(conf),
           codec,
           failureDetector,
           refuseUid = None))
@@ -338,7 +338,7 @@ class AkkaProtocolSpec extends AkkaSpec("""pekko.actor.provider = remote """) wi
           remoteAddress,
           statusPromise,
           transport,
-          new AkkaProtocolSettings(conf),
+          new PekkoProtocolSettings(conf),
           codec,
           failureDetector,
           refuseUid = None))
@@ -376,7 +376,7 @@ class AkkaProtocolSpec extends AkkaSpec("""pekko.actor.provider = remote """) wi
           remoteAddress,
           statusPromise,
           transport,
-          new AkkaProtocolSettings(conf),
+          new PekkoProtocolSettings(conf),
           codec,
           failureDetector,
           refuseUid = None))
@@ -416,7 +416,7 @@ class AkkaProtocolSpec extends AkkaSpec("""pekko.actor.provider = remote """) wi
           remoteAddress,
           statusPromise,
           transport,
-          new AkkaProtocolSettings(conf2),
+          new PekkoProtocolSettings(conf2),
           codec,
           failureDetector,
           refuseUid = None))
@@ -439,7 +439,7 @@ class AkkaProtocolSpec extends AkkaSpec("""pekko.actor.provider = remote """) wi
           HandshakeInfo(origin = localAddress, uid = 42),
           handle,
           ActorAssociationEventListener(testActor),
-          new AkkaProtocolSettings(conf2),
+          new PekkoProtocolSettings(conf2),
           codec,
           failureDetector))
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/AkkaProtocolStressTest.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/PekkoProtocolStressTest.scala
similarity index 93%
rename from akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/AkkaProtocolStressTest.scala
rename to akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/PekkoProtocolStressTest.scala
index f39d60eb5e..ca705713eb 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/AkkaProtocolStressTest.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/PekkoProtocolStressTest.scala
@@ -12,11 +12,11 @@ import com.typesafe.config.{ Config, ConfigFactory }
 import org.apache.pekko
 import pekko.actor._
 import pekko.remote.{ EndpointException, RARP }
-import pekko.remote.classic.transport.AkkaProtocolStressTest._
+import pekko.remote.classic.transport.PekkoProtocolStressTest._
 import pekko.remote.transport.FailureInjectorTransportAdapter.{ Drop, One }
-import pekko.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender, TimingTest, _ }
+import pekko.testkit._
 
-object AkkaProtocolStressTest {
+object PekkoProtocolStressTest {
   val configA: Config =
     ConfigFactory.parseString("""
     pekko {
@@ -93,7 +93,7 @@ object AkkaProtocolStressTest {
 
 }
 
-class AkkaProtocolStressTest extends AkkaSpec(configA) with ImplicitSender with DefaultTimeout {
+class PekkoProtocolStressTest extends PekkoSpec(configA) with ImplicitSender with DefaultTimeout {
 
   val systemB = ActorSystem("systemB", system.settings.config)
   val remote = systemB.actorOf(Props(new Actor {
@@ -109,7 +109,7 @@ class AkkaProtocolStressTest extends AkkaSpec(configA) with ImplicitSender with
     expectMsgType[ActorIdentity].ref.get
   }
 
-  "AkkaProtocolTransport" must {
+  "PekkoProtocolTransport" must {
     "guarantee at-most-once delivery and message ordering despite packet loss" taggedAs TimingTest in {
       system.eventStream.publish(TestEvent.Mute(DeadLettersFilter[Any]))
       systemB.eventStream.publish(TestEvent.Mute(DeadLettersFilter[Any]))
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/SwitchableLoggedBehaviorSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/SwitchableLoggedBehaviorSpec.scala
index 5e4a213e0c..c1fb672a92 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/SwitchableLoggedBehaviorSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/SwitchableLoggedBehaviorSpec.scala
@@ -9,15 +9,15 @@ import scala.util.Failure
 import scala.util.control.NoStackTrace
 
 import org.apache.pekko
-import pekko.AkkaException
+import pekko.PekkoException
 import pekko.remote.transport.TestTransport.SwitchableLoggedBehavior
-import pekko.testkit.{ AkkaSpec, DefaultTimeout }
+import pekko.testkit.{ DefaultTimeout, PekkoSpec }
 
 object SwitchableLoggedBehaviorSpec {
-  object TestException extends AkkaException("Test exception") with NoStackTrace
+  object TestException extends PekkoException("Test exception") with NoStackTrace
 }
 
-class SwitchableLoggedBehaviorSpec extends AkkaSpec with DefaultTimeout {
+class SwitchableLoggedBehaviorSpec extends PekkoSpec with DefaultTimeout {
   import pekko.remote.classic.transport.SwitchableLoggedBehaviorSpec._
 
   private def defaultBehavior = new SwitchableLoggedBehavior[Unit, Int](_ => Future.successful(3), _ => ())
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/SystemMessageDeliveryStressTest.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/SystemMessageDeliveryStressTest.scala
index 87c7f81981..e1c4a75356 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/SystemMessageDeliveryStressTest.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/SystemMessageDeliveryStressTest.scala
@@ -11,13 +11,13 @@ import scala.annotation.nowarn
 import com.typesafe.config.{ Config, ConfigFactory }
 
 import org.apache.pekko
-import pekko.actor.{ Actor, ActorRef, ActorSystem, ExtendedActorSystem, Props, RootActorPath, _ }
+import pekko.actor._
 import pekko.dispatch.sysmsg.{ Failed, SystemMessage }
 import pekko.remote.{ EndpointException, QuarantinedEvent, RARP }
 import pekko.remote.transport.AssociationHandle
 import pekko.remote.transport.FailureInjectorTransportAdapter.{ Drop, One }
 import pekko.remote.transport.ThrottlerTransportAdapter._
-import pekko.testkit.{ AkkaSpec, DefaultTimeout, EventFilter, ImplicitSender, TestEvent, TimingTest, _ }
+import pekko.testkit._
 
 object SystemMessageDeliveryStressTest {
   val msgCount = 5000
@@ -105,7 +105,7 @@ object SystemMessageDeliveryStressTest {
 
 @nowarn("msg=deprecated")
 abstract class SystemMessageDeliveryStressTest(msg: String, cfg: String)
-    extends AkkaSpec(ConfigFactory.parseString(cfg).withFallback(SystemMessageDeliveryStressTest.baseConfig))
+    extends PekkoSpec(ConfigFactory.parseString(cfg).withFallback(SystemMessageDeliveryStressTest.baseConfig))
     with ImplicitSender
     with DefaultTimeout {
   import SystemMessageDeliveryStressTest._
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/TestTransportSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/TestTransportSpec.scala
index 97688aabe5..4e82500b76 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/TestTransportSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/TestTransportSpec.scala
@@ -18,7 +18,7 @@ import pekko.testkit._
 import pekko.util.ByteString
 
 @nowarn("msg=deprecated")
-class TestTransportSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
+class TestTransportSpec extends PekkoSpec with DefaultTimeout with ImplicitSender {
 
   val addressA: Address = Address("test", "testsytemA", "testhostA", 4321)
   val addressB: Address = Address("test", "testsytemB", "testhostB", 5432)
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/ThrottleModeSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/ThrottleModeSpec.scala
index 314e95c1c1..4820192819 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/ThrottleModeSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/ThrottleModeSpec.scala
@@ -8,9 +8,9 @@ import java.util.concurrent.TimeUnit
 
 import org.apache.pekko
 import pekko.remote.transport.ThrottlerTransportAdapter.{ TokenBucket, Unthrottled }
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
-class ThrottleModeSpec extends AkkaSpec {
+class ThrottleModeSpec extends PekkoSpec {
 
   val halfSecond: Long = TimeUnit.MILLISECONDS.toNanos(500)
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/ThrottlerTransportAdapterSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/ThrottlerTransportAdapterSpec.scala
index cb9f3ebcbf..2971e04884 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/ThrottlerTransportAdapterSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/ThrottlerTransportAdapterSpec.scala
@@ -16,7 +16,7 @@ import pekko.remote.{ EndpointException, RemoteActorRefProvider }
 import pekko.remote.classic.transport.ThrottlerTransportAdapterSpec._
 import pekko.remote.transport.{ TestTransport, ThrottlerTransportAdapter }
 import pekko.remote.transport.ThrottlerTransportAdapter._
-import pekko.testkit.{ AkkaSpec, DefaultTimeout, EventFilter, ImplicitSender, TestEvent, TimingTest }
+import pekko.testkit.{ DefaultTimeout, EventFilter, ImplicitSender, PekkoSpec, TestEvent, TimingTest }
 
 object ThrottlerTransportAdapterSpec {
   val configA: Config =
@@ -76,7 +76,7 @@ object ThrottlerTransportAdapterSpec {
 }
 
 @nowarn("msg=deprecated")
-class ThrottlerTransportAdapterSpec extends AkkaSpec(configA) with ImplicitSender with DefaultTimeout {
+class ThrottlerTransportAdapterSpec extends PekkoSpec(configA) with ImplicitSender with DefaultTimeout {
 
   val systemB = ActorSystem("systemB", system.settings.config)
   val remote = systemB.actorOf(Props[Echo](), "echo")
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/netty/NettyTransportSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/netty/NettyTransportSpec.scala
index 2c5fa7f804..e1e5e27700 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/netty/NettyTransportSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/classic/transport/netty/NettyTransportSpec.scala
@@ -35,7 +35,7 @@ object NettyTransportSpec {
       Address(protocol, system.name, address.getAddress.getHostAddress, address.getPort)
   }
 
-  implicit class RichAkkaAddress(address: Address) {
+  implicit class RichPekkoAddress(address: Address) {
     def withProtocol(protocol: String) =
       address.copy(protocol = protocol)
   }
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/AllowJavaSerializationOffSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/AllowJavaSerializationOffSpec.scala
index 9d0e0e95ea..50535d2b00 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/AllowJavaSerializationOffSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/AllowJavaSerializationOffSpec.scala
@@ -15,7 +15,7 @@ import pekko.actor.BootstrapSetup
 import pekko.actor.ExtendedActorSystem
 import pekko.actor.setup.ActorSystemSetup
 import pekko.serialization._
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.TestKit
 
 class ConfigurationDummy
@@ -63,7 +63,7 @@ object AllowJavaSerializationOffSpec {
 }
 
 class AllowJavaSerializationOffSpec
-    extends AkkaSpec(ActorSystem("AllowJavaSerializationOffSpec", AllowJavaSerializationOffSpec.actorSystemSettings)) {
+    extends PekkoSpec(ActorSystem("AllowJavaSerializationOffSpec", AllowJavaSerializationOffSpec.actorSystemSettings)) {
 
   import AllowJavaSerializationOffSpec._
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/ArteryMessageSerializerSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/ArteryMessageSerializerSpec.scala
index 1b599f0734..4cb4baf78b 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/ArteryMessageSerializerSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/ArteryMessageSerializerSpec.scala
@@ -21,9 +21,9 @@ import pekko.remote.artery.compress.CompressionProtocol.{
 }
 import pekko.remote.artery.compress.CompressionTable
 import pekko.serialization.SerializationExtension
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
-class ArteryMessageSerializerSpec extends AkkaSpec {
+class ArteryMessageSerializerSpec extends PekkoSpec {
   "ArteryMessageSerializer" must {
     val actorA = system.actorOf(Props.empty)
     val actorB = system.actorOf(Props.empty)
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/DaemonMsgCreateSerializerAllowJavaSerializationSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/DaemonMsgCreateSerializerAllowJavaSerializationSpec.scala
index c7c9fdb946..7e218ec7dc 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/DaemonMsgCreateSerializerAllowJavaSerializationSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/DaemonMsgCreateSerializerAllowJavaSerializationSpec.scala
@@ -21,7 +21,7 @@ import pekko.routing.FromConfig
 import pekko.routing.RoundRobinPool
 import pekko.serialization.Serialization
 import pekko.serialization.SerializationExtension
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.JavaSerializable
 import pekko.util.unused
 
@@ -39,7 +39,7 @@ object DaemonMsgCreateSerializerAllowJavaSerializationSpec {
 
 case class DummyParameter(val inner: String) extends JavaSerializable
 
-private[pekko] trait SerializationVerification { self: AkkaSpec =>
+private[pekko] trait SerializationVerification { self: PekkoSpec =>
 
   def ser: Serialization
 
@@ -67,7 +67,7 @@ private[pekko] trait SerializationVerification { self: AkkaSpec =>
 }
 
 class DaemonMsgCreateSerializerAllowJavaSerializationSpec
-    extends AkkaSpec("""
+    extends PekkoSpec("""
   # test is verifying Java serialization  
   pekko.actor.allow-java-serialization = on
   pekko.actor.warn-about-java-serializer-usage = off
@@ -128,7 +128,7 @@ class DaemonMsgCreateSerializerAllowJavaSerializationSpec
   }
 }
 
-class DaemonMsgCreateSerializerNoJavaSerializationSpec extends AkkaSpec("""
+class DaemonMsgCreateSerializerNoJavaSerializationSpec extends PekkoSpec("""
    pekko.actor.allow-java-serialization=off
   """) with SerializationVerification {
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/MessageContainerSerializerSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/MessageContainerSerializerSpec.scala
index 46e18d00d9..4c238e1a31 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/MessageContainerSerializerSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/MessageContainerSerializerSpec.scala
@@ -11,10 +11,10 @@ import pekko.actor.SelectChildPattern
 import pekko.actor.SelectParent
 import pekko.remote.DaemonMsgCreate
 import pekko.serialization.SerializationExtension
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.TestActors
 
-class MessageContainerSerializerSpec extends AkkaSpec {
+class MessageContainerSerializerSpec extends PekkoSpec {
 
   val ser = SerializationExtension(system)
 
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/MiscMessageSerializerSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/MiscMessageSerializerSpec.scala
index 0d02186647..e33c06724a 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/MiscMessageSerializerSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/MiscMessageSerializerSpec.scala
@@ -22,7 +22,7 @@ import pekko.remote.{ RemoteScope, RemoteWatcher }
 import pekko.remote.routing.RemoteRouterConfig
 import pekko.routing._
 import pekko.serialization.SerializationExtension
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.JavaSerializable
 
 object MiscMessageSerializerSpec {
@@ -35,7 +35,7 @@ object MiscMessageSerializerSpec {
     }
     """
 
-  val testConfig = ConfigFactory.parseString(serializationTestOverrides).withFallback(AkkaSpec.testConf)
+  val testConfig = ConfigFactory.parseString(serializationTestOverrides).withFallback(PekkoSpec.testConf)
 
   class TestException(msg: String, cause: Throwable) extends RuntimeException(msg, cause) {
     def this(msg: String) = this(msg, null)
@@ -73,7 +73,7 @@ object MiscMessageSerializerSpec {
 
 }
 
-class MiscMessageSerializerSpec extends AkkaSpec(MiscMessageSerializerSpec.testConfig) {
+class MiscMessageSerializerSpec extends PekkoSpec(MiscMessageSerializerSpec.testConfig) {
   import MiscMessageSerializerSpec._
 
   val ref = system.actorOf(Props.empty, "hello")
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/PrimitivesSerializationSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/PrimitivesSerializationSpec.scala
index 84be615ee1..06a9c78944 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/PrimitivesSerializationSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/PrimitivesSerializationSpec.scala
@@ -14,7 +14,7 @@ import pekko.serialization.BaseSerializer
 import pekko.serialization.ByteBufferSerializer
 import pekko.serialization.SerializationExtension
 import pekko.serialization.Serializer
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.util.ByteString
 
 import java.io.NotSerializableException
@@ -22,11 +22,11 @@ import java.io.NotSerializableException
 object PrimitivesSerializationSpec {
   val serializationTestOverrides = ""
 
-  val testConfig = ConfigFactory.parseString(serializationTestOverrides).withFallback(AkkaSpec.testConf)
+  val testConfig = ConfigFactory.parseString(serializationTestOverrides).withFallback(PekkoSpec.testConf)
 }
 
 @deprecated("Moved to org.apache.pekko.serialization.* in akka-actor", "2.6.0")
-class PrimitivesSerializationSpec extends AkkaSpec(PrimitivesSerializationSpec.testConfig) {
+class PrimitivesSerializationSpec extends PekkoSpec(PrimitivesSerializationSpec.testConfig) {
 
   val buffer = {
     val b = ByteBuffer.allocate(4096)
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/ProtobufSerializerSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/ProtobufSerializerSpec.scala
index df99d0c994..b8db4a56d2 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/ProtobufSerializerSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/ProtobufSerializerSpec.scala
@@ -11,7 +11,7 @@ import pekko.remote.ProtobufProtocol.MyMessage
 import pekko.remote.WireFormats.SerializedMessage
 import pekko.remote.protobuf.v3.ProtobufProtocolV3.MyMessageV3
 import pekko.serialization.SerializationExtension
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.util.unused
 
 // those must be defined as top level classes, to have static parseFrom
@@ -45,7 +45,7 @@ object MaliciousMessage {
     new MaliciousMessage
 }
 
-class ProtobufSerializerSpec extends AkkaSpec(s"""
+class ProtobufSerializerSpec extends PekkoSpec(s"""
   pekko.serialization.protobuf.allowed-classes = [
       "com.google.protobuf.GeneratedMessage",
       "com.google.protobuf.GeneratedMessageV3",
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/SerializationTransportInformationSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/SerializationTransportInformationSpec.scala
index 935318d47d..4cd8a2b2bb 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/SerializationTransportInformationSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/SerializationTransportInformationSpec.scala
@@ -17,7 +17,7 @@ import pekko.actor.RootActorPath
 import pekko.remote.RARP
 import pekko.serialization.Serialization
 import pekko.serialization.SerializerWithStringManifest
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.ImplicitSender
 import pekko.testkit.JavaSerializable
 import pekko.testkit.TestActors
@@ -73,7 +73,7 @@ object SerializationTransportInformationSpec {
 }
 
 abstract class AbstractSerializationTransportInformationSpec(config: Config)
-    extends AkkaSpec(config.withFallback(
+    extends PekkoSpec(config.withFallback(
       ConfigFactory.parseString("""
     pekko {
       loglevel = info
diff --git a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/SystemMessageSerializationSpec.scala b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/SystemMessageSerializationSpec.scala
index 12bc9d3b4c..a89a1b10c3 100644
--- a/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/SystemMessageSerializationSpec.scala
+++ b/akka-remote/src/test/scala/org/apache/pekko/remote/serialization/SystemMessageSerializationSpec.scala
@@ -10,7 +10,7 @@ import org.apache.pekko
 import pekko.actor.{ ActorInitializationException, ExtendedActorSystem, InternalActorRef }
 import pekko.dispatch.sysmsg._
 import pekko.serialization.SerializationExtension
-import pekko.testkit.{ AkkaSpec, TestProbe }
+import pekko.testkit.{ PekkoSpec, TestProbe }
 import pekko.testkit.JavaSerializable
 
 object SystemMessageSerializationSpec {
@@ -18,7 +18,7 @@ object SystemMessageSerializationSpec {
     """
     """
 
-  val testConfig = ConfigFactory.parseString(serializationTestOverrides).withFallback(AkkaSpec.testConf)
+  val testConfig = ConfigFactory.parseString(serializationTestOverrides).withFallback(PekkoSpec.testConf)
 
   class TestException(msg: String) extends RuntimeException(msg) with JavaSerializable {
     override def equals(other: Any): Boolean = other match {
@@ -28,7 +28,7 @@ object SystemMessageSerializationSpec {
   }
 }
 
-class SystemMessageSerializationSpec extends AkkaSpec(PrimitivesSerializationSpec.testConfig) {
+class SystemMessageSerializationSpec extends PekkoSpec(PrimitivesSerializationSpec.testConfig) {
   import SystemMessageSerializationSpec._
 
   val testRef = TestProbe().ref.asInstanceOf[InternalActorRef]
diff --git a/akka-serialization-jackson/src/main/resources/reference.conf b/akka-serialization-jackson/src/main/resources/reference.conf
index 25fe23ce5a..6a070923c9 100644
--- a/akka-serialization-jackson/src/main/resources/reference.conf
+++ b/akka-serialization-jackson/src/main/resources/reference.conf
@@ -9,11 +9,11 @@
 pekko.serialization.jackson {
 
   # The Jackson JSON serializer will register these modules.
-  jackson-modules += "org.apache.pekko.serialization.jackson.AkkaJacksonModule"
-  # AkkaTypedJacksonModule optionally included if pekko-actor-typed is in classpath
-  jackson-modules += "org.apache.pekko.serialization.jackson.AkkaTypedJacksonModule"
-  # AkkaStreamsModule optionally included if pekko-streams is in classpath
-  jackson-modules += "org.apache.pekko.serialization.jackson.AkkaStreamJacksonModule"
+  jackson-modules += "org.apache.pekko.serialization.jackson.PekkoJacksonModule"
+  # PekkoTypedJacksonModule optionally included if pekko-actor-typed is in classpath
+  jackson-modules += "org.apache.pekko.serialization.jackson.PekkoTypedJacksonModule"
+  # PekkoStreamsModule optionally included if pekko-streams is in classpath
+  jackson-modules += "org.apache.pekko.serialization.jackson.PekkoStreamJacksonModule"
   jackson-modules += "com.fasterxml.jackson.module.paramnames.ParameterNamesModule"
   jackson-modules += "com.fasterxml.jackson.datatype.jdk8.Jdk8Module"
   jackson-modules += "com.fasterxml.jackson.datatype.jsr310.JavaTimeModule"
diff --git a/akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/AkkaJacksonModule.scala b/akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/AkkaJacksonModule.scala
deleted file mode 100644
index 6fb49102f2..0000000000
--- a/akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/AkkaJacksonModule.scala
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Copyright (C) 2019-2022 Lightbend Inc. 
- */
-
-package org.apache.pekko.serialization.jackson
-
-/**
- * Complete module with support for all custom serializers.
- */
-class AkkaJacksonModule extends JacksonModule with ActorRefModule with AddressModule with FiniteDurationModule {
-  override def getModuleName = "AkkaJacksonModule"
-}
-
-object AkkaJacksonModule extends AkkaJacksonModule
-
-class AkkaTypedJacksonModule extends JacksonModule with TypedActorRefModule {
-  override def getModuleName = "AkkaTypedJacksonModule"
-}
-
-object AkkaTypedJacksonModule extends AkkaJacksonModule
-
-class AkkaStreamJacksonModule extends JacksonModule with StreamRefModule {
-  override def getModuleName = "AkkaStreamJacksonModule"
-}
-
-object AkkaStreamJacksonModule extends AkkaJacksonModule
diff --git a/akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/AkkaSerializationModule.scala b/akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/AkkaSerializationModule.scala
index ee5a04baeb..d8466870ef 100644
--- a/akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/AkkaSerializationModule.scala
+++ b/akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/AkkaSerializationModule.scala
@@ -10,7 +10,7 @@ import com.fasterxml.jackson.databind.deser.std.StdScalarDeserializer
 import com.fasterxml.jackson.databind.ser.std.StdScalarSerializer
 import org.apache.pekko.serialization.{ SerializationExtension, Serializer, Serializers }
 
-final class AkkaSerializationSerializer extends StdScalarSerializer[AnyRef](classOf[AnyRef]) with ActorSystemAccess {
+final class PekkoSerializationSerializer extends StdScalarSerializer[AnyRef](classOf[AnyRef]) with ActorSystemAccess {
   def serialization = SerializationExtension(currentSystem())
   override def serialize(value: AnyRef, jgen: JsonGenerator, provider: SerializerProvider): Unit = {
     val serializer: Serializer = serialization.findSerializerFor(value)
@@ -25,7 +25,7 @@ final class AkkaSerializationSerializer extends StdScalarSerializer[AnyRef](clas
   }
 }
 
-final class AkkaSerializationDeserializer
+final class PekkoSerializationDeserializer
     extends StdScalarDeserializer[AnyRef](classOf[AnyRef])
     with ActorSystemAccess {
 
diff --git a/akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/JacksonObjectMapperProvider.scala b/akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/JacksonObjectMapperProvider.scala
index aa1e2d285c..67260db202 100644
--- a/akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/JacksonObjectMapperProvider.scala
+++ b/akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/JacksonObjectMapperProvider.scala
@@ -271,10 +271,10 @@ object JacksonObjectMapperProvider extends ExtensionId[JacksonObjectMapperProvid
 
   private def isModuleEnabled(fqcn: String, dynamicAccess: DynamicAccess): Boolean =
     fqcn match {
-      case "org.apache.pekko.serialization.jackson.AkkaTypedJacksonModule" =>
+      case "org.apache.pekko.serialization.jackson.PekkoTypedJacksonModule" =>
         // akka-actor-typed dependency is "provided" and may not be included
         dynamicAccess.classIsOnClasspath("org.apache.pekko.actor.typed.ActorRef")
-      case "org.apache.pekko.serialization.jackson.AkkaStreamJacksonModule" =>
+      case "org.apache.pekko.serialization.jackson.PekkoStreamJacksonModule" =>
         // akka-stream dependency is "provided" and may not be included
         dynamicAccess.classIsOnClasspath("org.apache.pekko.stream.Graph")
       case _ => true
diff --git a/akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/PekkoJacksonModule.scala b/akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/PekkoJacksonModule.scala
new file mode 100644
index 0000000000..7dc97976d6
--- /dev/null
+++ b/akka-serialization-jackson/src/main/scala/org/apache/pekko/serialization/jackson/PekkoJacksonModule.scala
@@ -0,0 +1,26 @@
+/*
+ * Copyright (C) 2019-2022 Lightbend Inc. 
+ */
+
+package org.apache.pekko.serialization.jackson
+
+/**
+ * Complete module with support for all custom serializers.
+ */
+class PekkoJacksonModule extends JacksonModule with ActorRefModule with AddressModule with FiniteDurationModule {
+  override def getModuleName = "PekkoJacksonModule"
+}
+
+object PekkoJacksonModule extends PekkoJacksonModule
+
+class PekkoTypedJacksonModule extends JacksonModule with TypedActorRefModule {
+  override def getModuleName = "PekkoTypedJacksonModule"
+}
+
+object PekkoTypedJacksonModule extends PekkoJacksonModule
+
+class PekkoStreamJacksonModule extends JacksonModule with StreamRefModule {
+  override def getModuleName = "PekkoStreamJacksonModule"
+}
+
+object PekkoStreamJacksonModule extends PekkoJacksonModule
diff --git a/akka-serialization-jackson/src/test/scala/org/apache/pekko/serialization/jackson/JacksonSerializerSpec.scala b/akka-serialization-jackson/src/test/scala/org/apache/pekko/serialization/jackson/JacksonSerializerSpec.scala
index c04e59d1c6..279fc93e15 100644
--- a/akka-serialization-jackson/src/test/scala/org/apache/pekko/serialization/jackson/JacksonSerializerSpec.scala
+++ b/akka-serialization-jackson/src/test/scala/org/apache/pekko/serialization/jackson/JacksonSerializerSpec.scala
@@ -121,18 +121,18 @@ object ScalaTestMessages {
   // #jackson-scala-enumeration
 
   // delegate to AkkaSerialization
-  object HasAkkaSerializer {
-    def apply(description: String): HasAkkaSerializer = new HasAkkaSerializer(description)
+  object HasPekkoSerializer {
+    def apply(description: String): HasPekkoSerializer = new HasPekkoSerializer(description)
   }
   // make sure jackson would fail
-  class HasAkkaSerializer private (@JsonIgnore val description: String) {
+  class HasPekkoSerializer private (@JsonIgnore val description: String) {
 
     override def toString: String = s"InnerSerialization($description)"
 
-    def canEqual(other: Any): Boolean = other.isInstanceOf[HasAkkaSerializer]
+    def canEqual(other: Any): Boolean = other.isInstanceOf[HasPekkoSerializer]
 
     override def equals(other: Any): Boolean = other match {
-      case that: HasAkkaSerializer =>
+      case that: HasPekkoSerializer =>
         (that.canEqual(this)) &&
         description == that.description
       case _ => false
@@ -147,14 +147,14 @@ object ScalaTestMessages {
   class InnerSerializationSerializer extends SerializerWithStringManifest {
     override def identifier: Int = 123451
     override def manifest(o: AnyRef): String = "M"
-    override def toBinary(o: AnyRef): Array[Byte] = o.asInstanceOf[HasAkkaSerializer].description.getBytes()
-    override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = HasAkkaSerializer(new String(bytes))
+    override def toBinary(o: AnyRef): Array[Byte] = o.asInstanceOf[HasPekkoSerializer].description.getBytes()
+    override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = HasPekkoSerializer(new String(bytes))
   }
 
-  final case class WithAkkaSerializer(
-      @JsonDeserialize(`using` = classOf[AkkaSerializationDeserializer])
-      @JsonSerialize(`using` = classOf[AkkaSerializationSerializer])
-      akkaSerializer: HasAkkaSerializer)
+  final case class WithPekkoSerializer(
+      @JsonDeserialize(`using` = classOf[PekkoSerializationDeserializer])
+      @JsonSerialize(`using` = classOf[PekkoSerializationSerializer])
+      akkaSerializer: HasPekkoSerializer)
       extends TestMessage
 }
 
@@ -434,7 +434,7 @@ class JacksonJsonSerializerSpec extends JacksonSerializerSpec("jackson-json") {
 
     // This test ensures the default behavior in Akka 2.6 series
     // (that is "FIELD = ANY") stays consistent
-    "be possible to tune the visibility at ObjectMapper level (Akka default)" in {
+    "be possible to tune the visibility at ObjectMapper level (Pekko default)" in {
       withSystem("""
         pekko.actor {
           serialization-bindings {
@@ -711,13 +711,13 @@ object JacksonSerializerSpec {
       }
     }
     pekko.serialization.jackson.allowed-class-prefix = ["org.apache.pekko.serialization.jackson.ScalaTestMessages$$OldCommand"]
-    
+
     pekko.actor {
       serializers {
           inner-serializer = "org.apache.pekko.serialization.jackson.ScalaTestMessages$$InnerSerializationSerializer"
       }
       serialization-bindings {
-        "org.apache.pekko.serialization.jackson.ScalaTestMessages$$HasAkkaSerializer" = "inner-serializer"
+        "org.apache.pekko.serialization.jackson.ScalaTestMessages$$HasPekkoSerializer" = "inner-serializer"
       }
     }
     """
@@ -1270,7 +1270,7 @@ abstract class JacksonSerializerSpec(serializerName: String)
     }
 
     "delegate to akka serialization" in {
-      checkSerialization(WithAkkaSerializer(HasAkkaSerializer("cat")))
+      checkSerialization(WithPekkoSerializer(HasPekkoSerializer("cat")))
     }
 
   }
diff --git a/akka-slf4j/src/test/scala/org/apache/pekko/event/slf4j/Slf4jLoggerSpec.scala b/akka-slf4j/src/test/scala/org/apache/pekko/event/slf4j/Slf4jLoggerSpec.scala
index 6412550b37..7238c8597e 100644
--- a/akka-slf4j/src/test/scala/org/apache/pekko/event/slf4j/Slf4jLoggerSpec.scala
+++ b/akka-slf4j/src/test/scala/org/apache/pekko/event/slf4j/Slf4jLoggerSpec.scala
@@ -16,7 +16,7 @@ import org.slf4j.{ Marker, MarkerFactory }
 import org.apache.pekko
 import pekko.actor.{ Actor, DiagnosticActorLogging, Props }
 import pekko.event.{ LogMarker, Logging }
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
 object Slf4jLoggerSpec {
 
@@ -74,7 +74,7 @@ object Slf4jLoggerSpec {
 
 }
 
-class Slf4jLoggerSpec extends AkkaSpec(Slf4jLoggerSpec.config) with BeforeAndAfterEach {
+class Slf4jLoggerSpec extends PekkoSpec(Slf4jLoggerSpec.config) with BeforeAndAfterEach {
   import Slf4jLoggerSpec._
 
   val producer = system.actorOf(Props[LogProducer](), name = "logProducer")
diff --git a/akka-slf4j/src/test/scala/org/apache/pekko/event/slf4j/Slf4jLoggingFilterSpec.scala b/akka-slf4j/src/test/scala/org/apache/pekko/event/slf4j/Slf4jLoggingFilterSpec.scala
index 7491ebafe7..f5b791569b 100644
--- a/akka-slf4j/src/test/scala/org/apache/pekko/event/slf4j/Slf4jLoggingFilterSpec.scala
+++ b/akka-slf4j/src/test/scala/org/apache/pekko/event/slf4j/Slf4jLoggingFilterSpec.scala
@@ -18,7 +18,7 @@ import pekko.event.Logging.InitializeLogger
 import pekko.event.Logging.LogEvent
 import pekko.event.Logging.LoggerInitialized
 import pekko.event.Logging.Warning
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.TestProbe
 
 object Slf4jLoggingFilterSpec {
@@ -71,7 +71,7 @@ object Slf4jLoggingFilterSpec {
 
 }
 
-class Slf4jLoggingFilterSpec extends AkkaSpec(Slf4jLoggingFilterSpec.config) with BeforeAndAfterEach {
+class Slf4jLoggingFilterSpec extends PekkoSpec(Slf4jLoggingFilterSpec.config) with BeforeAndAfterEach {
   import Slf4jLoggingFilterSpec._
 
   "Slf4jLoggingFilter" must {
diff --git a/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/BaseTwoStreamsSetup.scala b/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/BaseTwoStreamsSetup.scala
index 4c302463e7..f7f934fd4b 100644
--- a/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/BaseTwoStreamsSetup.scala
+++ b/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/BaseTwoStreamsSetup.scala
@@ -12,9 +12,9 @@ import org.reactivestreams.Publisher
 import org.apache.pekko
 import pekko.stream.scaladsl._
 import pekko.stream.testkit.scaladsl.StreamTestKit._
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
-abstract class BaseTwoStreamsSetup extends AkkaSpec("""
+abstract class BaseTwoStreamsSetup extends PekkoSpec("""
     pekko.stream.materializer.initial-input-buffer-size = 2
     pekko.stream.materializer.max-input-buffer-size = 2
   """) {
diff --git a/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/StreamSpec.scala b/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/StreamSpec.scala
index 77c4a6cd40..8e159f017f 100644
--- a/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/StreamSpec.scala
+++ b/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/StreamSpec.scala
@@ -14,24 +14,24 @@ import org.apache.pekko
 import pekko.actor.{ ActorRef, ActorSystem }
 import pekko.stream.impl.StreamSupervisor
 import pekko.stream.snapshot.{ MaterializerState, StreamSnapshotImpl }
-import pekko.testkit.{ AkkaSpec, TestProbe }
+import pekko.testkit.{ PekkoSpec, TestProbe }
 import pekko.testkit.TestKitUtils
 import pekko.stream.impl.PhasedFusingActorMaterializer
 import pekko.stream.testkit.scaladsl.StreamTestKit.{ assertNoChildren, stopAllChildren }
 import pekko.stream.Materializer
 
-abstract class StreamSpec(_system: ActorSystem) extends AkkaSpec(_system) {
+abstract class StreamSpec(_system: ActorSystem) extends PekkoSpec(_system) {
   def this(config: Config) =
     this(
       ActorSystem(
         TestKitUtils.testNameFromCallStack(classOf[StreamSpec], "".r),
-        ConfigFactory.load(config.withFallback(AkkaSpec.testConf))))
+        ConfigFactory.load(config.withFallback(PekkoSpec.testConf))))
 
   def this(s: String) = this(ConfigFactory.parseString(s))
 
-  def this(configMap: Map[String, _]) = this(AkkaSpec.mapToConfig(configMap))
+  def this(configMap: Map[String, _]) = this(PekkoSpec.mapToConfig(configMap))
 
-  def this() = this(ActorSystem(TestKitUtils.testNameFromCallStack(classOf[StreamSpec], "".r), AkkaSpec.testConf))
+  def this() = this(ActorSystem(TestKitUtils.testNameFromCallStack(classOf[StreamSpec], "".r), PekkoSpec.testConf))
 
   override def withFixture(test: NoArgTest) = {
     super.withFixture(test) match {
diff --git a/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/StreamTestKitSpec.scala b/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/StreamTestKitSpec.scala
index 31d2e749d3..82252fe389 100644
--- a/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/StreamTestKitSpec.scala
+++ b/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/StreamTestKitSpec.scala
@@ -15,7 +15,7 @@ import pekko.testkit._
 import pekko.testkit.TestEvent.Mute
 import pekko.testkit.TestEvent.UnMute
 
-class StreamTestKitSpec extends AkkaSpec {
+class StreamTestKitSpec extends PekkoSpec {
 
   val ex = new Exception("Boom!")
 
diff --git a/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/TestPublisherSubscriberSpec.scala b/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/TestPublisherSubscriberSpec.scala
index 60228a93e8..36a1c802d8 100644
--- a/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/TestPublisherSubscriberSpec.scala
+++ b/akka-stream-testkit/src/test/scala/org/apache/pekko/stream/testkit/TestPublisherSubscriberSpec.scala
@@ -12,9 +12,9 @@ import pekko.stream.scaladsl.Source
 import pekko.stream.testkit.TestPublisher._
 import pekko.stream.testkit.TestSubscriber._
 import pekko.stream.testkit.scaladsl.StreamTestKit._
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
-class TestPublisherSubscriberSpec extends AkkaSpec("""
+class TestPublisherSubscriberSpec extends PekkoSpec("""
     pekko.stream.materializer.initial-input-buffer-size = 2
     pekko.stream.materializer.max-input-buffer-size = 2
   """) {
diff --git a/akka-stream-tests-tck/src/test/scala-jdk9-only/org/apache/pekko/stream/tck/IterablePublisherViaJavaFlowPublisherTest.scala b/akka-stream-tests-tck/src/test/scala-jdk9-only/org/apache/pekko/stream/tck/IterablePublisherViaJavaFlowPublisherTest.scala
index 42d0e20bc8..6ddc4ce9ea 100644
--- a/akka-stream-tests-tck/src/test/scala-jdk9-only/org/apache/pekko/stream/tck/IterablePublisherViaJavaFlowPublisherTest.scala
+++ b/akka-stream-tests-tck/src/test/scala-jdk9-only/org/apache/pekko/stream/tck/IterablePublisherViaJavaFlowPublisherTest.scala
@@ -11,7 +11,7 @@ import pekko.NotUsed
 import pekko.stream.scaladsl.{ Flow, JavaFlowSupport, Sink, Source }
 import org.reactivestreams._
 
-class IterablePublisherViaJavaFlowPublisherTest extends AkkaPublisherVerification[Int] {
+class IterablePublisherViaJavaFlowPublisherTest extends PekkoPublisherVerification[Int] {
 
   override def createPublisher(elements: Long): Publisher[Int] = {
     val sourceViaJavaFlowPublisher: JavaFlow.Publisher[Int] = Source(iterable(elements))
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ActorSystemLifecycle.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ActorSystemLifecycle.scala
index 4503918286..4913cf17ac 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ActorSystemLifecycle.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ActorSystemLifecycle.scala
@@ -18,7 +18,7 @@ import org.apache.pekko
 import pekko.actor.ActorSystem
 import pekko.actor.ActorSystemImpl
 import pekko.event.Logging
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.EventFilter
 import pekko.testkit.TestEvent
 
@@ -34,7 +34,7 @@ trait ActorSystemLifecycle {
 
   @BeforeClass
   def createActorSystem(): Unit = {
-    _system = ActorSystem(Logging.simpleName(getClass), additionalConfig.withFallback(AkkaSpec.testConf))
+    _system = ActorSystem(Logging.simpleName(getClass), additionalConfig.withFallback(PekkoSpec.testConf))
     _system.eventStream.publish(TestEvent.Mute(EventFilter[RuntimeException]("Test exception")))
   }
 
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/AkkaSubscriberVerification.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/AkkaSubscriberVerification.scala
index e44b4295db..96fb21f288 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/AkkaSubscriberVerification.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/AkkaSubscriberVerification.scala
@@ -11,10 +11,10 @@ import org.scalatestplus.testng.TestNGSuiteLike
 
 import org.apache.pekko.actor.ActorSystem
 
-abstract class AkkaSubscriberBlackboxVerification[T](env: TestEnvironment)
+abstract class PekkoSubscriberBlackboxVerification[T](env: TestEnvironment)
     extends SubscriberBlackboxVerification[T](env)
     with TestNGSuiteLike
-    with AkkaSubscriberVerificationLike
+    with PekkoSubscriberVerificationLike
     with ActorSystemLifecycle {
 
   def this(printlnDebug: Boolean) =
@@ -23,10 +23,10 @@ abstract class AkkaSubscriberBlackboxVerification[T](env: TestEnvironment)
   def this() = this(false)
 }
 
-abstract class AkkaSubscriberWhiteboxVerification[T](env: TestEnvironment)
+abstract class PekkoSubscriberWhiteboxVerification[T](env: TestEnvironment)
     extends SubscriberWhiteboxVerification[T](env)
     with TestNGSuiteLike
-    with AkkaSubscriberVerificationLike {
+    with PekkoSubscriberVerificationLike {
 
   def this(printlnDebug: Boolean) =
     this(new TestEnvironment(Timeouts.defaultTimeoutMillis, Timeouts.defaultNoSignalsTimeoutMillis, printlnDebug))
@@ -34,6 +34,6 @@ abstract class AkkaSubscriberWhiteboxVerification[T](env: TestEnvironment)
   def this() = this(false)
 }
 
-trait AkkaSubscriberVerificationLike {
+trait PekkoSubscriberVerificationLike {
   implicit def system: ActorSystem
 }
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/CancelledSinkSubscriberTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/CancelledSinkSubscriberTest.scala
index 638b61c132..435ae328cb 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/CancelledSinkSubscriberTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/CancelledSinkSubscriberTest.scala
@@ -9,7 +9,7 @@ import org.testng.SkipException
 
 import org.apache.pekko.stream.scaladsl._
 
-class CancelledSinkSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] {
+class CancelledSinkSubscriberTest extends PekkoSubscriberBlackboxVerification[Int] {
 
   override def createSubscriber(): Subscriber[Int] =
     Flow[Int].to(Sink.cancelled).runWith(Source.asSubscriber)
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ConcatTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ConcatTest.scala
index f7ee32825c..c286082abf 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ConcatTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ConcatTest.scala
@@ -10,7 +10,7 @@ import org.apache.pekko
 import pekko.stream.scaladsl.Sink
 import pekko.stream.scaladsl.Source
 
-class ConcatTest extends AkkaPublisherVerification[Int] {
+class ConcatTest extends PekkoPublisherVerification[Int] {
 
   def createPublisher(elements: Long): Publisher[Int] = {
     Source(iterable(elements / 2)).concat(Source(iterable((elements + 1) / 2))).runWith(Sink.asPublisher(false))
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/EmptyPublisherTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/EmptyPublisherTest.scala
index 82c3b6e775..e5b0639da6 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/EmptyPublisherTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/EmptyPublisherTest.scala
@@ -8,7 +8,7 @@ import org.reactivestreams.Publisher
 
 import org.apache.pekko.stream.impl.EmptyPublisher
 
-class EmptyPublisherTest extends AkkaPublisherVerification[Int] {
+class EmptyPublisherTest extends PekkoPublisherVerification[Int] {
 
   def createPublisher(elements: Long): Publisher[Int] = EmptyPublisher[Int]
 
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FanoutPublisherTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FanoutPublisherTest.scala
index fbe4eabab5..96fbf059a4 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FanoutPublisherTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FanoutPublisherTest.scala
@@ -12,7 +12,7 @@ import org.apache.pekko
 import pekko.stream.scaladsl.Sink
 import pekko.stream.scaladsl.Source
 
-class FanoutPublisherTest extends AkkaPublisherVerification[Int] {
+class FanoutPublisherTest extends PekkoPublisherVerification[Int] {
 
   def createPublisher(elements: Long): Publisher[Int] = {
     val iterable: immutable.Iterable[Int] =
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FilePublisherTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FilePublisherTest.scala
index ed6e231e25..78bae997ae 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FilePublisherTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FilePublisherTest.scala
@@ -15,17 +15,17 @@ import pekko.event.Logging
 import pekko.stream.scaladsl.{ FileIO, Sink }
 import pekko.stream.testkit.Utils._
 import pekko.testkit.{ EventFilter, TestEvent }
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.util.ByteString
 
-class FilePublisherTest extends AkkaPublisherVerification[ByteString] {
+class FilePublisherTest extends PekkoPublisherVerification[ByteString] {
 
   val ChunkSize = 256
   val Elements = 1000
 
   @BeforeClass
   override def createActorSystem(): Unit = {
-    _system = ActorSystem(Logging.simpleName(getClass), UnboundedMailboxConfig.withFallback(AkkaSpec.testConf))
+    _system = ActorSystem(Logging.simpleName(getClass), UnboundedMailboxConfig.withFallback(PekkoSpec.testConf))
     _system.eventStream.publish(TestEvent.Mute(EventFilter[RuntimeException]("Test exception")))
   }
 
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FlatMapConcatDoubleSubscriberTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FlatMapConcatDoubleSubscriberTest.scala
index 6c2eb1dd82..7c029afa2e 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FlatMapConcatDoubleSubscriberTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FlatMapConcatDoubleSubscriberTest.scala
@@ -11,7 +11,7 @@ import org.reactivestreams.{ Publisher, Subscriber }
 
 import org.apache.pekko.stream.scaladsl.{ Sink, Source }
 
-class FlatMapConcatDoubleSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] {
+class FlatMapConcatDoubleSubscriberTest extends PekkoSubscriberBlackboxVerification[Int] {
 
   def createSubscriber(): Subscriber[Int] = {
     val subscriber = Promise[Subscriber[Int]]()
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FlatMapPrefixTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FlatMapPrefixTest.scala
index 5e09664115..2d141387cd 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FlatMapPrefixTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FlatMapPrefixTest.scala
@@ -8,7 +8,7 @@ import org.reactivestreams.Publisher
 
 import org.apache.pekko.stream.scaladsl.{ Flow, Keep, Sink, Source }
 
-class FlatMapPrefixTest extends AkkaPublisherVerification[Int] {
+class FlatMapPrefixTest extends PekkoPublisherVerification[Int] {
   override def createPublisher(elements: Long): Publisher[Int] = {
     val publisher = Source(iterable(elements))
       .map(_.toInt)
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FlattenTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FlattenTest.scala
index 14aec1aa42..65ccdf16a7 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FlattenTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FlattenTest.scala
@@ -11,7 +11,7 @@ import pekko.stream.scaladsl.Sink
 import pekko.stream.scaladsl.Source
 import pekko.util.ConstantFun
 
-class FlattenTest extends AkkaPublisherVerification[Int] {
+class FlattenTest extends PekkoPublisherVerification[Int] {
 
   def createPublisher(elements: Long): Publisher[Int] = {
     val s1 = Source(iterable(elements / 2))
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FoldSinkSubscriberTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FoldSinkSubscriberTest.scala
index 814ce95884..57657a74f0 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FoldSinkSubscriberTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FoldSinkSubscriberTest.scala
@@ -8,7 +8,7 @@ import org.reactivestreams.Subscriber
 
 import org.apache.pekko.stream.scaladsl._
 
-class FoldSinkSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] {
+class FoldSinkSubscriberTest extends PekkoSubscriberBlackboxVerification[Int] {
 
   override def createSubscriber(): Subscriber[Int] =
     Flow[Int].to(Sink.fold(0)(_ + _)).runWith(Source.asSubscriber)
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ForeachSinkSubscriberTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ForeachSinkSubscriberTest.scala
index d8409ffd81..49b34b9e2b 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ForeachSinkSubscriberTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/ForeachSinkSubscriberTest.scala
@@ -8,7 +8,7 @@ import org.reactivestreams.Subscriber
 
 import org.apache.pekko.stream.scaladsl._
 
-class ForeachSinkSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] {
+class ForeachSinkSubscriberTest extends PekkoSubscriberBlackboxVerification[Int] {
 
   override def createSubscriber(): Subscriber[Int] =
     Flow[Int]
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FusableProcessorTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FusableProcessorTest.scala
index fc51a192c2..c84c938ace 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FusableProcessorTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FusableProcessorTest.scala
@@ -10,7 +10,7 @@ import org.apache.pekko
 import pekko.stream._
 import pekko.stream.scaladsl.Flow
 
-class FusableProcessorTest extends AkkaIdentityProcessorVerification[Int] {
+class FusableProcessorTest extends PekkoIdentityProcessorVerification[Int] {
 
   override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = {
     Flow[Int]
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FuturePublisherTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FuturePublisherTest.scala
index eff1d1d2e0..9c2865b211 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FuturePublisherTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/FuturePublisherTest.scala
@@ -12,7 +12,7 @@ import org.apache.pekko
 import pekko.stream.scaladsl.Sink
 import pekko.stream.scaladsl.Source
 
-class FuturePublisherTest extends AkkaPublisherVerification[Int] {
+class FuturePublisherTest extends PekkoPublisherVerification[Int] {
 
   def createPublisher(elements: Long): Publisher[Int] = {
     val p = Promise[Int]()
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/GroupByTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/GroupByTest.scala
index d74c100499..d43333bc6d 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/GroupByTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/GroupByTest.scala
@@ -14,7 +14,7 @@ import pekko.stream.impl.EmptyPublisher
 import pekko.stream.scaladsl.Sink
 import pekko.stream.scaladsl.Source
 
-class GroupByTest extends AkkaPublisherVerification[Int] {
+class GroupByTest extends PekkoPublisherVerification[Int] {
 
   def createPublisher(elements: Long): Publisher[Int] =
     if (elements == 0) EmptyPublisher[Int]
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/InputStreamSourceTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/InputStreamSourceTest.scala
index 6b5a62bd96..27e814cd75 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/InputStreamSourceTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/InputStreamSourceTest.scala
@@ -13,7 +13,7 @@ import pekko.stream.ActorAttributes
 import pekko.stream.scaladsl.{ Sink, StreamConverters }
 import pekko.util.ByteString
 
-class InputStreamSourceTest extends AkkaPublisherVerification[ByteString] {
+class InputStreamSourceTest extends PekkoPublisherVerification[ByteString] {
 
   def createPublisher(elements: Long): Publisher[ByteString] = {
     StreamConverters
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/IterablePublisherTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/IterablePublisherTest.scala
index 7eb2920a86..f031da9171 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/IterablePublisherTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/IterablePublisherTest.scala
@@ -10,7 +10,7 @@ import org.apache.pekko
 import pekko.stream.scaladsl.Sink
 import pekko.stream.scaladsl.Source
 
-class IterablePublisherTest extends AkkaPublisherVerification[Int] {
+class IterablePublisherTest extends PekkoPublisherVerification[Int] {
 
   override def createPublisher(elements: Long): Publisher[Int] = {
     Source(iterable(elements)).runWith(Sink.asPublisher(false))
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/MapTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/MapTest.scala
index 909ec4dd9a..1bcf6977a8 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/MapTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/MapTest.scala
@@ -8,7 +8,7 @@ import org.reactivestreams.Processor
 
 import org.apache.pekko.stream.scaladsl.Flow
 
-class MapTest extends AkkaIdentityProcessorVerification[Int] {
+class MapTest extends PekkoIdentityProcessorVerification[Int] {
 
   override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = {
     Flow[Int].map(elem => elem).named("identity").toProcessor.run()
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/MaybeSourceTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/MaybeSourceTest.scala
index 79a7287717..6e8c9ccd8a 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/MaybeSourceTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/MaybeSourceTest.scala
@@ -8,7 +8,7 @@ import org.reactivestreams.Publisher
 
 import org.apache.pekko.stream.scaladsl.{ Keep, Sink, Source }
 
-class MaybeSourceTest extends AkkaPublisherVerification[Int] {
+class MaybeSourceTest extends PekkoPublisherVerification[Int] {
 
   def createPublisher(elements: Long): Publisher[Int] = {
     val (p, pub) = Source.maybe[Int].toMat(Sink.asPublisher(false))(Keep.both).run()
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/AkkaIdentityProcessorVerification.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/PekkoIdentityProcessorVerification.scala
similarity index 94%
rename from akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/AkkaIdentityProcessorVerification.scala
rename to akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/PekkoIdentityProcessorVerification.scala
index 98cb5b50c4..57fd006ee3 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/AkkaIdentityProcessorVerification.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/PekkoIdentityProcessorVerification.scala
@@ -16,7 +16,7 @@ import org.testng.annotations.AfterClass
 
 import org.apache.pekko.stream.testkit.TestPublisher
 
-abstract class AkkaIdentityProcessorVerification[T](env: TestEnvironment, publisherShutdownTimeout: Long)
+abstract class PekkoIdentityProcessorVerification[T](env: TestEnvironment, publisherShutdownTimeout: Long)
     extends IdentityProcessorVerification[T](env, publisherShutdownTimeout)
     with TestNGSuiteLike
     with ActorSystemLifecycle {
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/AkkaPublisherVerification.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/PekkoPublisherVerification.scala
similarity index 93%
rename from akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/AkkaPublisherVerification.scala
rename to akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/PekkoPublisherVerification.scala
index 29f7fb021a..1c54c815d5 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/AkkaPublisherVerification.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/PekkoPublisherVerification.scala
@@ -15,7 +15,7 @@ import org.scalatestplus.testng.TestNGSuiteLike
 
 import org.apache.pekko.stream.testkit.TestPublisher
 
-abstract class AkkaPublisherVerification[T](val env: TestEnvironment, publisherShutdownTimeout: Long)
+abstract class PekkoPublisherVerification[T](val env: TestEnvironment, publisherShutdownTimeout: Long)
     extends PublisherVerification[T](env, publisherShutdownTimeout)
     with TestNGSuiteLike
     with ActorSystemLifecycle {
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/PrefixAndTailTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/PrefixAndTailTest.scala
index 88d1dda150..dd5382c63c 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/PrefixAndTailTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/PrefixAndTailTest.scala
@@ -13,7 +13,7 @@ import org.apache.pekko
 import pekko.stream.scaladsl.Sink
 import pekko.stream.scaladsl.Source
 
-class PrefixAndTailTest extends AkkaPublisherVerification[Int] {
+class PrefixAndTailTest extends PekkoPublisherVerification[Int] {
 
   def createPublisher(elements: Long): Publisher[Int] = {
     val futureTailSource = Source(iterable(elements)).prefixAndTail(0).map { case (_, tail) => tail }.runWith(Sink.head)
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/SingleElementSourceTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/SingleElementSourceTest.scala
index dc10c4481e..4ff25261f3 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/SingleElementSourceTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/SingleElementSourceTest.scala
@@ -10,7 +10,7 @@ import org.apache.pekko
 import pekko.stream.scaladsl.Sink
 import pekko.stream.scaladsl.Source
 
-class SingleElementSourceTest extends AkkaPublisherVerification[Int] {
+class SingleElementSourceTest extends PekkoPublisherVerification[Int] {
 
   def createPublisher(elements: Long): Publisher[Int] =
     Source.single(1).runWith(Sink.asPublisher(false))
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/SplitWhenTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/SplitWhenTest.scala
index a4c26360de..166d102ec1 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/SplitWhenTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/SplitWhenTest.scala
@@ -14,7 +14,7 @@ import pekko.stream.impl.EmptyPublisher
 import pekko.stream.scaladsl.Sink
 import pekko.stream.scaladsl.Source
 
-class SplitWhenTest extends AkkaPublisherVerification[Int] {
+class SplitWhenTest extends PekkoPublisherVerification[Int] {
 
   def createPublisher(elements: Long): Publisher[Int] =
     if (elements == 0) EmptyPublisher[Int]
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/TransformProcessorTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/TransformProcessorTest.scala
index 16a57607d5..4dd48fffe3 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/TransformProcessorTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/TransformProcessorTest.scala
@@ -14,7 +14,7 @@ import pekko.stream.stage.GraphStageLogic
 import pekko.stream.stage.InHandler
 import pekko.stream.stage.OutHandler
 
-class TransformProcessorTest extends AkkaIdentityProcessorVerification[Int] {
+class TransformProcessorTest extends PekkoIdentityProcessorVerification[Int] {
 
   override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = {
     val stage =
diff --git a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/VirtualPublisherTest.scala b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/VirtualPublisherTest.scala
index d58a181e72..0cf7b7a0ae 100644
--- a/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/VirtualPublisherTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/org/apache/pekko/stream/tck/VirtualPublisherTest.scala
@@ -10,7 +10,7 @@ import org.apache.pekko
 import pekko.stream.impl.VirtualProcessor
 import pekko.stream.scaladsl.Flow
 
-class VirtualProcessorTest extends AkkaIdentityProcessorVerification[Int] {
+class VirtualProcessorTest extends PekkoIdentityProcessorVerification[Int] {
 
   override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = {
     val identity = Flow[Int].map(elem => elem).named("identity").toProcessor.run()
@@ -24,7 +24,7 @@ class VirtualProcessorTest extends AkkaIdentityProcessorVerification[Int] {
 
 }
 
-class VirtualProcessorSingleTest extends AkkaIdentityProcessorVerification[Int] {
+class VirtualProcessorSingleTest extends PekkoIdentityProcessorVerification[Int] {
 
   override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] =
     new VirtualProcessor[Int]
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/StreamAttributeDocTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/StreamAttributeDocTest.java
index 65e46c0d2e..16316edc91 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/StreamAttributeDocTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/StreamAttributeDocTest.java
@@ -10,8 +10,8 @@ import org.apache.pekko.stream.javadsl.RunnableGraph;
 import org.apache.pekko.stream.javadsl.Sink;
 import org.apache.pekko.stream.javadsl.Source;
 import org.apache.pekko.stream.scaladsl.TcpAttributes;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
 import com.typesafe.config.ConfigFactory;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -28,11 +28,11 @@ public class StreamAttributeDocTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource(
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource(
           "StreamAttributeDocTest",
           ConfigFactory.parseString("my-stream-dispatcher = pekko.test.stream-dispatcher")
-              .withFallback(AkkaSpec.testConf()));
+              .withFallback(PekkoSpec.testConf()));
 
   @Test
   public void runnableAttributesExample() throws Exception {
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/StreamTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/StreamTest.java
index 6412e3ebbd..04bc496791 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/StreamTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/StreamTest.java
@@ -5,13 +5,13 @@
 package org.apache.pekko.stream;
 
 import org.apache.pekko.actor.ActorSystem;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
 import org.scalatestplus.junit.JUnitSuite;
 
 public abstract class StreamTest extends JUnitSuite {
   protected final ActorSystem system;
 
-  protected StreamTest(AkkaJUnitActorSystemResource actorSystemResource) {
+  protected StreamTest(PekkoJUnitActorSystemResource actorSystemResource) {
     system = actorSystemResource.getSystem();
   }
 }
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/InputStreamSinkTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/InputStreamSinkTest.java
index bdf765697c..ae14ecba4e 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/InputStreamSinkTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/InputStreamSinkTest.java
@@ -5,7 +5,7 @@
 package org.apache.pekko.stream.io;
 
 import org.apache.pekko.stream.StreamTest;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
 import org.apache.pekko.stream.javadsl.Sink;
 import org.apache.pekko.stream.javadsl.Source;
 import org.apache.pekko.stream.javadsl.StreamConverters;
@@ -28,8 +28,8 @@ public class InputStreamSinkTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("InputStreamSink", Utils.UnboundedMailboxConfig());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("InputStreamSink", Utils.UnboundedMailboxConfig());
 
   @Test
   public void mustReadEventViaInputStream() throws Exception {
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/OutputStreamSinkTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/OutputStreamSinkTest.java
index 9434f85a49..6a20f6f453 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/OutputStreamSinkTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/OutputStreamSinkTest.java
@@ -7,7 +7,7 @@ package org.apache.pekko.stream.io;
 import org.apache.pekko.stream.IOOperationIncompleteException;
 import org.apache.pekko.stream.IOResult;
 import org.apache.pekko.stream.StreamTest;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
 import org.apache.pekko.stream.javadsl.Source;
 import org.apache.pekko.stream.javadsl.StreamConverters;
 import org.apache.pekko.stream.testkit.Utils;
@@ -29,8 +29,8 @@ public class OutputStreamSinkTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("OutputStreamSinkTest", Utils.UnboundedMailboxConfig());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("OutputStreamSinkTest", Utils.UnboundedMailboxConfig());
 
   @Test
   public void mustSignalFailureViaFailingFuture() {
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/OutputStreamSourceTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/OutputStreamSourceTest.java
index 08310018a6..d7e7ff6891 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/OutputStreamSourceTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/OutputStreamSourceTest.java
@@ -16,7 +16,7 @@ import org.junit.Test;
 import org.apache.pekko.actor.ActorRef;
 import org.apache.pekko.japi.function.Procedure;
 import org.apache.pekko.stream.StreamTest;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
 import org.apache.pekko.stream.javadsl.Sink;
 import org.apache.pekko.stream.javadsl.Source;
 import org.apache.pekko.stream.javadsl.StreamConverters;
@@ -29,8 +29,8 @@ public class OutputStreamSourceTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("OutputStreamSourceTest", Utils.UnboundedMailboxConfig());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("OutputStreamSourceTest", Utils.UnboundedMailboxConfig());
 
   @Test
   public void mustSendEventsViaOutputStream() throws Exception {
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/SinkAsJavaSourceTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/SinkAsJavaSourceTest.java
index cdf5d72648..1d1e91114e 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/SinkAsJavaSourceTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/io/SinkAsJavaSourceTest.java
@@ -5,7 +5,7 @@
 package org.apache.pekko.stream.io;
 
 import org.apache.pekko.stream.StreamTest;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
 import org.apache.pekko.stream.javadsl.Sink;
 import org.apache.pekko.stream.javadsl.Source;
 import org.apache.pekko.stream.javadsl.StreamConverters;
@@ -26,8 +26,8 @@ public class SinkAsJavaSourceTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("SinkAsJavaSourceTest", Utils.UnboundedMailboxConfig());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("SinkAsJavaSourceTest", Utils.UnboundedMailboxConfig());
 
   @Test
   public void mustBeAbleToUseAsJavaStream() throws Exception {
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/AttributesTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/AttributesTest.java
index d81493dcf4..dd7d7740a8 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/AttributesTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/AttributesTest.java
@@ -15,8 +15,8 @@ import org.junit.Test;
 
 import org.apache.pekko.stream.Attributes;
 import org.apache.pekko.stream.StreamTest;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
 
 public class AttributesTest extends StreamTest {
 
@@ -25,8 +25,8 @@ public class AttributesTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("AttributesTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("AttributesTest", PekkoSpec.testConf());
 
   final Attributes attributes =
       Attributes.name("a").and(Attributes.name("b")).and(Attributes.inputBuffer(1, 2));
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowTest.java
index d0f208dadb..b8db697bed 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowTest.java
@@ -17,14 +17,14 @@ import org.apache.pekko.stream.testkit.javadsl.TestSink;
 import org.apache.pekko.util.ConstantFun;
 import org.apache.pekko.stream.javadsl.GraphDSL.Builder;
 import org.apache.pekko.stream.stage.*;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.apache.pekko.stream.testkit.TestPublisher;
 import org.apache.pekko.testkit.javadsl.TestKit;
 import org.junit.Assert;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.reactivestreams.Publisher;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
 
 import java.util.*;
 import java.util.function.Supplier;
@@ -48,8 +48,8 @@ public class FlowTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("FlowTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("FlowTest", PekkoSpec.testConf());
 
   interface Fruit {}
 
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowThrottleTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowThrottleTest.java
index f9b0f3ee3e..cad37cd9a5 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowThrottleTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowThrottleTest.java
@@ -7,8 +7,8 @@ package org.apache.pekko.stream.javadsl;
 import org.apache.pekko.NotUsed;
 import org.apache.pekko.stream.StreamTest;
 import org.apache.pekko.stream.ThrottleMode;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.junit.ClassRule;
 import org.junit.Test;
 
@@ -25,8 +25,8 @@ public class FlowThrottleTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("ThrottleTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("ThrottleTest", PekkoSpec.testConf());
 
   @Test
   public void mustWorksForTwoStreams() throws Exception {
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowWithContextTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowWithContextTest.java
index 7837496f96..66cd826896 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowWithContextTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowWithContextTest.java
@@ -7,8 +7,8 @@ package org.apache.pekko.stream.javadsl;
 import org.apache.pekko.NotUsed;
 import org.apache.pekko.japi.Pair;
 import org.apache.pekko.stream.StreamTest;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.junit.ClassRule;
 import org.junit.Test;
 
@@ -26,8 +26,8 @@ public class FlowWithContextTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("FlowWithContextTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("FlowWithContextTest", PekkoSpec.testConf());
 
   @Test
   public void simpleCaseHappyPath() throws Exception {
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowWithContextThrottleTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowWithContextThrottleTest.java
index cf0c91edf3..f3d9fb0000 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowWithContextThrottleTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FlowWithContextThrottleTest.java
@@ -8,8 +8,8 @@ import org.apache.pekko.NotUsed;
 import org.apache.pekko.japi.Pair;
 import org.apache.pekko.stream.StreamTest;
 import org.apache.pekko.stream.ThrottleMode;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.junit.ClassRule;
 import org.junit.Test;
 
@@ -26,8 +26,8 @@ public class FlowWithContextThrottleTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("ThrottleTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("ThrottleTest", PekkoSpec.testConf());
 
   @Test
   public void mustWorksForTwoStreams() throws Exception {
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FramingTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FramingTest.java
index 1736a8753c..b90080e0e9 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FramingTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/FramingTest.java
@@ -6,11 +6,11 @@ package org.apache.pekko.stream.javadsl;
 
 import org.apache.pekko.NotUsed;
 import org.apache.pekko.stream.StreamTest;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.apache.pekko.util.ByteString;
 import org.junit.ClassRule;
 import org.junit.Test;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
 
 public class FramingTest extends StreamTest {
   public FramingTest() {
@@ -18,8 +18,8 @@ public class FramingTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("FramingTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("FramingTest", PekkoSpec.testConf());
 
   @Test
   public void mustBeAbleToUseFraming() throws Exception {
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/GraphDslTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/GraphDslTest.java
index b90dbe8840..90b09d7e5f 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/GraphDslTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/GraphDslTest.java
@@ -7,8 +7,8 @@ package org.apache.pekko.stream.javadsl;
 import org.apache.pekko.NotUsed;
 import org.apache.pekko.japi.Pair;
 import org.apache.pekko.stream.*;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.junit.Assert;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -30,8 +30,8 @@ public class GraphDslTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("GraphDslTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("GraphDslTest", PekkoSpec.testConf());
 
   @Test
   public void demonstrateBuildSimpleGraph() throws Exception {
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/JsonFramingTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/JsonFramingTest.java
index 238d12054c..2fdac26a07 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/JsonFramingTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/JsonFramingTest.java
@@ -6,8 +6,8 @@ package org.apache.pekko.stream.javadsl;
 
 import org.apache.pekko.util.ByteString;
 import org.apache.pekko.stream.StreamTest;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.junit.ClassRule;
 import org.junit.Test;
 
@@ -26,8 +26,8 @@ public class JsonFramingTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("JsonFramingTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("JsonFramingTest", PekkoSpec.testConf());
 
   @Test
   public void mustBeAbleToParseJsonArray()
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/KillSwitchTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/KillSwitchTest.java
index bc5f12d129..f72cfca0a0 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/KillSwitchTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/KillSwitchTest.java
@@ -9,13 +9,13 @@ import org.apache.pekko.stream.*;
 import org.apache.pekko.stream.testkit.TestPublisher;
 import org.apache.pekko.stream.testkit.TestSubscriber;
 import org.apache.pekko.stream.testkit.Utils;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.junit.ClassRule;
 import org.junit.Test;
 
 import java.util.concurrent.CompletionStage;
 import java.util.concurrent.TimeUnit;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
 
 import static org.apache.pekko.Done.done;
 import static org.junit.Assert.*;
@@ -26,8 +26,8 @@ public class KillSwitchTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("KillSwitchTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("KillSwitchTest", PekkoSpec.testConf());
 
   @Test
   public void beAbleToUseKillSwitch() throws Exception {
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/LazyAndFutureFlowTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/LazyAndFutureFlowTest.java
index 6723747037..626b85f0c7 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/LazyAndFutureFlowTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/LazyAndFutureFlowTest.java
@@ -5,8 +5,8 @@
 package org.apache.pekko.stream.javadsl;
 
 import org.apache.pekko.stream.StreamTest;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.junit.ClassRule;
 import org.junit.Test;
 
@@ -21,8 +21,8 @@ import static org.junit.Assert.assertEquals;
 public class LazyAndFutureFlowTest extends StreamTest {
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("LazyAndFutureFlowTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("LazyAndFutureFlowTest", PekkoSpec.testConf());
 
   public LazyAndFutureFlowTest() {
     super(actorSystemResource);
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/LazyAndFutureSourcesTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/LazyAndFutureSourcesTest.java
index 1bb8fbe459..cb094973e7 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/LazyAndFutureSourcesTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/LazyAndFutureSourcesTest.java
@@ -7,8 +7,8 @@ package org.apache.pekko.stream.javadsl;
 import org.apache.pekko.NotUsed;
 import org.apache.pekko.japi.Pair;
 import org.apache.pekko.stream.StreamTest;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.junit.ClassRule;
 import org.junit.Test;
 import scala.concurrent.Future;
@@ -24,8 +24,8 @@ import static org.junit.Assert.assertEquals;
 public class LazyAndFutureSourcesTest extends StreamTest {
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("LazyAndFutureSourcesTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("LazyAndFutureSourcesTest", PekkoSpec.testConf());
 
   public LazyAndFutureSourcesTest() {
     super(actorSystemResource);
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/RetryFlowTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/RetryFlowTest.java
index 85e81d7dbb..6ad2f07f54 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/RetryFlowTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/RetryFlowTest.java
@@ -11,8 +11,8 @@ import org.apache.pekko.stream.testkit.TestPublisher;
 import org.apache.pekko.stream.testkit.TestSubscriber;
 import org.apache.pekko.stream.testkit.javadsl.TestSink;
 import org.apache.pekko.stream.testkit.javadsl.TestSource;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.junit.ClassRule;
 import org.junit.Test;
 import scala.util.Failure;
@@ -30,8 +30,8 @@ public class RetryFlowTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("RetryFlowTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("RetryFlowTest", PekkoSpec.testConf());
 
   public static
   // #withBackoff-signature
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/RunnableGraphTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/RunnableGraphTest.java
index 681a77703c..5c9c63162d 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/RunnableGraphTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/RunnableGraphTest.java
@@ -7,8 +7,8 @@ package org.apache.pekko.stream.javadsl;
 import org.apache.pekko.NotUsed;
 import org.apache.pekko.stream.StreamTest;
 import org.apache.pekko.stream.SystemMaterializer;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.junit.ClassRule;
 import org.junit.Test;
 
@@ -20,8 +20,8 @@ public class RunnableGraphTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("RunnableGraphTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("RunnableGraphTest", PekkoSpec.testConf());
 
   @Test
   public void beAbleToConvertFromJavaToScala() {
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SetupTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SetupTest.java
index 850c59d82d..fec70aa988 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SetupTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SetupTest.java
@@ -7,8 +7,8 @@ package org.apache.pekko.stream.javadsl;
 import org.apache.pekko.NotUsed;
 import org.apache.pekko.japi.Pair;
 import org.apache.pekko.stream.StreamTest;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.junit.ClassRule;
 import org.junit.Test;
 
@@ -23,8 +23,8 @@ public class SetupTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("SetupTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("SetupTest", PekkoSpec.testConf());
 
   @Test
   public void shouldExposeMaterializerAndAttributesToSource() throws Exception {
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SinkTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SinkTest.java
index 99d4e9f7bd..8c3a6f1eea 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SinkTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SinkTest.java
@@ -21,8 +21,8 @@ import org.apache.pekko.testkit.javadsl.TestKit;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.reactivestreams.Publisher;
-import org.apache.pekko.testkit.AkkaSpec;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
 
 import static org.junit.Assert.*;
 
@@ -32,8 +32,8 @@ public class SinkTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("SinkTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("SinkTest", PekkoSpec.testConf());
 
   @Test
   public void mustBeAbleToUseFanoutPublisher() throws Exception {
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SourceTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SourceTest.java
index 23a17576d4..5b842b6b38 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SourceTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SourceTest.java
@@ -24,8 +24,8 @@ import org.apache.pekko.stream.stage.GraphStageLogic;
 import org.apache.pekko.stream.testkit.TestPublisher;
 import org.apache.pekko.stream.testkit.TestSubscriber;
 import org.apache.pekko.stream.testkit.javadsl.TestSink;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.apache.pekko.testkit.javadsl.TestKit;
 import org.apache.pekko.util.ConstantFun;
 import com.google.common.collect.Iterables;
@@ -56,8 +56,8 @@ public class SourceTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("SourceTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("SourceTest", PekkoSpec.testConf());
 
   interface Fruit {}
 
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SourceWithContextThrottleTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SourceWithContextThrottleTest.java
index 9c35e75c6e..441f18664e 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SourceWithContextThrottleTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/SourceWithContextThrottleTest.java
@@ -7,8 +7,8 @@ package org.apache.pekko.stream.javadsl;
 import org.apache.pekko.japi.Pair;
 import org.apache.pekko.stream.StreamTest;
 import org.apache.pekko.stream.ThrottleMode;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.junit.ClassRule;
 import org.junit.Test;
 
@@ -26,8 +26,8 @@ public class SourceWithContextThrottleTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("ThrottleTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("ThrottleTest", PekkoSpec.testConf());
 
   @Test
   public void mustBeAbleToUseThrottle() throws Exception {
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/TcpTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/TcpTest.java
index 60ef1e52e9..5dffd03457 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/TcpTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/javadsl/TcpTest.java
@@ -13,8 +13,8 @@ import org.apache.pekko.stream.StreamTest;
 
 import org.apache.pekko.stream.javadsl.Tcp.IncomingConnection;
 import org.apache.pekko.stream.javadsl.Tcp.ServerBinding;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoSpec;
 import org.apache.pekko.testkit.SocketUtil;
 import org.apache.pekko.testkit.javadsl.EventFilter;
 import org.apache.pekko.testkit.javadsl.TestKit;
@@ -60,8 +60,8 @@ public class TcpTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("TcpTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("TcpTest", PekkoSpec.testConf());
 
   final Sink> echoHandler =
       Sink.foreach(
diff --git a/akka-stream-tests/src/test/java/org/apache/pekko/stream/stage/StageTest.java b/akka-stream-tests/src/test/java/org/apache/pekko/stream/stage/StageTest.java
index f7147d2ab6..5468ed83c0 100644
--- a/akka-stream-tests/src/test/java/org/apache/pekko/stream/stage/StageTest.java
+++ b/akka-stream-tests/src/test/java/org/apache/pekko/stream/stage/StageTest.java
@@ -6,10 +6,10 @@ package org.apache.pekko.stream.stage;
 
 import org.apache.pekko.NotUsed;
 import org.apache.pekko.stream.StreamTest;
-import org.apache.pekko.testkit.AkkaJUnitActorSystemResource;
+import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
 import org.apache.pekko.stream.javadsl.Sink;
 import org.apache.pekko.stream.javadsl.Source;
-import org.apache.pekko.testkit.AkkaSpec;
+import org.apache.pekko.testkit.PekkoSpec;
 
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -26,8 +26,8 @@ public class StageTest extends StreamTest {
   }
 
   @ClassRule
-  public static AkkaJUnitActorSystemResource actorSystemResource =
-      new AkkaJUnitActorSystemResource("StageTest", AkkaSpec.testConf());
+  public static PekkoJUnitActorSystemResource actorSystemResource =
+      new PekkoJUnitActorSystemResource("StageTest", PekkoSpec.testConf());
 
   @Test
   public void javaStageUsage() throws Exception {
diff --git a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/LinearTraversalBuilderSpec.scala b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/LinearTraversalBuilderSpec.scala
index 90511a2fce..eac2d10488 100644
--- a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/LinearTraversalBuilderSpec.scala
+++ b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/LinearTraversalBuilderSpec.scala
@@ -9,9 +9,9 @@ import pekko.NotUsed
 import pekko.stream._
 import pekko.stream.impl.TraversalTestUtils._
 import pekko.stream.scaladsl.Keep
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
-class LinearTraversalBuilderSpec extends AkkaSpec {
+class LinearTraversalBuilderSpec extends PekkoSpec {
 
   "LinearTraversalBuilder" must {
     val source = new LinearTestSource
diff --git a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/TraversalBuilderSpec.scala b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/TraversalBuilderSpec.scala
index eee5eec217..a80cd4b3e0 100644
--- a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/TraversalBuilderSpec.scala
+++ b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/TraversalBuilderSpec.scala
@@ -9,9 +9,9 @@ import pekko.NotUsed
 import pekko.stream._
 import pekko.stream.impl.TraversalTestUtils._
 import pekko.stream.scaladsl.Keep
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
-class TraversalBuilderSpec extends AkkaSpec {
+class TraversalBuilderSpec extends PekkoSpec {
 
   "CompositeTraversalBuilder" must {
     val source = new CompositeTestSource
diff --git a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/fusing/AsyncCallbackSpec.scala b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/fusing/AsyncCallbackSpec.scala
index 6ef014eab5..7107127c5b 100644
--- a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/fusing/AsyncCallbackSpec.scala
+++ b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/fusing/AsyncCallbackSpec.scala
@@ -20,10 +20,10 @@ import pekko.stream.stage._
 import pekko.stream.testkit.TestPublisher
 import pekko.stream.testkit.TestSubscriber
 import pekko.stream.testkit.Utils.TE
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.testkit.TestProbe
 
-class AsyncCallbackSpec extends AkkaSpec("""
+class AsyncCallbackSpec extends PekkoSpec("""
     pekko.stream.materializer.debug.fuzzing-mode = off
   """) {
 
diff --git a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/fusing/ChasingEventsSpec.scala b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/fusing/ChasingEventsSpec.scala
index 03f72801e2..ad7112573e 100644
--- a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/fusing/ChasingEventsSpec.scala
+++ b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/impl/fusing/ChasingEventsSpec.scala
@@ -15,9 +15,9 @@ import pekko.stream.stage.OutHandler
 import pekko.stream.testkit.TestPublisher
 import pekko.stream.testkit.TestSubscriber
 import pekko.stream.testkit.Utils.TE
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
-class ChasingEventsSpec extends AkkaSpec("""
+class ChasingEventsSpec extends PekkoSpec("""
     pekko.stream.materializer.debug.fuzzing-mode = off
   """) {
 
diff --git a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/io/DeprecatedTlsSpec.scala b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/io/DeprecatedTlsSpec.scala
index bdcdb27570..a4113e7bd5 100644
--- a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/io/DeprecatedTlsSpec.scala
+++ b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/io/DeprecatedTlsSpec.scala
@@ -17,7 +17,7 @@ import scala.concurrent.Future
 import scala.concurrent.duration._
 import scala.util.Random
 
-import com.typesafe.sslconfig.pekko.AkkaSSLConfig
+import com.typesafe.sslconfig.pekko.PekkoSSLConfig
 
 import org.apache.pekko
 import pekko.NotUsed
@@ -105,7 +105,7 @@ class DeprecatedTlsSpec extends StreamSpec(DeprecatedTlsSpec.configOverrides) wi
   import GraphDSL.Implicits._
   import system.dispatcher
 
-  val sslConfig: Option[AkkaSSLConfig] = None // no special settings to be applied here
+  val sslConfig: Option[PekkoSSLConfig] = None // no special settings to be applied here
 
   "SslTls with deprecated SSLContext setup" must {
 
diff --git a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/io/TcpSpec.scala b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/io/TcpSpec.scala
index 3c93f6c479..884c21b241 100644
--- a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/io/TcpSpec.scala
+++ b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/io/TcpSpec.scala
@@ -1012,13 +1012,13 @@ class TcpSpec extends StreamSpec("""
       import java.security.KeyStore
       import javax.net.ssl._
 
-      import com.typesafe.sslconfig.pekko.AkkaSSLConfig
+      import com.typesafe.sslconfig.pekko.PekkoSSLConfig
 
       import org.apache.pekko
       import pekko.stream.TLSClientAuth
       import pekko.stream.TLSProtocol
 
-      val sslConfig = AkkaSSLConfig(system)
+      val sslConfig = PekkoSSLConfig(system)
 
       // Don't hardcode your password in actual code
       val password = "abcdef".toCharArray
diff --git a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/AggregateWithBoundarySpec.scala b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/AggregateWithBoundarySpec.scala
index 9ce391e7bb..079da149a9 100644
--- a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/AggregateWithBoundarySpec.scala
+++ b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/AggregateWithBoundarySpec.scala
@@ -16,7 +16,7 @@ import org.apache.pekko
 import pekko.actor.ActorSystem
 import pekko.stream.OverflowStrategy
 import pekko.stream.testkit.{ StreamSpec, TestPublisher, TestSubscriber }
-import pekko.testkit.{ AkkaSpec, ExplicitlyTriggeredScheduler }
+import pekko.testkit.{ ExplicitlyTriggeredScheduler, PekkoSpec }
 
 class AggregateWithBoundarySpec extends StreamSpec {
 
@@ -76,7 +76,7 @@ class AggregateWithTimeBoundaryAndSimulatedTimeSpec extends AnyWordSpecLike with
     ActorSystem(
       s"ActorSystemWithExplicitlyTriggeredScheduler-$id",
       ConfigFactory.load(
-        AkkaSpec.testConf.withValue(
+        PekkoSpec.testConf.withValue(
           "pekko.scheduler.implementation",
           ConfigValueFactory.fromAnyRef("org.apache.pekko.testkit.ExplicitlyTriggeredScheduler"))))
 
diff --git a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/FlowOrElseSpec.scala b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/FlowOrElseSpec.scala
index 26cf3b0c32..84642f5e32 100644
--- a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/FlowOrElseSpec.scala
+++ b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/FlowOrElseSpec.scala
@@ -11,9 +11,9 @@ import org.apache.pekko
 import pekko.stream.testkit.TestPublisher
 import pekko.stream.testkit.TestSubscriber
 import pekko.stream.testkit.Utils.TE
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
-class FlowOrElseSpec extends AkkaSpec {
+class FlowOrElseSpec extends PekkoSpec {
 
   "An OrElse flow" should {
 
diff --git a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/FlowPrependSpec.scala b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/FlowPrependSpec.scala
index 3b14a8a3f3..056586aec5 100644
--- a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/FlowPrependSpec.scala
+++ b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/FlowPrependSpec.scala
@@ -4,11 +4,11 @@
 
 package org.apache.pekko.stream.scaladsl
 
-import org.apache.pekko.testkit.AkkaSpec
+import org.apache.pekko.testkit.PekkoSpec
 import scala.annotation.nowarn
 
 @nowarn // for keeping imports
-class FlowPrependSpec extends AkkaSpec {
+class FlowPrependSpec extends PekkoSpec {
 
 //#prepend
 
diff --git a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/JsonFramingSpec.scala b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/JsonFramingSpec.scala
index 0d0e0f539f..76b5e23b04 100644
--- a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/JsonFramingSpec.scala
+++ b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/JsonFramingSpec.scala
@@ -14,10 +14,10 @@ import pekko.stream.scaladsl.Framing.FramingException
 import pekko.stream.scaladsl.JsonFraming.PartialObjectException
 import pekko.stream.testkit.{ TestPublisher, TestSubscriber }
 import pekko.stream.testkit.scaladsl.TestSink
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 import pekko.util.ByteString
 
-class JsonFramingSpec extends AkkaSpec {
+class JsonFramingSpec extends PekkoSpec {
 
   "collecting multiple json" should {
     "parse json array" in {
diff --git a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/QueueSourceSpec.scala b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/QueueSourceSpec.scala
index 6324a71a76..0b30d2fce8 100644
--- a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/QueueSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/QueueSourceSpec.scala
@@ -27,7 +27,7 @@ class QueueSourceSpec extends StreamSpec {
   implicit val ec: ExecutionContextExecutor = system.dispatcher
   val pause = 300.millis
 
-  // more frequent checks than defaults from AkkaSpec
+  // more frequent checks than defaults from PekkoSpec
   implicit val testPatience: PatienceConfig = {
     import org.apache.pekko.testkit.TestDuration
     PatienceConfig(testKitSettings.DefaultTimeout.duration.dilated, Span(5, org.scalatest.time.Millis))
diff --git a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/StreamRefsSpec.scala b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/StreamRefsSpec.scala
index 7617653718..44635ee001 100644
--- a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/StreamRefsSpec.scala
+++ b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/StreamRefsSpec.scala
@@ -22,7 +22,7 @@ import pekko.stream.impl.streamref.{ SinkRefImpl, SourceRefImpl }
 import pekko.stream.testkit.TestPublisher
 import pekko.stream.testkit.Utils.TE
 import pekko.stream.testkit.scaladsl._
-import pekko.testkit.{ AkkaSpec, TestKit, TestProbe }
+import pekko.testkit.{ PekkoSpec, TestKit, TestProbe }
 import pekko.util.ByteString
 
 object StreamRefsSpec {
@@ -197,7 +197,7 @@ object StreamRefsSpec {
   }
 }
 
-class StreamRefsSpec extends AkkaSpec(StreamRefsSpec.config()) {
+class StreamRefsSpec extends PekkoSpec(StreamRefsSpec.config()) {
   import StreamRefsSpec._
 
   val remoteSystem = ActorSystem("RemoteSystem", StreamRefsSpec.config())
diff --git a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/snapshot/MaterializerStateSpec.scala b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/snapshot/MaterializerStateSpec.scala
index d1647fdf5f..e50586b60b 100644
--- a/akka-stream-tests/src/test/scala/org/apache/pekko/stream/snapshot/MaterializerStateSpec.scala
+++ b/akka-stream-tests/src/test/scala/org/apache/pekko/stream/snapshot/MaterializerStateSpec.scala
@@ -13,9 +13,9 @@ import org.apache.pekko
 import pekko.stream.{ FlowShape, Materializer }
 import pekko.stream.scaladsl.{ Flow, GraphDSL, Keep, Merge, Partition, Sink, Source, Tcp }
 import pekko.stream.testkit.scaladsl.TestSink
-import pekko.testkit.AkkaSpec
+import pekko.testkit.PekkoSpec
 
-class MaterializerStateSpec extends AkkaSpec() {
+class MaterializerStateSpec extends PekkoSpec() {
 
   "The MaterializerSnapshotting" must {
 
diff --git a/akka-stream/src/main/resources/reference.conf b/akka-stream/src/main/resources/reference.conf
index 124f98003d..445e1fb653 100644
--- a/akka-stream/src/main/resources/reference.conf
+++ b/akka-stream/src/main/resources/reference.conf
@@ -196,5 +196,5 @@ pekko {
 # ssl configuration
 # folded in from former ssl-config-pekko module
 ssl-config {
-  logger = "com.typesafe.sslconfig.pekko.util.AkkaLoggerBridge"
+  logger = "com.typesafe.sslconfig.pekko.util.PekkoLoggerBridge"
 }
diff --git a/akka-stream/src/main/scala/com/typesafe/sslconfig/pekko/AkkaSSLConfig.scala b/akka-stream/src/main/scala/com/typesafe/sslconfig/pekko/PekkoSSLConfig.scala
similarity index 80%
rename from akka-stream/src/main/scala/com/typesafe/sslconfig/pekko/AkkaSSLConfig.scala
rename to akka-stream/src/main/scala/com/typesafe/sslconfig/pekko/PekkoSSLConfig.scala
index fc06bedd3c..e5af681a97 100644
--- a/akka-stream/src/main/scala/com/typesafe/sslconfig/pekko/AkkaSSLConfig.scala
+++ b/akka-stream/src/main/scala/com/typesafe/sslconfig/pekko/PekkoSSLConfig.scala
@@ -6,7 +6,7 @@ package com.typesafe.sslconfig.pekko
 
 import java.util.Collections
 import javax.net.ssl._
-import com.typesafe.sslconfig.pekko.util.AkkaLoggerFactory
+import com.typesafe.sslconfig.pekko.util.PekkoLoggerFactory
 import com.typesafe.sslconfig.ssl._
 import com.typesafe.sslconfig.util.LoggerFactory
 import org.apache.pekko
@@ -16,18 +16,18 @@ import pekko.event.Logging
 import scala.annotation.nowarn
 
 @deprecated("Use Tcp and TLS with SSLEngine parameters instead. Setup the SSLEngine with needed parameters.", "2.6.0")
-object AkkaSSLConfig extends ExtensionId[AkkaSSLConfig] with ExtensionIdProvider {
+object PekkoSSLConfig extends ExtensionId[PekkoSSLConfig] with ExtensionIdProvider {
 
   //////////////////// EXTENSION SETUP ///////////////////
 
-  override def get(system: ActorSystem): AkkaSSLConfig = super.get(system)
-  override def get(system: ClassicActorSystemProvider): AkkaSSLConfig = super.get(system)
-  def apply()(implicit system: ActorSystem): AkkaSSLConfig = super.apply(system)
+  override def get(system: ActorSystem): PekkoSSLConfig = super.get(system)
+  override def get(system: ClassicActorSystemProvider): PekkoSSLConfig = super.get(system)
+  def apply()(implicit system: ActorSystem): PekkoSSLConfig = super.apply(system)
 
-  override def lookup = AkkaSSLConfig
+  override def lookup = PekkoSSLConfig
 
-  override def createExtension(system: ExtendedActorSystem): AkkaSSLConfig =
-    new AkkaSSLConfig(system, defaultSSLConfigSettings(system))
+  override def createExtension(system: ExtendedActorSystem): PekkoSSLConfig =
+    new PekkoSSLConfig(system, defaultSSLConfigSettings(system))
 
   def defaultSSLConfigSettings(system: ActorSystem): SSLConfigSettings = {
     val akkaOverrides = system.settings.config.getConfig("pekko.ssl-config")
@@ -38,35 +38,35 @@ object AkkaSSLConfig extends ExtensionId[AkkaSSLConfig] with ExtensionIdProvider
 }
 
 @deprecated("Use Tcp and TLS with SSLEngine parameters instead. Setup the SSLEngine with needed parameters.", "2.6.0")
-final class AkkaSSLConfig(system: ExtendedActorSystem, val config: SSLConfigSettings) extends Extension {
+final class PekkoSSLConfig(system: ExtendedActorSystem, val config: SSLConfigSettings) extends Extension {
 
-  private val mkLogger = new AkkaLoggerFactory(system)
+  private val mkLogger = new PekkoLoggerFactory(system)
 
-  private val log = Logging(system, classOf[AkkaSSLConfig])
-  log.debug("Initializing AkkaSSLConfig extension...")
+  private val log = Logging(system, classOf[PekkoSSLConfig])
+  log.debug("Initializing PekkoSSLConfig extension...")
 
   /** Can be used to modify the underlying config, most typically used to change a few values in the default config */
-  def withSettings(c: SSLConfigSettings): AkkaSSLConfig =
-    new AkkaSSLConfig(system, c)
+  def withSettings(c: SSLConfigSettings): PekkoSSLConfig =
+    new PekkoSSLConfig(system, c)
 
   /**
-   * Returns a new [[AkkaSSLConfig]] instance with the settings changed by the given function.
+   * Returns a new [[PekkoSSLConfig]] instance with the settings changed by the given function.
    * Please note that the ActorSystem-wide extension always remains configured via typesafe config,
    * custom ones can be created for special-handling specific connections
    */
-  def mapSettings(f: SSLConfigSettings => SSLConfigSettings): AkkaSSLConfig =
-    new AkkaSSLConfig(system, f(config))
+  def mapSettings(f: SSLConfigSettings => SSLConfigSettings): PekkoSSLConfig =
+    new PekkoSSLConfig(system, f(config))
 
   /**
-   * Returns a new [[AkkaSSLConfig]] instance with the settings changed by the given function.
+   * Returns a new [[PekkoSSLConfig]] instance with the settings changed by the given function.
    * Please note that the ActorSystem-wide extension always remains configured via typesafe config,
    * custom ones can be created for special-handling specific connections
    *
    * Java API
    */
   // Not same signature as mapSettings to allow latter deprecation of this once we hit Scala 2.12
-  def convertSettings(f: java.util.function.Function[SSLConfigSettings, SSLConfigSettings]): AkkaSSLConfig =
-    new AkkaSSLConfig(system, f.apply(config))
+  def convertSettings(f: java.util.function.Function[SSLConfigSettings, SSLConfigSettings]): PekkoSSLConfig =
+    new PekkoSSLConfig(system, f.apply(config))
 
   val hostnameVerifier = buildHostnameVerifier(config)
 
@@ -170,7 +170,7 @@ final class AkkaSSLConfig(system: ExtendedActorSystem, val config: SSLConfigSett
   // LOOSE SETTINGS //
 
   private def looseDisableSNI(defaultParams: SSLParameters): Unit = if (config.loose.disableSNI) {
-    // this will be logged once for each AkkaSSLConfig
+    // this will be logged once for each PekkoSSLConfig
     log.warning(
       "You are using ssl-config.loose.disableSNI=true! " +
       "It is strongly discouraged to disable Server Name Indication, as it is crucial to preventing man-in-the-middle attacks.")
diff --git a/akka-stream/src/main/scala/com/typesafe/sslconfig/pekko/util/AkkaLoggerBridge.scala b/akka-stream/src/main/scala/com/typesafe/sslconfig/pekko/util/PekkoLoggerBridge.scala
similarity index 72%
rename from akka-stream/src/main/scala/com/typesafe/sslconfig/pekko/util/AkkaLoggerBridge.scala
rename to akka-stream/src/main/scala/com/typesafe/sslconfig/pekko/util/PekkoLoggerBridge.scala
index b3b397037c..3d9c36fde0 100644
--- a/akka-stream/src/main/scala/com/typesafe/sslconfig/pekko/util/AkkaLoggerBridge.scala
+++ b/akka-stream/src/main/scala/com/typesafe/sslconfig/pekko/util/PekkoLoggerBridge.scala
@@ -11,14 +11,14 @@ import pekko.actor.ActorSystem
 import pekko.event.{ DummyClassForStringSources, EventStream }
 import pekko.event.Logging._
 
-final class AkkaLoggerFactory(system: ActorSystem) extends LoggerFactory {
-  override def apply(clazz: Class[_]): NoDepsLogger = new AkkaLoggerBridge(system.eventStream, clazz)
+final class PekkoLoggerFactory(system: ActorSystem) extends LoggerFactory {
+  override def apply(clazz: Class[_]): NoDepsLogger = new PekkoLoggerBridge(system.eventStream, clazz)
 
   override def apply(name: String): NoDepsLogger =
-    new AkkaLoggerBridge(system.eventStream, name, classOf[DummyClassForStringSources])
+    new PekkoLoggerBridge(system.eventStream, name, classOf[DummyClassForStringSources])
 }
 
-class AkkaLoggerBridge(bus: EventStream, logSource: String, logClass: Class[_]) extends NoDepsLogger {
+class PekkoLoggerBridge(bus: EventStream, logSource: String, logClass: Class[_]) extends NoDepsLogger {
   def this(bus: EventStream, clazz: Class[_]) = this(bus, clazz.getCanonicalName, clazz)
 
   override def isDebugEnabled: Boolean = true
diff --git a/akka-stream/src/main/scala/org/apache/pekko/stream/impl/io/TLSActor.scala b/akka-stream/src/main/scala/org/apache/pekko/stream/impl/io/TLSActor.scala
index 4b450f3e68..33f06580c9 100644
--- a/akka-stream/src/main/scala/org/apache/pekko/stream/impl/io/TLSActor.scala
+++ b/akka-stream/src/main/scala/org/apache/pekko/stream/impl/io/TLSActor.scala
@@ -34,8 +34,8 @@ import pekko.util.ByteString
 
   def props(
       maxInputBufferSize: Int,
-      createSSLEngine: ActorSystem => SSLEngine, // ActorSystem is only needed to support the AkkaSSLConfig legacy, see #21753
-      verifySession: (ActorSystem, SSLSession) => Try[Unit], // ActorSystem is only needed to support the AkkaSSLConfig legacy, see #21753
+      createSSLEngine: ActorSystem => SSLEngine, // ActorSystem is only needed to support the PekkoSSLConfig legacy, see #21753
+      verifySession: (ActorSystem, SSLSession) => Try[Unit], // ActorSystem is only needed to support the PekkoSSLConfig legacy, see #21753
       closing: TLSClosing,
       tracing: Boolean = false): Props =
     Props(new TLSActor(maxInputBufferSize, createSSLEngine, verifySession, closing, tracing)).withDeploy(Deploy.local)
@@ -52,8 +52,8 @@ import pekko.util.ByteString
  */
 @InternalApi private[stream] class TLSActor(
     maxInputBufferSize: Int,
-    createSSLEngine: ActorSystem => SSLEngine, // ActorSystem is only needed to support the AkkaSSLConfig legacy, see #21753
-    verifySession: (ActorSystem, SSLSession) => Try[Unit], // ActorSystem is only needed to support the AkkaSSLConfig legacy, see #21753
+    createSSLEngine: ActorSystem => SSLEngine, // ActorSystem is only needed to support the PekkoSSLConfig legacy, see #21753
+    verifySession: (ActorSystem, SSLSession) => Try[Unit], // ActorSystem is only needed to support the PekkoSSLConfig legacy, see #21753
     closing: TLSClosing,
     tracing: Boolean)
     extends Actor
diff --git a/akka-stream/src/main/scala/org/apache/pekko/stream/impl/io/TlsModule.scala b/akka-stream/src/main/scala/org/apache/pekko/stream/impl/io/TlsModule.scala
index 41343e731c..eeac848d11 100644
--- a/akka-stream/src/main/scala/org/apache/pekko/stream/impl/io/TlsModule.scala
+++ b/akka-stream/src/main/scala/org/apache/pekko/stream/impl/io/TlsModule.scala
@@ -28,8 +28,8 @@ import pekko.util.ByteString
     cipherOut: Outlet[ByteString],
     shape: BidiShape[SslTlsOutbound, ByteString, ByteString, SslTlsInbound],
     attributes: Attributes,
-    createSSLEngine: ActorSystem => SSLEngine, // ActorSystem is only needed to support the AkkaSSLConfig legacy, see #21753
-    verifySession: (ActorSystem, SSLSession) => Try[Unit], // ActorSystem is only needed to support the AkkaSSLConfig legacy, see #21753
+    createSSLEngine: ActorSystem => SSLEngine, // ActorSystem is only needed to support the PekkoSSLConfig legacy, see #21753
+    verifySession: (ActorSystem, SSLSession) => Try[Unit], // ActorSystem is only needed to support the PekkoSSLConfig legacy, see #21753
     closing: TLSClosing)
     extends AtomicModule[BidiShape[SslTlsOutbound, ByteString, ByteString, SslTlsInbound], NotUsed] {
 
@@ -47,8 +47,8 @@ import pekko.util.ByteString
 @InternalApi private[stream] object TlsModule {
   def apply(
       attributes: Attributes,
-      createSSLEngine: ActorSystem => SSLEngine, // ActorSystem is only needed to support the AkkaSSLConfig legacy, see #21753
-      verifySession: (ActorSystem, SSLSession) => Try[Unit], // ActorSystem is only needed to support the AkkaSSLConfig legacy, see #21753
+      createSSLEngine: ActorSystem => SSLEngine, // ActorSystem is only needed to support the PekkoSSLConfig legacy, see #21753
+      verifySession: (ActorSystem, SSLSession) => Try[Unit], // ActorSystem is only needed to support the PekkoSSLConfig legacy, see #21753
       closing: TLSClosing): TlsModule = {
     val name = attributes.nameOrDefault(s"StreamTls()")
     val cipherIn = Inlet[ByteString](s"$name.cipherIn")
diff --git a/akka-stream/src/main/scala/org/apache/pekko/stream/javadsl/TLS.scala b/akka-stream/src/main/scala/org/apache/pekko/stream/javadsl/TLS.scala
index 0d5cfb401b..91066dd432 100644
--- a/akka-stream/src/main/scala/org/apache/pekko/stream/javadsl/TLS.scala
+++ b/akka-stream/src/main/scala/org/apache/pekko/stream/javadsl/TLS.scala
@@ -11,7 +11,7 @@ import javax.net.ssl.{ SSLContext, SSLEngine, SSLSession }
 import scala.compat.java8.OptionConverters
 import scala.util.Try
 
-import com.typesafe.sslconfig.pekko.AkkaSSLConfig
+import com.typesafe.sslconfig.pekko.PekkoSSLConfig
 
 import org.apache.pekko
 import pekko.{ japi, NotUsed }
@@ -70,7 +70,7 @@ object TLS {
   @deprecated("Use create that takes a SSLEngine factory instead. Setup the SSLEngine with needed parameters.", "2.6.0")
   def create(
       sslContext: SSLContext,
-      sslConfig: Optional[AkkaSSLConfig],
+      sslConfig: Optional[PekkoSSLConfig],
       firstSession: NegotiateNewSession,
       role: TLSRole): BidiFlow[SslTlsOutbound, ByteString, ByteString, SslTlsInbound, NotUsed] =
     new javadsl.BidiFlow(scaladsl.TLS.apply(sslContext, OptionConverters.toScala(sslConfig), firstSession, role))
@@ -112,7 +112,7 @@ object TLS {
   @deprecated("Use create that takes a SSLEngine factory instead. Setup the SSLEngine with needed parameters.", "2.6.0")
   def create(
       sslContext: SSLContext,
-      sslConfig: Optional[AkkaSSLConfig],
+      sslConfig: Optional[PekkoSSLConfig],
       firstSession: NegotiateNewSession,
       role: TLSRole,
       hostInfo: Optional[japi.Pair[String, java.lang.Integer]],
diff --git a/akka-stream/src/main/scala/org/apache/pekko/stream/scaladsl/TLS.scala b/akka-stream/src/main/scala/org/apache/pekko/stream/scaladsl/TLS.scala
index b7d26d61b4..effc0ea7e9 100644
--- a/akka-stream/src/main/scala/org/apache/pekko/stream/scaladsl/TLS.scala
+++ b/akka-stream/src/main/scala/org/apache/pekko/stream/scaladsl/TLS.scala
@@ -10,7 +10,7 @@ import javax.net.ssl.SSLParameters
 
 import scala.util.{ Failure, Success, Try }
 
-import com.typesafe.sslconfig.pekko.AkkaSSLConfig
+import com.typesafe.sslconfig.pekko.PekkoSSLConfig
 
 import org.apache.pekko
 import pekko.NotUsed
@@ -76,14 +76,14 @@ object TLS {
   @deprecated("Use apply that takes a SSLEngine factory instead. Setup the SSLEngine with needed parameters.", "2.6.0")
   def apply(
       sslContext: SSLContext,
-      sslConfig: Option[AkkaSSLConfig],
+      sslConfig: Option[PekkoSSLConfig],
       firstSession: NegotiateNewSession,
       role: TLSRole,
       closing: TLSClosing = IgnoreComplete,
       hostInfo: Option[(String, Int)] = None)
       : scaladsl.BidiFlow[SslTlsOutbound, ByteString, ByteString, SslTlsInbound, NotUsed] = {
-    def theSslConfig(system: ActorSystem): AkkaSSLConfig =
-      sslConfig.getOrElse(AkkaSSLConfig(system))
+    def theSslConfig(system: ActorSystem): PekkoSSLConfig =
+      sslConfig.getOrElse(PekkoSSLConfig(system))
 
     val createSSLEngine = { (system: ActorSystem) =>
       val config = theSslConfig(system)
diff --git a/akka-testkit/src/test/java/org/apache/pekko/testkit/AkkaJUnitActorSystemResource.java b/akka-testkit/src/test/java/org/apache/pekko/testkit/PekkoJUnitActorSystemResource.java
similarity index 79%
rename from akka-testkit/src/test/java/org/apache/pekko/testkit/AkkaJUnitActorSystemResource.java
rename to akka-testkit/src/test/java/org/apache/pekko/testkit/PekkoJUnitActorSystemResource.java
index d748543de8..5da219529a 100644
--- a/akka-testkit/src/test/java/org/apache/pekko/testkit/AkkaJUnitActorSystemResource.java
+++ b/akka-testkit/src/test/java/org/apache/pekko/testkit/PekkoJUnitActorSystemResource.java
@@ -15,14 +15,14 @@ import org.junit.rules.ExternalResource;
  *
  * 

To use it on a class level add this to your test class: * @ClassRule - * public static AkkaJUnitActorSystemResource actorSystemResource = - * new AkkaJUnitActorSystemResource(name, config); + * public static PekkoJUnitActorSystemResource actorSystemResource = + * new PekkoJUnitActorSystemResource(name, config); * * private final ActorSystem system = actorSystemResource.getSystem(); * To use it on a per test level add this to your test class: * @Rule - * public AkkaJUnitActorSystemResource actorSystemResource = - * new AkkaJUnitActorSystemResource(name, config); + * public PekkoJUnitActorSystemResource actorSystemResource = + * new PekkoJUnitActorSystemResource(name, config); * * private ActorSystem system = null; * @@ -34,7 +34,7 @@ import org.junit.rules.ExternalResource; * test, because some test runners may create an instance of the class without actually using it * later, resulting in memory leaks because of not shutting down the actor system. */ -public class AkkaJUnitActorSystemResource extends ExternalResource { +public class PekkoJUnitActorSystemResource extends ExternalResource { private ActorSystem system = null; private final String name; private final Config config; @@ -49,13 +49,13 @@ public class AkkaJUnitActorSystemResource extends ExternalResource { } } - public AkkaJUnitActorSystemResource(String name, Config config) { + public PekkoJUnitActorSystemResource(String name, Config config) { this.name = name; this.config = config; } - public AkkaJUnitActorSystemResource(String name) { - this(name, AkkaSpec.testConf()); + public PekkoJUnitActorSystemResource(String name) { + this(name, PekkoSpec.testConf()); } @Override diff --git a/akka-testkit/src/test/scala/org/apache/pekko/testkit/AbstractSpec.scala b/akka-testkit/src/test/scala/org/apache/pekko/testkit/AbstractSpec.scala index 10007b81ec..4514f1712c 100644 --- a/akka-testkit/src/test/scala/org/apache/pekko/testkit/AbstractSpec.scala +++ b/akka-testkit/src/test/scala/org/apache/pekko/testkit/AbstractSpec.scala @@ -8,5 +8,5 @@ import org.scalatest.BeforeAndAfterEach import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpecLike -// we could migrate AkkaSpec to extend this +// we could migrate PekkoSpec to extend this abstract class AbstractSpec extends AnyWordSpecLike with Matchers with BeforeAndAfterEach diff --git a/akka-testkit/src/test/scala/org/apache/pekko/testkit/DefaultTimeoutSpec.scala b/akka-testkit/src/test/scala/org/apache/pekko/testkit/DefaultTimeoutSpec.scala index 5aa2711df3..e4e07946ae 100644 --- a/akka-testkit/src/test/scala/org/apache/pekko/testkit/DefaultTimeoutSpec.scala +++ b/akka-testkit/src/test/scala/org/apache/pekko/testkit/DefaultTimeoutSpec.scala @@ -12,7 +12,7 @@ import org.apache.pekko.actor.ActorSystem class DefaultTimeoutSpec extends AnyWordSpec with Matchers with BeforeAndAfterAll with TestKitBase with DefaultTimeout { - implicit lazy val system: ActorSystem = ActorSystem("AkkaCustomSpec") + implicit lazy val system: ActorSystem = ActorSystem("PekkoCustomSpec") override def afterAll() = system.terminate() diff --git a/akka-testkit/src/test/scala/org/apache/pekko/testkit/ExplicitlyTriggeredSchedulerSpec.scala b/akka-testkit/src/test/scala/org/apache/pekko/testkit/ExplicitlyTriggeredSchedulerSpec.scala index 094f1531c3..a5b10f9fe4 100644 --- a/akka-testkit/src/test/scala/org/apache/pekko/testkit/ExplicitlyTriggeredSchedulerSpec.scala +++ b/akka-testkit/src/test/scala/org/apache/pekko/testkit/ExplicitlyTriggeredSchedulerSpec.scala @@ -8,7 +8,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.concurrent.duration.DurationInt -class ExplicitlyTriggeredSchedulerSpec extends AkkaSpec { +class ExplicitlyTriggeredSchedulerSpec extends PekkoSpec { "ExplicitlyTriggeredScheduler" must { diff --git a/akka-testkit/src/test/scala/org/apache/pekko/testkit/ImplicitSenderSpec.scala b/akka-testkit/src/test/scala/org/apache/pekko/testkit/ImplicitSenderSpec.scala index 5e1d4d039d..712afc5bb6 100644 --- a/akka-testkit/src/test/scala/org/apache/pekko/testkit/ImplicitSenderSpec.scala +++ b/akka-testkit/src/test/scala/org/apache/pekko/testkit/ImplicitSenderSpec.scala @@ -12,7 +12,7 @@ import org.apache.pekko.actor.ActorSystem class ImplicitSenderSpec extends AnyWordSpec with Matchers with BeforeAndAfterAll with TestKitBase with ImplicitSender { - implicit lazy val system: ActorSystem = ActorSystem("AkkaCustomSpec") + implicit lazy val system: ActorSystem = ActorSystem("PekkoCustomSpec") override def afterAll() = system.terminate() diff --git a/akka-testkit/src/test/scala/org/apache/pekko/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/org/apache/pekko/testkit/PekkoSpec.scala similarity index 90% rename from akka-testkit/src/test/scala/org/apache/pekko/testkit/AkkaSpec.scala rename to akka-testkit/src/test/scala/org/apache/pekko/testkit/PekkoSpec.scala index 59716344d4..8925b75167 100644 --- a/akka-testkit/src/test/scala/org/apache/pekko/testkit/AkkaSpec.scala +++ b/akka-testkit/src/test/scala/org/apache/pekko/testkit/PekkoSpec.scala @@ -26,7 +26,7 @@ import pekko.event.Logging import pekko.event.LoggingAdapter import pekko.testkit.TestEvent._ -object AkkaSpec { +object PekkoSpec { val testConf: Config = ConfigFactory.parseString(""" pekko { loggers = ["org.apache.pekko.testkit.TestEventListener"] @@ -52,7 +52,7 @@ object AkkaSpec { } -abstract class AkkaSpec(_system: ActorSystem) +abstract class PekkoSpec(_system: ActorSystem) extends TestKit(_system) with AnyWordSpecLike with Matchers @@ -64,14 +64,14 @@ abstract class AkkaSpec(_system: ActorSystem) def this(config: Config) = this( ActorSystem( - TestKitUtils.testNameFromCallStack(classOf[AkkaSpec], "".r), - ConfigFactory.load(config.withFallback(AkkaSpec.testConf)))) + TestKitUtils.testNameFromCallStack(classOf[PekkoSpec], "".r), + ConfigFactory.load(config.withFallback(PekkoSpec.testConf)))) def this(s: String) = this(ConfigFactory.parseString(s)) - def this(configMap: Map[String, _]) = this(AkkaSpec.mapToConfig(configMap)) + def this(configMap: Map[String, _]) = this(PekkoSpec.mapToConfig(configMap)) - def this() = this(ActorSystem(TestKitUtils.testNameFromCallStack(classOf[AkkaSpec], "".r), AkkaSpec.testConf)) + def this() = this(ActorSystem(TestKitUtils.testNameFromCallStack(classOf[PekkoSpec], "".r), PekkoSpec.testConf)) implicit val patience: PatienceConfig = PatienceConfig(testKitSettings.SingleExpectDefaultTimeout.dilated, Span(100, Millis)) diff --git a/akka-testkit/src/test/scala/org/apache/pekko/testkit/AkkaSpecSpec.scala b/akka-testkit/src/test/scala/org/apache/pekko/testkit/PekkoSpecSpec.scala similarity index 84% rename from akka-testkit/src/test/scala/org/apache/pekko/testkit/AkkaSpecSpec.scala rename to akka-testkit/src/test/scala/org/apache/pekko/testkit/PekkoSpecSpec.scala index 03bbd9dc2a..a8f2fdfcf6 100644 --- a/akka-testkit/src/test/scala/org/apache/pekko/testkit/AkkaSpecSpec.scala +++ b/akka-testkit/src/test/scala/org/apache/pekko/testkit/PekkoSpecSpec.scala @@ -21,12 +21,12 @@ import pekko.pattern.ask import pekko.util.Timeout @nowarn -class AkkaSpecSpec extends AnyWordSpec with Matchers { +class PekkoSpecSpec extends AnyWordSpec with Matchers { - "An AkkaSpec" must { + "An PekkoSpec" must { "warn about unhandled messages" in { - implicit val system = ActorSystem("AkkaSpec0", AkkaSpec.testConf) + implicit val system = ActorSystem("PekkoSpec0", PekkoSpec.testConf) try { val a = system.actorOf(Props.empty) EventFilter.warning(start = "unhandled message", occurrences = 1).intercept { @@ -45,17 +45,17 @@ class AkkaSpecSpec extends AnyWordSpec with Matchers { "pekko.actor.debug.event-stream" -> true, "pekko.loglevel" -> "DEBUG", "pekko.stdout-loglevel" -> "DEBUG") - val localSystem = ActorSystem("AkkaSpec1", ConfigFactory.parseMap(conf.asJava).withFallback(AkkaSpec.testConf)) + val localSystem = ActorSystem("PekkoSpec1", ConfigFactory.parseMap(conf.asJava).withFallback(PekkoSpec.testConf)) var refs = Seq.empty[ActorRef] - val spec = new AkkaSpec(localSystem) { refs = Seq(testActor, localSystem.actorOf(Props.empty, "name")) } + val spec = new PekkoSpec(localSystem) { refs = Seq(testActor, localSystem.actorOf(Props.empty, "name")) } refs.foreach(_.isTerminated should not be true) TestKit.shutdownActorSystem(localSystem) spec.awaitCond(refs.forall(_.isTerminated), 2 seconds) } "stop correctly when sending PoisonPill to rootGuardian" in { - val system = ActorSystem("AkkaSpec2", AkkaSpec.testConf) - new AkkaSpec(system) {} + val system = ActorSystem("PekkoSpec2", PekkoSpec.testConf) + new PekkoSpec(system) {} val latch = new TestLatch(1)(system) system.registerOnTermination(latch.countDown()) @@ -65,7 +65,7 @@ class AkkaSpecSpec extends AnyWordSpec with Matchers { } "enqueue unread messages from testActor to deadLetters" in { - val system, otherSystem = ActorSystem("AkkaSpec3", AkkaSpec.testConf) + val system, otherSystem = ActorSystem("PekkoSpec3", PekkoSpec.testConf) try { var locker = Seq.empty[DeadLetter] diff --git a/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestActorRefSpec.scala index eeb88c52a9..b9f801dcf9 100644 --- a/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestActorRefSpec.scala @@ -109,7 +109,7 @@ object TestActorRefSpec { } -class TestActorRefSpec extends AkkaSpec("disp1.type=Dispatcher") with BeforeAndAfterEach with DefaultTimeout { +class TestActorRefSpec extends PekkoSpec("disp1.type=Dispatcher") with BeforeAndAfterEach with DefaultTimeout { import TestActorRefSpec._ diff --git a/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestActorsSpec.scala b/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestActorsSpec.scala index 3089309941..9d6a86d9d8 100644 --- a/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestActorsSpec.scala +++ b/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestActorsSpec.scala @@ -4,7 +4,7 @@ package org.apache.pekko.testkit -class TestActorsSpec extends AkkaSpec with ImplicitSender { +class TestActorsSpec extends PekkoSpec with ImplicitSender { import TestActors.{ echoActorProps, forwardActorProps } diff --git a/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestEventListenerSpec.scala b/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestEventListenerSpec.scala index 8dec90b52d..cd776fa6aa 100644 --- a/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestEventListenerSpec.scala +++ b/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestEventListenerSpec.scala @@ -9,7 +9,7 @@ import pekko.event.Logging import pekko.event.Logging.Error import pekko.event.Logging.Warning -class TestEventListenerSpec extends AkkaSpec with ImplicitSender { +class TestEventListenerSpec extends PekkoSpec with ImplicitSender { "The classic EventFilter.error" must { "filter errors without cause" in { diff --git a/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestFSMRefSpec.scala b/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestFSMRefSpec.scala index 4c2cfd239d..d7fcf7ecfe 100644 --- a/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestFSMRefSpec.scala +++ b/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestFSMRefSpec.scala @@ -10,7 +10,7 @@ import language.postfixOps import org.apache.pekko.actor._ -class TestFSMRefSpec extends AkkaSpec { +class TestFSMRefSpec extends PekkoSpec { "A TestFSMRef" must { diff --git a/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestProbeSpec.scala b/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestProbeSpec.scala index 87fc1a105f..d07b595d49 100644 --- a/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestProbeSpec.scala +++ b/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestProbeSpec.scala @@ -17,7 +17,7 @@ import org.apache.pekko import pekko.actor._ import pekko.pattern.ask -class TestProbeSpec extends AkkaSpec with DefaultTimeout with Eventually { +class TestProbeSpec extends PekkoSpec with DefaultTimeout with Eventually { "A TestProbe" must { diff --git a/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestTimeSpec.scala b/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestTimeSpec.scala index 6a94f1b377..f376bf3ad8 100644 --- a/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestTimeSpec.scala +++ b/akka-testkit/src/test/scala/org/apache/pekko/testkit/TestTimeSpec.scala @@ -8,7 +8,7 @@ import scala.concurrent.duration._ import org.scalatest.exceptions.TestFailedException -class TestTimeSpec extends AkkaSpec(Map("pekko.test.timefactor" -> 2.0)) { +class TestTimeSpec extends PekkoSpec(Map("pekko.test.timefactor" -> 2.0)) { "A TestKit" must { diff --git a/akka-testkit/src/test/scala/org/apache/pekko/testkit/metrics/MetricsKit.scala b/akka-testkit/src/test/scala/org/apache/pekko/testkit/metrics/MetricsKit.scala index e1fb7ab88c..bacca4fdd4 100644 --- a/akka-testkit/src/test/scala/org/apache/pekko/testkit/metrics/MetricsKit.scala +++ b/akka-testkit/src/test/scala/org/apache/pekko/testkit/metrics/MetricsKit.scala @@ -17,7 +17,7 @@ import com.typesafe.config.Config import org.scalatest.Notifying import org.apache.pekko -import pekko.testkit.metrics.reporter.AkkaConsoleReporter +import pekko.testkit.metrics.reporter.PekkoConsoleReporter /** * Allows to easily measure performance / memory / file descriptor use in tests. @@ -46,7 +46,7 @@ private[pekko] trait MetricsKit extends MetricsKitOps { */ def metricsConfig: Config - private[metrics] val registry = new MetricRegistry() with AkkaMetricRegistry + private[metrics] val registry = new MetricRegistry() with PekkoMetricRegistry initMetricReporters() @@ -55,7 +55,7 @@ private[pekko] trait MetricsKit extends MetricsKitOps { def configureConsoleReporter(): Unit = { if (settings.Reporters.contains("console")) { - val akkaConsoleReporter = new AkkaConsoleReporter(registry, settings.ConsoleReporter.Verbose) + val akkaConsoleReporter = new PekkoConsoleReporter(registry, settings.ConsoleReporter.Verbose) if (settings.ConsoleReporter.ScheduledReportInterval > Duration.Zero) akkaConsoleReporter.start(settings.ConsoleReporter.ScheduledReportInterval.toMillis, TimeUnit.MILLISECONDS) @@ -189,7 +189,7 @@ private[pekko] object MetricsKit { } /** Provides access to custom Akka `com.codahale.metrics.Metric`, with named methods. */ -trait AkkaMetricRegistry { +trait PekkoMetricRegistry { this: MetricRegistry => def getKnownOpsInTimespanCounters = filterFor(classOf[KnownOpsInTimespanTimer]) diff --git a/akka-testkit/src/test/scala/org/apache/pekko/testkit/metrics/reporter/AkkaConsoleReporter.scala b/akka-testkit/src/test/scala/org/apache/pekko/testkit/metrics/reporter/PekkoConsoleReporter.scala similarity index 98% rename from akka-testkit/src/test/scala/org/apache/pekko/testkit/metrics/reporter/AkkaConsoleReporter.scala rename to akka-testkit/src/test/scala/org/apache/pekko/testkit/metrics/reporter/PekkoConsoleReporter.scala index f3ad15c281..6677e17855 100644 --- a/akka-testkit/src/test/scala/org/apache/pekko/testkit/metrics/reporter/AkkaConsoleReporter.scala +++ b/akka-testkit/src/test/scala/org/apache/pekko/testkit/metrics/reporter/PekkoConsoleReporter.scala @@ -16,7 +16,7 @@ import pekko.testkit.metrics._ /** * Used to report `org.apache.pekko.testkit.metric.Metric` types that the original `com.codahale.metrics.ConsoleReporter` is unaware of (cannot re-use directly because of private constructor). */ -class AkkaConsoleReporter(registry: AkkaMetricRegistry, verbose: Boolean, output: PrintStream = System.out) +class PekkoConsoleReporter(registry: PekkoMetricRegistry, verbose: Boolean, output: PrintStream = System.out) extends ScheduledReporter( registry.asInstanceOf[MetricRegistry], "akka-console-reporter", diff --git a/build.sbt b/build.sbt index c4357a9e5c..62965b1063 100644 --- a/build.sbt +++ b/build.sbt @@ -23,7 +23,7 @@ addCommandAlias( addCommandAlias(name = "sortImports", value = ";scalafixEnable; scalafixAll SortImports; scalafmtAll") -import org.apache.pekko.AkkaBuild._ +import org.apache.pekko.PekkoBuild._ import com.typesafe.sbt.MultiJvmPlugin.MultiJvmKeys.MultiJvm import com.typesafe.tools.mima.plugin.MimaPlugin import sbt.Keys.{ initialCommands, parallelExecution } @@ -35,7 +35,7 @@ initialize := { initialize.value } -AkkaBuild.buildSettings +PekkoBuild.buildSettings shellPrompt := { s => Project.extract(s).currentProject.id + " > " } @@ -99,7 +99,7 @@ lazy val root = Project(id = "pekko", base = file(".")) docs, serialversionRemoverPlugin)) .settings(Compile / headerCreate / unmanagedSources := (baseDirectory.value / "project").**("*.scala").get) - .settings(AkkaBuild.welcomeSettings) + .settings(PekkoBuild.welcomeSettings) .enablePlugins(CopyrightHeaderForBuild) lazy val actor = akkaModule("akka-actor") @@ -231,7 +231,7 @@ lazy val docs = akkaModule("docs") streamTestkit % "compile->compile;test->test", persistenceTestkit % "compile->compile;test->test") .settings(Dependencies.docs) - .settings(AkkaDisciplinePlugin.docs) + .settings(PekkoDisciplinePlugin.docs) .settings(Paradox.settings) .settings(javacOptions += "-parameters") // for Jackson .enablePlugins( @@ -265,7 +265,7 @@ lazy val multiNodeTestkit = akkaModule("akka-multi-node-testkit") .settings(Dependencies.multiNodeTestkit) .settings(Protobuf.settings) .settings(AutomaticModuleName.settings("akka.remote.testkit")) - .settings(AkkaBuild.mayChangeSettings) + .settings(PekkoBuild.mayChangeSettings) lazy val osgi = akkaModule("akka-osgi") .dependsOn(actor) @@ -331,7 +331,7 @@ lazy val persistenceTypedTests = akkaModule("akka-persistence-typed-tests") actorTestkitTyped % "test", persistence % "test->test", // for SteppingInMemJournal jackson % "test->test") - .settings(AkkaBuild.mayChangeSettings) + .settings(PekkoBuild.mayChangeSettings) .settings(Dependencies.persistenceTypedTests) .settings(javacOptions += "-parameters") // for Jackson .disablePlugins(MimaPlugin) @@ -552,7 +552,7 @@ lazy val actorTestkitTyped = akkaModule("akka-actor-testkit-typed") lazy val actorTypedTests = akkaModule("akka-actor-typed-tests") .dependsOn(actorTyped % "compile->CompileJdk9", actorTestkitTyped % "compile->compile;test->test", actor) - .settings(AkkaBuild.mayChangeSettings) + .settings(PekkoBuild.mayChangeSettings) .disablePlugins(MimaPlugin) .enablePlugins(NoPublish) @@ -570,10 +570,10 @@ lazy val coordination = akkaModule("akka-coordination") lazy val billOfMaterials = Project("akka-bill-of-materials", file("akka-bill-of-materials")) .enablePlugins(BillOfMaterialsPlugin) - .disablePlugins(MimaPlugin, AkkaDisciplinePlugin) + .disablePlugins(MimaPlugin, PekkoDisciplinePlugin) // buildSettings and defaultSettings configure organization name, licenses, etc... - .settings(AkkaBuild.buildSettings) - .settings(AkkaBuild.defaultSettings) + .settings(PekkoBuild.buildSettings) + .settings(PekkoBuild.defaultSettings) .settings( name := "akka-bom", bomIncludeProjects := userProjects, @@ -597,8 +597,8 @@ def akkaModule(name: String): Project = Project(id = name, base = file(name)) .enablePlugins(ReproducibleBuildsPlugin) .disablePlugins(WelcomePlugin) - .settings(AkkaBuild.buildSettings) - .settings(AkkaBuild.defaultSettings) + .settings(PekkoBuild.buildSettings) + .settings(PekkoBuild.defaultSettings) .enablePlugins(BootstrapGenjavadoc) /* Command aliases one can run locally against a module diff --git a/docs/src/main/paradox/actors.md b/docs/src/main/paradox/actors.md index 04cbbe2262..843e9c7e3f 100644 --- a/docs/src/main/paradox/actors.md +++ b/docs/src/main/paradox/actors.md @@ -7,16 +7,16 @@ To use Classic Actors, add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion group2="com.typesafe.akka" artifact2="akka-testkit_$scala.binary.version$" scope2=test - version2=AkkaVersion + version2=PekkoVersion } @@project-info{ projectId="akka-actor" } diff --git a/docs/src/main/paradox/additional/osgi.md b/docs/src/main/paradox/additional/osgi.md index cdb587e290..e0ac04621c 100644 --- a/docs/src/main/paradox/additional/osgi.md +++ b/docs/src/main/paradox/additional/osgi.md @@ -5,12 +5,12 @@ To use Akka in OSGi, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-osgi_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } ## Background diff --git a/docs/src/main/paradox/cluster-client.md b/docs/src/main/paradox/cluster-client.md index f4a0e3a376..dc2ad27766 100644 --- a/docs/src/main/paradox/cluster-client.md +++ b/docs/src/main/paradox/cluster-client.md @@ -14,12 +14,12 @@ It is not advised to build new applications with Cluster Client, and existing us To use Cluster Client, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { -bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion -symbol1=AkkaVersion +bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion +symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-tools_$scala.binary.version$ -version=AkkaVersion +version=PekkoVersion } @@project-info{ projectId="akka-cluster-tools" } diff --git a/docs/src/main/paradox/cluster-metrics.md b/docs/src/main/paradox/cluster-metrics.md index 4e97738285..8dcf7389de 100644 --- a/docs/src/main/paradox/cluster-metrics.md +++ b/docs/src/main/paradox/cluster-metrics.md @@ -5,12 +5,12 @@ To use Cluster Metrics Extension, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-metrics_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } and add the following configuration stanza to your `application.conf` diff --git a/docs/src/main/paradox/cluster-routing.md b/docs/src/main/paradox/cluster-routing.md index fd0cf96f9d..cbd083f347 100644 --- a/docs/src/main/paradox/cluster-routing.md +++ b/docs/src/main/paradox/cluster-routing.md @@ -32,12 +32,12 @@ on other nodes in the cluster. To use Cluster aware routers, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-cluster_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Router with Group of Routees diff --git a/docs/src/main/paradox/cluster-sharding.md b/docs/src/main/paradox/cluster-sharding.md index 96894cf63c..3d0e9d6dd6 100644 --- a/docs/src/main/paradox/cluster-sharding.md +++ b/docs/src/main/paradox/cluster-sharding.md @@ -8,12 +8,12 @@ For the full documentation of this feature and for new projects see @ref:[Cluste To use Cluster Sharding, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-sharding_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } @@project-info{ projectId="akka-cluster-sharding" } diff --git a/docs/src/main/paradox/cluster-singleton.md b/docs/src/main/paradox/cluster-singleton.md index f024825ee8..6101b17c8a 100644 --- a/docs/src/main/paradox/cluster-singleton.md +++ b/docs/src/main/paradox/cluster-singleton.md @@ -8,12 +8,12 @@ For the full documentation of this feature and for new projects see @ref:[Cluste To use Cluster Singleton, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-tools_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } @@project-info{ projectId="akka-cluster-tools" } diff --git a/docs/src/main/paradox/cluster-usage.md b/docs/src/main/paradox/cluster-usage.md index d854561601..ae01162c89 100644 --- a/docs/src/main/paradox/cluster-usage.md +++ b/docs/src/main/paradox/cluster-usage.md @@ -24,12 +24,12 @@ recommendation if you don't have other preferences or constraints. To use Akka Cluster add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-cluster_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } @@project-info{ projectId="akka-cluster" } diff --git a/docs/src/main/paradox/coordination.md b/docs/src/main/paradox/coordination.md index 59e6550731..84d408e1fa 100644 --- a/docs/src/main/paradox/coordination.md +++ b/docs/src/main/paradox/coordination.md @@ -8,12 +8,12 @@ Akka Coordination is a set of tools for distributed coordination. ## Module info @@dependency[sbt,Gradle,Maven] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-coordination_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } @@project-info{ projectId="akka-coordination" } diff --git a/docs/src/main/paradox/discovery/index.md b/docs/src/main/paradox/discovery/index.md index 7ad68d7150..69401db0bd 100644 --- a/docs/src/main/paradox/discovery/index.md +++ b/docs/src/main/paradox/discovery/index.md @@ -34,12 +34,12 @@ See @ref:[Migration hints](#migrating-from-akka-management-discovery-before-1-0- ## Module info @@dependency[sbt,Gradle,Maven] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-discovery_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } @@project-info{ projectId="akka-discovery" } diff --git a/docs/src/main/paradox/dispatchers.md b/docs/src/main/paradox/dispatchers.md index 4c5676c5bd..ef3c55f856 100644 --- a/docs/src/main/paradox/dispatchers.md +++ b/docs/src/main/paradox/dispatchers.md @@ -8,12 +8,12 @@ For the full documentation of this feature and for new projects see @ref:[Dispat Dispatchers are part of core Akka, which means that they are part of the akka-actor dependency: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } diff --git a/docs/src/main/paradox/distributed-data.md b/docs/src/main/paradox/distributed-data.md index 91c65260bb..7409085b0c 100644 --- a/docs/src/main/paradox/distributed-data.md +++ b/docs/src/main/paradox/distributed-data.md @@ -8,12 +8,12 @@ For the full documentation of this feature and for new projects see @ref:[Distri To use Akka Distributed Data, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-distributed-data_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/distributed-pub-sub.md b/docs/src/main/paradox/distributed-pub-sub.md index 89e1de5423..d06c826a34 100644 --- a/docs/src/main/paradox/distributed-pub-sub.md +++ b/docs/src/main/paradox/distributed-pub-sub.md @@ -8,12 +8,12 @@ For the new API see @ref[Distributed Publish Subscribe in Cluster](./typed/distr To use Distributed Publish Subscribe you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-cluster-tools_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } @@project-info{ projectId="akka-cluster-tools" } diff --git a/docs/src/main/paradox/durable-state/persistence-query.md b/docs/src/main/paradox/durable-state/persistence-query.md index cc69070df9..866b98d017 100644 --- a/docs/src/main/paradox/durable-state/persistence-query.md +++ b/docs/src/main/paradox/durable-state/persistence-query.md @@ -8,12 +8,12 @@ project.description: Query side to Akka Persistence allowing for building CQRS a To use Persistence Query, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-persistence-query_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } This will also add dependency on the @ref[Akka Persistence](../persistence.md) module. diff --git a/docs/src/main/paradox/fault-tolerance.md b/docs/src/main/paradox/fault-tolerance.md index 578eb778f3..3a158fe914 100644 --- a/docs/src/main/paradox/fault-tolerance.md +++ b/docs/src/main/paradox/fault-tolerance.md @@ -8,12 +8,12 @@ For the full documentation of this feature and for new projects see @ref:[fault The concept of fault tolerance relates to actors, so in order to use these make sure to depend on actors: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/fsm.md b/docs/src/main/paradox/fsm.md index 04856b0dc3..59603876a2 100644 --- a/docs/src/main/paradox/fsm.md +++ b/docs/src/main/paradox/fsm.md @@ -8,12 +8,12 @@ For the documentation of the new API of this feature and for new projects see @r To use Finite State Machine actors, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Overview @@ -142,7 +142,7 @@ whereas `stay()` will *not*. @@@ To verify that this buncher actually works, it is quite easy to write a test -using the @scala[@ref:[Testing Actor Systems which is conveniently bundled with ScalaTest traits into `AkkaSpec`](testing.md)]@java[@ref:[TestKit](testing.md), here using JUnit as an example]: +using the @scala[@ref:[Testing Actor Systems which is conveniently bundled with ScalaTest traits into `PekkoSpec`](testing.md)]@java[@ref:[TestKit](testing.md), here using JUnit as an example]: Scala : @@snip [FSMDocSpec.scala](/docs/src/test/scala/docs/actor/FSMDocSpec.scala) { #test-code } diff --git a/docs/src/main/paradox/futures.md b/docs/src/main/paradox/futures.md index 79065effef..12283bad99 100644 --- a/docs/src/main/paradox/futures.md +++ b/docs/src/main/paradox/futures.md @@ -5,12 +5,12 @@ Akka offers tiny helpers for use with @scala[@scaladoc[Future](scala.concurrent.Future)s]@java[@javadoc[CompletionStage](java.util.concurrent.CompletionStage)]. These are part of Akka's core module: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## After diff --git a/docs/src/main/paradox/index-actors.md b/docs/src/main/paradox/index-actors.md index 25ed9cbef0..6f2d788768 100644 --- a/docs/src/main/paradox/index-actors.md +++ b/docs/src/main/paradox/index-actors.md @@ -7,16 +7,16 @@ To use Classic Akka Actors, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion group2="com.typesafe.akka" artifact2="akka-testkit_$scala.binary.version$" scope2=test - version2=AkkaVersion + version2=PekkoVersion } @@toc { depth=2 } diff --git a/docs/src/main/paradox/index-utilities-classic.md b/docs/src/main/paradox/index-utilities-classic.md index a9844084f1..d0806682ab 100644 --- a/docs/src/main/paradox/index-utilities-classic.md +++ b/docs/src/main/paradox/index-utilities-classic.md @@ -5,16 +5,16 @@ To use Utilities, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion group2="com.typesafe.akka" artifact2="akka-testkit_$scala.binary.version$" scope2=test - version2=AkkaVersion + version2=PekkoVersion } @@toc { depth=2 } diff --git a/docs/src/main/paradox/io-tcp.md b/docs/src/main/paradox/io-tcp.md index 00d560352d..6fa9a7a29a 100644 --- a/docs/src/main/paradox/io-tcp.md +++ b/docs/src/main/paradox/io-tcp.md @@ -8,12 +8,12 @@ project.description: Low level API for using TCP with classic actors. To use TCP, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/io-udp.md b/docs/src/main/paradox/io-udp.md index 4f05b48b63..796fdf55ec 100644 --- a/docs/src/main/paradox/io-udp.md +++ b/docs/src/main/paradox/io-udp.md @@ -8,12 +8,12 @@ project.description: Low level API for using UDP with classic actors. To use UDP, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/io.md b/docs/src/main/paradox/io.md index dd70118dc4..3b89b56b1c 100644 --- a/docs/src/main/paradox/io.md +++ b/docs/src/main/paradox/io.md @@ -5,12 +5,12 @@ To use I/O, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/logging.md b/docs/src/main/paradox/logging.md index b513a6eddf..22f724be3b 100644 --- a/docs/src/main/paradox/logging.md +++ b/docs/src/main/paradox/logging.md @@ -8,12 +8,12 @@ For the new API see @ref[Logging](typed/logging.md). To use Logging, you must at least use the Akka actors dependency in your project, and will most likely want to configure logging via the SLF4J module (@ref:[see below](#slf4j)). @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } @@project-info{ projectId="akka-slf4j" } @@ -348,12 +348,12 @@ Akka provides a logger for [SLF4J](https://www.slf4j.org/). This module is avail It has a single dependency: the slf4j-api jar. In your runtime, you also need a SLF4J backend. We recommend [Logback](https://logback.qos.ch/): @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-slf4j_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion group2="ch.qos.logback" artifact2="logback-classic" version2="$logback_version$" diff --git a/docs/src/main/paradox/mailboxes.md b/docs/src/main/paradox/mailboxes.md index 99adfe2fec..6620d0f1ca 100644 --- a/docs/src/main/paradox/mailboxes.md +++ b/docs/src/main/paradox/mailboxes.md @@ -8,12 +8,12 @@ For the full documentation of this feature and for new projects see @ref:[mailbo To use Mailboxes, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/multi-node-testing.md b/docs/src/main/paradox/multi-node-testing.md index 9e55965791..4d443c3028 100644 --- a/docs/src/main/paradox/multi-node-testing.md +++ b/docs/src/main/paradox/multi-node-testing.md @@ -8,12 +8,12 @@ project.description: Multi node testing of distributed systems built with Akka. To use Multi Node Testing, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-multi-node-testkit_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion scope=test } diff --git a/docs/src/main/paradox/persistence-fsm.md b/docs/src/main/paradox/persistence-fsm.md index 8f7eab3caf..1f62d29752 100644 --- a/docs/src/main/paradox/persistence-fsm.md +++ b/docs/src/main/paradox/persistence-fsm.md @@ -7,12 +7,12 @@ Persistent FSMs are part of Akka persistence, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-persistence_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } @@@ warning diff --git a/docs/src/main/paradox/persistence-journals.md b/docs/src/main/paradox/persistence-journals.md index e5538eb284..2c41afcfcc 100644 --- a/docs/src/main/paradox/persistence-journals.md +++ b/docs/src/main/paradox/persistence-journals.md @@ -102,12 +102,12 @@ In order to help developers build correct and high quality storage plugins, we p The TCK is usable from Java as well as Scala projects. To test your implementation (independently of language) you need to include the akka-persistence-tck dependency: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-persistence-tck_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } To include the Journal TCK tests in your test suite simply extend the provided @scala[`JournalSpec`]@java[`JavaJournalSpec`]: diff --git a/docs/src/main/paradox/persistence-query-leveldb.md b/docs/src/main/paradox/persistence-query-leveldb.md index 8302c1728c..e8b46b4bac 100644 --- a/docs/src/main/paradox/persistence-query-leveldb.md +++ b/docs/src/main/paradox/persistence-query-leveldb.md @@ -8,12 +8,12 @@ As a replacement we recommend using [Akka Persistence JDBC](https://doc.akka.io/ To use Persistence Query, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-persistence-query_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } This will also add dependency on the @ref[akka-persistence](persistence.md) module. diff --git a/docs/src/main/paradox/persistence-query.md b/docs/src/main/paradox/persistence-query.md index 550a8d9137..f4f487cd5f 100644 --- a/docs/src/main/paradox/persistence-query.md +++ b/docs/src/main/paradox/persistence-query.md @@ -8,12 +8,12 @@ project.description: Query side to Akka Persistence allowing for building CQRS a To use Persistence Query, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-persistence-query_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } This will also add dependency on the @ref[Akka Persistence](persistence.md) module. diff --git a/docs/src/main/paradox/persistence-schema-evolution.md b/docs/src/main/paradox/persistence-schema-evolution.md index 06fe7dd126..46bafda83a 100644 --- a/docs/src/main/paradox/persistence-schema-evolution.md +++ b/docs/src/main/paradox/persistence-schema-evolution.md @@ -5,15 +5,15 @@ This documentation page touches upon @ref[Akka Persistence](persistence.md), so to follow those examples you will want to depend on: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-persistence_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion group2="com.typesafe.akka" artifact2="akka-persistence-testkit_$scala.binary.version$" - version2=AkkaVersion + version2=PekkoVersion scope2=test } diff --git a/docs/src/main/paradox/persistence.md b/docs/src/main/paradox/persistence.md index c278aab1a4..5c1406c1a6 100644 --- a/docs/src/main/paradox/persistence.md +++ b/docs/src/main/paradox/persistence.md @@ -11,15 +11,15 @@ For the full documentation of this feature and for new projects see @ref:[Event To use Akka Persistence, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-persistence_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion group2="com.typesafe.akka" artifact2="akka-persistence-testkit_$scala.binary.version$" - version2=AkkaVersion + version2=PekkoVersion scope2=test } diff --git a/docs/src/main/paradox/remoting-artery.md b/docs/src/main/paradox/remoting-artery.md index 54c1ace80f..2413fb2755 100644 --- a/docs/src/main/paradox/remoting-artery.md +++ b/docs/src/main/paradox/remoting-artery.md @@ -23,12 +23,12 @@ If migrating from classic remoting see @ref:[what's new in Artery](#what-is-new- To use Artery Remoting, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-remote_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } One option is to use Artery with Aeron, see @ref:[Selecting a transport](#selecting-a-transport). diff --git a/docs/src/main/paradox/remoting.md b/docs/src/main/paradox/remoting.md index fd402e9a46..141ab9e454 100644 --- a/docs/src/main/paradox/remoting.md +++ b/docs/src/main/paradox/remoting.md @@ -25,12 +25,12 @@ such as [HTTP](https://doc.akka.io/docs/akka-http/current/), To use Akka Remoting, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-remote_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } @@project-info{ projectId="akka-remote" } diff --git a/docs/src/main/paradox/routing.md b/docs/src/main/paradox/routing.md index f65954f12e..89ad1ab365 100644 --- a/docs/src/main/paradox/routing.md +++ b/docs/src/main/paradox/routing.md @@ -8,12 +8,12 @@ For the documentation of the new API of this feature and for new projects see @r To use Routing, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/scheduler.md b/docs/src/main/paradox/scheduler.md index 54196bd177..b0f8aede18 100644 --- a/docs/src/main/paradox/scheduler.md +++ b/docs/src/main/paradox/scheduler.md @@ -11,12 +11,12 @@ For the new API see @ref:[typed scheduling](typed/interaction-patterns.md#typed- To use Scheduler, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/serialization-classic.md b/docs/src/main/paradox/serialization-classic.md index 72b9665211..8ded554cb0 100644 --- a/docs/src/main/paradox/serialization-classic.md +++ b/docs/src/main/paradox/serialization-classic.md @@ -10,12 +10,12 @@ aside from serialization of `ActorRef` that is described @ref:[here](#serializin To use Serialization, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Serializing ActorRefs diff --git a/docs/src/main/paradox/serialization-jackson.md b/docs/src/main/paradox/serialization-jackson.md index 972275f19d..bc21fab4f3 100644 --- a/docs/src/main/paradox/serialization-jackson.md +++ b/docs/src/main/paradox/serialization-jackson.md @@ -8,12 +8,12 @@ project.description: Serialization with Jackson for Akka. To use Jackson Serialization, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-serialization-jackson_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction @@ -461,8 +461,8 @@ the binding name (for example `jackson-cbor`). ## Using Akka Serialization for embedded types -For types that already have an Akka Serializer defined that are embedded in types serialized with Jackson the @apidoc[AkkaSerializationSerializer] and -@apidoc[AkkaSerializationDeserializer] can be used to Akka Serialization for individual fields. +For types that already have an Akka Serializer defined that are embedded in types serialized with Jackson the @apidoc[PekkoSerializationSerializer] and +@apidoc[PekkoSerializationDeserializer] can be used to Akka Serialization for individual fields. The serializer/deserializer are not enabled automatically. The @javadoc[@JsonSerialize](com.fasterxml.jackson.databind.annotation.JsonSerialize) and @javadoc[@JsonDeserialize](com.fasterxml.jackson.databind.annotation.JsonDeserialize) annotation needs to be added to the fields containing the types to be serialized with Akka Serialization. diff --git a/docs/src/main/paradox/serialization.md b/docs/src/main/paradox/serialization.md index 309db607e4..90eed9aaa5 100644 --- a/docs/src/main/paradox/serialization.md +++ b/docs/src/main/paradox/serialization.md @@ -8,12 +8,12 @@ project.description: Serialization APIs built into Akka. To use Serialization, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/split-brain-resolver.md b/docs/src/main/paradox/split-brain-resolver.md index fc4dfba653..30778f04f4 100644 --- a/docs/src/main/paradox/split-brain-resolver.md +++ b/docs/src/main/paradox/split-brain-resolver.md @@ -16,12 +16,12 @@ To use Akka Split Brain Resolver is part of `akka-cluster` and you probably alre dependency included. Otherwise, add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } @@project-info{ projectId="akka-cluster" } diff --git a/docs/src/main/paradox/stream/actor-interop.md b/docs/src/main/paradox/stream/actor-interop.md index 226689a1da..905863cb91 100644 --- a/docs/src/main/paradox/stream/actor-interop.md +++ b/docs/src/main/paradox/stream/actor-interop.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Overview diff --git a/docs/src/main/paradox/stream/futures-interop.md b/docs/src/main/paradox/stream/futures-interop.md index 7d573674f1..ce371e5f7a 100644 --- a/docs/src/main/paradox/stream/futures-interop.md +++ b/docs/src/main/paradox/stream/futures-interop.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Overview diff --git a/docs/src/main/paradox/stream/index.md b/docs/src/main/paradox/stream/index.md index 4784199a14..23bbf8271d 100644 --- a/docs/src/main/paradox/stream/index.md +++ b/docs/src/main/paradox/stream/index.md @@ -8,15 +8,15 @@ project.description: An intuitive and safe way to do asynchronous, non-blocking To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion group2="com.typesafe.akka" artifact2="akka-stream-testkit_$scala.binary.version$" - version2=AkkaVersion + version2=PekkoVersion scope2=test } diff --git a/docs/src/main/paradox/stream/operators/ActorFlow/ask.md b/docs/src/main/paradox/stream/operators/ActorFlow/ask.md index 41e4411122..e66427280b 100644 --- a/docs/src/main/paradox/stream/operators/ActorFlow/ask.md +++ b/docs/src/main/paradox/stream/operators/ActorFlow/ask.md @@ -9,12 +9,12 @@ Use the "Ask Pattern" to send each stream element as an `ask` to the target acto This operator is included in: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream-typed_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Signature diff --git a/docs/src/main/paradox/stream/operators/ActorFlow/askWithContext.md b/docs/src/main/paradox/stream/operators/ActorFlow/askWithContext.md index 16096594c4..34f1093d29 100644 --- a/docs/src/main/paradox/stream/operators/ActorFlow/askWithContext.md +++ b/docs/src/main/paradox/stream/operators/ActorFlow/askWithContext.md @@ -9,12 +9,12 @@ Use the "Ask Pattern" to send each stream element (without the context) as an `a This operator is included in: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream-typed_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Signature diff --git a/docs/src/main/paradox/stream/operators/ActorFlow/askWithStatus.md b/docs/src/main/paradox/stream/operators/ActorFlow/askWithStatus.md index 6d230d444d..8cc9c88c7f 100644 --- a/docs/src/main/paradox/stream/operators/ActorFlow/askWithStatus.md +++ b/docs/src/main/paradox/stream/operators/ActorFlow/askWithStatus.md @@ -9,11 +9,11 @@ Use the "Ask Pattern" to send each stream element as an `ask` to the target acto This operator is included in: @@dependency[sbt,Maven,Gradle] { - symbol1=AkkaVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream-typed_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Signature diff --git a/docs/src/main/paradox/stream/operators/ActorFlow/askWithStatusAndContext.md b/docs/src/main/paradox/stream/operators/ActorFlow/askWithStatusAndContext.md index 1f26abbb45..435fa94894 100644 --- a/docs/src/main/paradox/stream/operators/ActorFlow/askWithStatusAndContext.md +++ b/docs/src/main/paradox/stream/operators/ActorFlow/askWithStatusAndContext.md @@ -9,12 +9,12 @@ Use the "Ask Pattern" to send each stream element (without the context) as an `a This operator is included in: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream-typed_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Signature diff --git a/docs/src/main/paradox/stream/operators/ActorSink/actorRef.md b/docs/src/main/paradox/stream/operators/ActorSink/actorRef.md index ccf2552770..103b13f9ec 100644 --- a/docs/src/main/paradox/stream/operators/ActorSink/actorRef.md +++ b/docs/src/main/paradox/stream/operators/ActorSink/actorRef.md @@ -9,12 +9,12 @@ Sends the elements of the stream to the given @java[`ActorRef`]@scala[`ActorR This operator is included in: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream-typed_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Signature diff --git a/docs/src/main/paradox/stream/operators/ActorSink/actorRefWithBackpressure.md b/docs/src/main/paradox/stream/operators/ActorSink/actorRefWithBackpressure.md index 74dcdf199d..46a2583483 100644 --- a/docs/src/main/paradox/stream/operators/ActorSink/actorRefWithBackpressure.md +++ b/docs/src/main/paradox/stream/operators/ActorSink/actorRefWithBackpressure.md @@ -9,12 +9,12 @@ Sends the elements of the stream to the given @java[`ActorRef`]@scala[`ActorR This operator is included in: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream-typed_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Signature diff --git a/docs/src/main/paradox/stream/operators/ActorSource/actorRef.md b/docs/src/main/paradox/stream/operators/ActorSource/actorRef.md index cbf793556e..e582b89843 100644 --- a/docs/src/main/paradox/stream/operators/ActorSource/actorRef.md +++ b/docs/src/main/paradox/stream/operators/ActorSource/actorRef.md @@ -9,12 +9,12 @@ Materialize an @java[`ActorRef`]@scala[`ActorRef[T]`] of the new actors API; This operator is included in: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream-typed_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Signature diff --git a/docs/src/main/paradox/stream/operators/ActorSource/actorRefWithBackpressure.md b/docs/src/main/paradox/stream/operators/ActorSource/actorRefWithBackpressure.md index cdead1bc46..29a61e496a 100644 --- a/docs/src/main/paradox/stream/operators/ActorSource/actorRefWithBackpressure.md +++ b/docs/src/main/paradox/stream/operators/ActorSource/actorRefWithBackpressure.md @@ -9,12 +9,12 @@ Materialize an @java[`ActorRef`]@scala[`ActorRef[T]`] of the new actors API; This operator is included in: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream-typed_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Signature diff --git a/docs/src/main/paradox/stream/operators/PubSub/sink.md b/docs/src/main/paradox/stream/operators/PubSub/sink.md index b770fdd2cc..5769b58556 100644 --- a/docs/src/main/paradox/stream/operators/PubSub/sink.md +++ b/docs/src/main/paradox/stream/operators/PubSub/sink.md @@ -14,12 +14,12 @@ If the topic does not have any subscribers when a message is published, or the t This operator is included in: @@dependency[sbt,Maven,Gradle] { -bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion -symbol1=AkkaVersion +bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion +symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream-typed_$scala.binary.version$" -version=AkkaVersion +version=PekkoVersion } ## Signature diff --git a/docs/src/main/paradox/stream/operators/PubSub/source.md b/docs/src/main/paradox/stream/operators/PubSub/source.md index acd614cd13..d4783e508e 100644 --- a/docs/src/main/paradox/stream/operators/PubSub/source.md +++ b/docs/src/main/paradox/stream/operators/PubSub/source.md @@ -17,12 +17,12 @@ strategy. This operator is included in: @@dependency[sbt,Maven,Gradle] { -bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion -symbol1=AkkaVersion +bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion +symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream-typed_$scala.binary.version$" -version=AkkaVersion +version=PekkoVersion } ## Signature diff --git a/docs/src/main/paradox/stream/operators/Source/range.md b/docs/src/main/paradox/stream/operators/Source/range.md index 7fd9a1ee4a..778b03069d 100644 --- a/docs/src/main/paradox/stream/operators/Source/range.md +++ b/docs/src/main/paradox/stream/operators/Source/range.md @@ -7,12 +7,12 @@ Emit each integer in a range, with an option to take bigger steps than 1. ## Dependency @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } diff --git a/docs/src/main/paradox/stream/reactive-streams-interop.md b/docs/src/main/paradox/stream/reactive-streams-interop.md index 8f40c9a640..139f6da28a 100644 --- a/docs/src/main/paradox/stream/reactive-streams-interop.md +++ b/docs/src/main/paradox/stream/reactive-streams-interop.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } diff --git a/docs/src/main/paradox/stream/stream-composition.md b/docs/src/main/paradox/stream/stream-composition.md index ca7a3726f6..31f3fc2725 100644 --- a/docs/src/main/paradox/stream/stream-composition.md +++ b/docs/src/main/paradox/stream/stream-composition.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/stream/stream-cookbook.md b/docs/src/main/paradox/stream/stream-cookbook.md index 48e68c6ed1..ed64c31286 100644 --- a/docs/src/main/paradox/stream/stream-cookbook.md +++ b/docs/src/main/paradox/stream/stream-cookbook.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/stream/stream-customize.md b/docs/src/main/paradox/stream/stream-customize.md index 0ba88c9636..62ab8e1f0c 100644 --- a/docs/src/main/paradox/stream/stream-customize.md +++ b/docs/src/main/paradox/stream/stream-customize.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/stream/stream-dynamic.md b/docs/src/main/paradox/stream/stream-dynamic.md index a375f58e84..37dbbff645 100644 --- a/docs/src/main/paradox/stream/stream-dynamic.md +++ b/docs/src/main/paradox/stream/stream-dynamic.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/stream/stream-error.md b/docs/src/main/paradox/stream/stream-error.md index e0ab1d3577..6445e660dd 100644 --- a/docs/src/main/paradox/stream/stream-error.md +++ b/docs/src/main/paradox/stream/stream-error.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/stream/stream-flows-and-basics.md b/docs/src/main/paradox/stream/stream-flows-and-basics.md index 3fe1418601..9cb4f01bbf 100644 --- a/docs/src/main/paradox/stream/stream-flows-and-basics.md +++ b/docs/src/main/paradox/stream/stream-flows-and-basics.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/stream/stream-graphs.md b/docs/src/main/paradox/stream/stream-graphs.md index 7a854a90e8..26a185645c 100644 --- a/docs/src/main/paradox/stream/stream-graphs.md +++ b/docs/src/main/paradox/stream/stream-graphs.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/stream/stream-io.md b/docs/src/main/paradox/stream/stream-io.md index 9a0a61ae4d..adbafe2968 100644 --- a/docs/src/main/paradox/stream/stream-io.md +++ b/docs/src/main/paradox/stream/stream-io.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/stream/stream-parallelism.md b/docs/src/main/paradox/stream/stream-parallelism.md index 643f13734b..2e39e3ae03 100644 --- a/docs/src/main/paradox/stream/stream-parallelism.md +++ b/docs/src/main/paradox/stream/stream-parallelism.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/stream/stream-quickstart.md b/docs/src/main/paradox/stream/stream-quickstart.md index 4d2b91fe92..8276473360 100644 --- a/docs/src/main/paradox/stream/stream-quickstart.md +++ b/docs/src/main/paradox/stream/stream-quickstart.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } @@@ note diff --git a/docs/src/main/paradox/stream/stream-rate.md b/docs/src/main/paradox/stream/stream-rate.md index 711667067d..046011ff27 100644 --- a/docs/src/main/paradox/stream/stream-rate.md +++ b/docs/src/main/paradox/stream/stream-rate.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/stream/stream-refs.md b/docs/src/main/paradox/stream/stream-refs.md index 53f39cd179..e0557c78f5 100644 --- a/docs/src/main/paradox/stream/stream-refs.md +++ b/docs/src/main/paradox/stream/stream-refs.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/stream/stream-substream.md b/docs/src/main/paradox/stream/stream-substream.md index aebcba0f54..a826815941 100644 --- a/docs/src/main/paradox/stream/stream-substream.md +++ b/docs/src/main/paradox/stream/stream-substream.md @@ -5,12 +5,12 @@ To use Akka Streams, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/stream/stream-testkit.md b/docs/src/main/paradox/stream/stream-testkit.md index 04c4438bc5..1107ee725b 100644 --- a/docs/src/main/paradox/stream/stream-testkit.md +++ b/docs/src/main/paradox/stream/stream-testkit.md @@ -5,12 +5,12 @@ To use Akka Stream TestKit, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-stream-testkit_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion scope="test" } diff --git a/docs/src/main/paradox/testing.md b/docs/src/main/paradox/testing.md index e14d03be11..5e5ff8671e 100644 --- a/docs/src/main/paradox/testing.md +++ b/docs/src/main/paradox/testing.md @@ -8,12 +8,12 @@ For the new API see @ref[testing](typed/testing.md). To use Akka Testkit, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-testkit_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion scope="test" } diff --git a/docs/src/main/paradox/typed/actor-discovery.md b/docs/src/main/paradox/typed/actor-discovery.md index 87bfa6cb8b..578fe7f20d 100644 --- a/docs/src/main/paradox/typed/actor-discovery.md +++ b/docs/src/main/paradox/typed/actor-discovery.md @@ -7,12 +7,12 @@ You are viewing the documentation for the new actor APIs, to view the Akka Class To use Akka Actor Typed, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-actor-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } ## Obtaining Actor references diff --git a/docs/src/main/paradox/typed/actor-lifecycle.md b/docs/src/main/paradox/typed/actor-lifecycle.md index 7c91cc5c4d..16d1ff3145 100644 --- a/docs/src/main/paradox/typed/actor-lifecycle.md +++ b/docs/src/main/paradox/typed/actor-lifecycle.md @@ -10,12 +10,12 @@ You are viewing the documentation for the new actor APIs, to view the Akka Class To use Akka Actor Typed, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-actor-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/typed/actors.md b/docs/src/main/paradox/typed/actors.md index 9bb8be8525..a936af1e6c 100644 --- a/docs/src/main/paradox/typed/actors.md +++ b/docs/src/main/paradox/typed/actors.md @@ -10,15 +10,15 @@ You are viewing the documentation for the new actor APIs, to view the Akka Class To use Akka Actors, add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-actor-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion group2=com.typesafe.akka artifact2=akka-actor-testkit-typed_$scala.binary.version$ - version2=AkkaVersion + version2=PekkoVersion scope2=test } diff --git a/docs/src/main/paradox/typed/cluster-dc.md b/docs/src/main/paradox/typed/cluster-dc.md index 80bb8a455a..a03592e1fe 100644 --- a/docs/src/main/paradox/typed/cluster-dc.md +++ b/docs/src/main/paradox/typed/cluster-dc.md @@ -19,12 +19,12 @@ up a large cluster into smaller groups of nodes for better scalability. To use Akka Cluster add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } ## Motivation diff --git a/docs/src/main/paradox/typed/cluster-sharded-daemon-process.md b/docs/src/main/paradox/typed/cluster-sharded-daemon-process.md index 6e4deec0aa..ca4bd5e2a4 100644 --- a/docs/src/main/paradox/typed/cluster-sharded-daemon-process.md +++ b/docs/src/main/paradox/typed/cluster-sharded-daemon-process.md @@ -5,12 +5,12 @@ To use Akka Sharded Daemon Process, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-sharding-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } @@project-info{ projectId="akka-cluster-sharding-typed" } diff --git a/docs/src/main/paradox/typed/cluster-sharding.md b/docs/src/main/paradox/typed/cluster-sharding.md index 9d39b3aa20..b87d4bf37c 100644 --- a/docs/src/main/paradox/typed/cluster-sharding.md +++ b/docs/src/main/paradox/typed/cluster-sharding.md @@ -10,12 +10,12 @@ You are viewing the documentation for the new actor APIs, to view the Akka Class To use Akka Cluster Sharding, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-sharding-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } @@project-info{ projectId="akka-cluster-sharding-typed" } diff --git a/docs/src/main/paradox/typed/cluster-singleton.md b/docs/src/main/paradox/typed/cluster-singleton.md index cc167f95b7..e9e34aaaca 100644 --- a/docs/src/main/paradox/typed/cluster-singleton.md +++ b/docs/src/main/paradox/typed/cluster-singleton.md @@ -7,12 +7,12 @@ You are viewing the documentation for the new actor APIs, to view the Akka Class To use Cluster Singleton, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } @@project-info{ projectId="akka-cluster-typed" } diff --git a/docs/src/main/paradox/typed/cluster.md b/docs/src/main/paradox/typed/cluster.md index 96fd300c4a..37745f0b3f 100644 --- a/docs/src/main/paradox/typed/cluster.md +++ b/docs/src/main/paradox/typed/cluster.md @@ -26,12 +26,12 @@ recommendation if you don't have other preferences or constraints. To use Akka Cluster add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } @@project-info{ projectId="akka-cluster-typed" } diff --git a/docs/src/main/paradox/typed/coexisting.md b/docs/src/main/paradox/typed/coexisting.md index 7376312877..582b0d599b 100644 --- a/docs/src/main/paradox/typed/coexisting.md +++ b/docs/src/main/paradox/typed/coexisting.md @@ -5,12 +5,12 @@ To use Akka Actor Typed, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-actor-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/typed/dispatchers.md b/docs/src/main/paradox/typed/dispatchers.md index 43a38d5617..9247e3a24c 100644 --- a/docs/src/main/paradox/typed/dispatchers.md +++ b/docs/src/main/paradox/typed/dispatchers.md @@ -11,12 +11,12 @@ Dispatchers are part of core Akka, which means that they are part of the `akka-a page describes how to use dispatchers with `akka-actor-typed`, which has dependency: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor-typed_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/typed/distributed-data.md b/docs/src/main/paradox/typed/distributed-data.md index 1d65bcdae4..c632b9dccd 100644 --- a/docs/src/main/paradox/typed/distributed-data.md +++ b/docs/src/main/paradox/typed/distributed-data.md @@ -10,12 +10,12 @@ You are viewing the documentation for the new actor APIs, to view the Akka Class To use Akka Cluster Distributed Data, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } @@project-info{ projectId="akka-cluster-typed" } diff --git a/docs/src/main/paradox/typed/distributed-pub-sub.md b/docs/src/main/paradox/typed/distributed-pub-sub.md index 98b21f6094..83158e81e5 100644 --- a/docs/src/main/paradox/typed/distributed-pub-sub.md +++ b/docs/src/main/paradox/typed/distributed-pub-sub.md @@ -8,12 +8,12 @@ The distributed publish subscribe topic API is available and usable with the cor when used in a clustered application: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-cluster-typed_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## The Topic Actor diff --git a/docs/src/main/paradox/typed/durable-state/persistence.md b/docs/src/main/paradox/typed/durable-state/persistence.md index ba9c3ed1f6..6bafbf111a 100644 --- a/docs/src/main/paradox/typed/durable-state/persistence.md +++ b/docs/src/main/paradox/typed/durable-state/persistence.md @@ -8,15 +8,15 @@ project.description: Durable State with Akka Persistence enables actors to persi To use Akka Persistence, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-persistence-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion group2=com.typesafe.akka artifact2=akka-persistence-testkit_$scala.binary.version$ - version2=AkkaVersion + version2=PekkoVersion scope2=test } diff --git a/docs/src/main/paradox/typed/from-classic.md b/docs/src/main/paradox/typed/from-classic.md index 0f45219892..499f484366 100644 --- a/docs/src/main/paradox/typed/from-classic.md +++ b/docs/src/main/paradox/typed/from-classic.md @@ -26,12 +26,12 @@ module, with a few exceptions. For example `akka-cluster-typed`: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } Artifact names: diff --git a/docs/src/main/paradox/typed/guide/modules.md b/docs/src/main/paradox/typed/guide/modules.md index e624cb0650..7d1b0cc072 100644 --- a/docs/src/main/paradox/typed/guide/modules.md +++ b/docs/src/main/paradox/typed/guide/modules.md @@ -36,12 +36,12 @@ This page does not list all available modules, but overviews the main functional ### Actor library @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-actor-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } The core Akka library is `akka-actor-typed`, but actors are used across Akka libraries, providing a consistent, integrated model that relieves you from individually @@ -64,12 +64,12 @@ Challenges that actors solve include the following: ### Remoting @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-remote_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } Remoting enables actors that live on different computers to seamlessly exchange messages. @@ -90,12 +90,12 @@ Challenges Remoting solves include the following: ### Cluster @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } If you have a set of actor systems that cooperate to solve some business problem, then you likely want to manage these set of @@ -116,12 +116,12 @@ Challenges the Cluster module solves include the following: ### Cluster Sharding @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-sharding-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } Sharding helps to solve the problem of distributing a set of actors among members of an Akka cluster. @@ -138,12 +138,12 @@ Challenges that Sharding solves include the following: ### Cluster Singleton @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-singleton_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } A common (in fact, a bit too common) use case in distributed systems is to have a single entity responsible @@ -162,12 +162,12 @@ The Singleton module can be used to solve these challenges: ### Persistence @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-persistence-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } Just like objects in OOP, actors keep their state in volatile memory. Once the system is shut down, gracefully or @@ -187,12 +187,12 @@ Persistence tackles the following challenges: ### Projections @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-projection-core_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } Projections provides a simple API for consuming a stream of events for projection into a variety of downstream options. The core dependency provides only the API and other provider dependencies are required for different source and sink implementations. @@ -207,12 +207,12 @@ Challenges Projections solve include the following: ### Distributed Data @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } In situations where eventual consistency is acceptable, it is possible to share data between nodes in @@ -229,12 +229,12 @@ Distributed Data is intended to solve the following challenges: ### Streams @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-stream-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } Actors are a fundamental model for concurrency, but there are common patterns where their use requires the user diff --git a/docs/src/main/paradox/typed/guide/tutorial_1.md b/docs/src/main/paradox/typed/guide/tutorial_1.md index b8463cc2b9..ea4e5e9d77 100644 --- a/docs/src/main/paradox/typed/guide/tutorial_1.md +++ b/docs/src/main/paradox/typed/guide/tutorial_1.md @@ -5,12 +5,12 @@ Add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor-typed_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/typed/interaction-patterns.md b/docs/src/main/paradox/typed/interaction-patterns.md index cb1c52fac5..ecfe37dbdf 100644 --- a/docs/src/main/paradox/typed/interaction-patterns.md +++ b/docs/src/main/paradox/typed/interaction-patterns.md @@ -7,12 +7,12 @@ You are viewing the documentation for the new actor APIs, to view the Akka Class To use Akka Actor Typed, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-actor-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/typed/logging.md b/docs/src/main/paradox/typed/logging.md index 153632f881..99915b3a79 100644 --- a/docs/src/main/paradox/typed/logging.md +++ b/docs/src/main/paradox/typed/logging.md @@ -11,12 +11,12 @@ To use Logging, you must at least use the Akka actors dependency in your project via the SLF4J backend, such as Logback configuration. @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor-typed_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/typed/mailboxes.md b/docs/src/main/paradox/typed/mailboxes.md index dec72b721d..0ce8d95f42 100644 --- a/docs/src/main/paradox/typed/mailboxes.md +++ b/docs/src/main/paradox/typed/mailboxes.md @@ -8,12 +8,12 @@ Mailboxes are part of core Akka, which means that they are part of the `akka-act page describes how to use mailboxes with `akka-actor-typed`, which has dependency: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-actor-typed_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/typed/persistence-testing.md b/docs/src/main/paradox/typed/persistence-testing.md index 685c47e8c0..dc76188564 100644 --- a/docs/src/main/paradox/typed/persistence-testing.md +++ b/docs/src/main/paradox/typed/persistence-testing.md @@ -5,15 +5,15 @@ To use Akka Persistence TestKit, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group1=com.typesafe.akka artifact1=akka-persistence-typed_$scala.binary.version$ - version1=AkkaVersion + version1=PekkoVersion group2=com.typesafe.akka artifact2=akka-persistence-testkit_$scala.binary.version$ - version2=AkkaVersion + version2=PekkoVersion scope2=test } @@ -63,12 +63,12 @@ Persistence testkit allows to check events saved in a storage, emulate storage o To use the testkit you need to add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-persistence-testkit_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } There are two testkit classes which have similar api: @@ -200,12 +200,12 @@ the plugins at the same time. To coordinate initialization you can use the `Pers `PersistenceInit` is part of `akka-persistence-testkit` and you need to add the dependency to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group="com.typesafe.akka" artifact="akka-persistence-testkit_$scala.binary.version$" - version=AkkaVersion + version=PekkoVersion } Scala diff --git a/docs/src/main/paradox/typed/persistence.md b/docs/src/main/paradox/typed/persistence.md index af4c6a953f..85d0bc9053 100644 --- a/docs/src/main/paradox/typed/persistence.md +++ b/docs/src/main/paradox/typed/persistence.md @@ -10,15 +10,15 @@ You are viewing the documentation for the new actor APIs, to view the Akka Class To use Akka Persistence, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-persistence-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion group2=com.typesafe.akka artifact2=akka-persistence-testkit_$scala.binary.version$ - version2=AkkaVersion + version2=PekkoVersion scope2=test } diff --git a/docs/src/main/paradox/typed/reliable-delivery.md b/docs/src/main/paradox/typed/reliable-delivery.md index cb30e4532c..8ee89431e1 100644 --- a/docs/src/main/paradox/typed/reliable-delivery.md +++ b/docs/src/main/paradox/typed/reliable-delivery.md @@ -18,12 +18,12 @@ warning or deprecation period. It is also not recommended to use this module in To use reliable delivery, add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-actor-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } ## Introduction @@ -255,12 +255,12 @@ In that case some of these may already have been processed by the previous worke To use reliable delivery with Cluster Sharding, add the following module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-cluster-sharding-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } Reliable delivery between a producer actor sending messages to @ref:[sharded](cluster-sharding.md) consumer @@ -364,12 +364,12 @@ Be aware of that a `DurableProducerQueue` will add a substantial performance ove When using the `EventSourcedProducerQueue` the following dependency is needed: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-persistence-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } You also have to select journal plugin and snapshot store plugin, see diff --git a/docs/src/main/paradox/typed/routers.md b/docs/src/main/paradox/typed/routers.md index 7784d74708..f5ae6447bc 100644 --- a/docs/src/main/paradox/typed/routers.md +++ b/docs/src/main/paradox/typed/routers.md @@ -7,12 +7,12 @@ You are viewing the documentation for the new actor APIs, to view the Akka Class To use Akka Actor Typed, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-actor-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/typed/stash.md b/docs/src/main/paradox/typed/stash.md index 61db63ba33..2d8e4892f4 100644 --- a/docs/src/main/paradox/typed/stash.md +++ b/docs/src/main/paradox/typed/stash.md @@ -7,12 +7,12 @@ You are viewing the documentation for the new actor APIs, to view the Akka Class To use Akka Actor Typed, you must add the following dependency in your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-actor-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion } ## Introduction diff --git a/docs/src/main/paradox/typed/testing.md b/docs/src/main/paradox/typed/testing.md index 5ebe6df5a9..652a58d482 100644 --- a/docs/src/main/paradox/typed/testing.md +++ b/docs/src/main/paradox/typed/testing.md @@ -7,12 +7,12 @@ You are viewing the documentation for the new actor APIs, to view the Akka Class To use Actor TestKit add the module to your project: @@dependency[sbt,Maven,Gradle] { - bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=AkkaVersion - symbol1=AkkaVersion + bomGroup=com.typesafe.akka bomArtifact=akka-bom_$scala.binary.version$ bomVersionSymbols=PekkoVersion + symbol1=PekkoVersion value1="$pekko.version$" group=com.typesafe.akka artifact=akka-actor-testkit-typed_$scala.binary.version$ - version=AkkaVersion + version=PekkoVersion scope=test } diff --git a/docs/src/test/java/jdocs/actor/SchedulerDocTest.java b/docs/src/test/java/jdocs/actor/SchedulerDocTest.java index 49f1ce2d0a..33cc07622c 100644 --- a/docs/src/test/java/jdocs/actor/SchedulerDocTest.java +++ b/docs/src/test/java/jdocs/actor/SchedulerDocTest.java @@ -17,15 +17,15 @@ import org.apache.pekko.actor.Props; import org.apache.pekko.actor.AbstractActor; import org.apache.pekko.actor.ActorRef; import org.apache.pekko.actor.ActorSystem; -import org.apache.pekko.testkit.AkkaSpec; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoSpec; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import org.junit.*; public class SchedulerDocTest extends AbstractJavaTest { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("SchedulerDocTest", AkkaSpec.testConf()); + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("SchedulerDocTest", PekkoSpec.testConf()); private final ActorSystem system = actorSystemResource.getSystem(); private ActorRef testActor = system.actorOf(Props.create(MyActor.class)); diff --git a/docs/src/test/java/jdocs/dispatcher/DispatcherDocTest.java b/docs/src/test/java/jdocs/dispatcher/DispatcherDocTest.java index 76e0a26769..fceb724eb8 100644 --- a/docs/src/test/java/jdocs/dispatcher/DispatcherDocTest.java +++ b/docs/src/test/java/jdocs/dispatcher/DispatcherDocTest.java @@ -6,7 +6,7 @@ package jdocs.dispatcher; import org.apache.pekko.dispatch.ControlMessage; import org.apache.pekko.dispatch.RequiresMessageQueue; -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoSpec; import org.apache.pekko.testkit.javadsl.TestKit; import com.typesafe.config.ConfigFactory; import docs.dispatcher.DispatcherDocSpec; @@ -29,7 +29,7 @@ import org.apache.pekko.event.LoggingAdapter; // #imports-prio-mailbox import org.apache.pekko.dispatch.PriorityGenerator; import org.apache.pekko.dispatch.UnboundedStablePriorityMailbox; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import com.typesafe.config.Config; // #imports-prio-mailbox @@ -41,12 +41,12 @@ import com.typesafe.config.Config; public class DispatcherDocTest extends AbstractJavaTest { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource( + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource( "DispatcherDocTest", ConfigFactory.parseString(DispatcherDocSpec.javaConfig()) .withFallback(ConfigFactory.parseString(DispatcherDocSpec.config())) - .withFallback(AkkaSpec.testConf())); + .withFallback(PekkoSpec.testConf())); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/docs/src/test/java/jdocs/event/EventBusDocTest.java b/docs/src/test/java/jdocs/event/EventBusDocTest.java index 9f4d9aeb6b..deb33da75c 100644 --- a/docs/src/test/java/jdocs/event/EventBusDocTest.java +++ b/docs/src/test/java/jdocs/event/EventBusDocTest.java @@ -15,7 +15,7 @@ import org.junit.Test; import org.apache.pekko.actor.ActorSystem; import org.apache.pekko.actor.ActorRef; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import org.apache.pekko.util.Subclassification; // #lookup-bus @@ -260,8 +260,8 @@ public class EventBusDocTest extends AbstractJavaTest { // #actor-bus @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("EventBusDocTest"); + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("EventBusDocTest"); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/docs/src/test/java/jdocs/future/FutureDocTest.java b/docs/src/test/java/jdocs/future/FutureDocTest.java index 4be45ee0ef..e7bec28e76 100644 --- a/docs/src/test/java/jdocs/future/FutureDocTest.java +++ b/docs/src/test/java/jdocs/future/FutureDocTest.java @@ -7,8 +7,8 @@ package jdocs.future; import org.apache.pekko.actor.typed.ActorSystem; import org.apache.pekko.dispatch.Futures; import org.apache.pekko.pattern.Patterns; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoSpec; import org.apache.pekko.util.Timeout; import jdocs.AbstractJavaTest; import org.junit.ClassRule; @@ -27,7 +27,6 @@ import java.util.concurrent.CompletionStage; import static org.apache.pekko.actor.typed.javadsl.Adapter.toTyped; import static org.apache.pekko.dispatch.Futures.future; // #imports -import org.apache.pekko.pattern.Patterns; // #imports import static java.util.concurrent.TimeUnit.SECONDS; @@ -35,8 +34,8 @@ import static java.util.concurrent.TimeUnit.SECONDS; public class FutureDocTest extends AbstractJavaTest { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("FutureDocTest", AkkaSpec.testConf()); + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("FutureDocTest", PekkoSpec.testConf()); private final ActorSystem system = toTyped(actorSystemResource.getSystem()); diff --git a/docs/src/test/java/jdocs/io/japi/IODocTest.java b/docs/src/test/java/jdocs/io/japi/IODocTest.java index b897255614..b3df14a067 100644 --- a/docs/src/test/java/jdocs/io/japi/IODocTest.java +++ b/docs/src/test/java/jdocs/io/japi/IODocTest.java @@ -4,7 +4,7 @@ package jdocs.io.japi; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import jdocs.AbstractJavaTest; import org.apache.pekko.testkit.javadsl.TestKit; import org.junit.ClassRule; @@ -26,7 +26,7 @@ import org.apache.pekko.io.TcpMessage; import org.apache.pekko.util.ByteString; // #imports -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoSpec; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -173,8 +173,8 @@ public class IODocTest extends AbstractJavaTest { // #client @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("IODocTest", AkkaSpec.testConf()); + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("IODocTest", PekkoSpec.testConf()); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/docs/src/test/java/jdocs/remoting/RemoteDeploymentDocTest.java b/docs/src/test/java/jdocs/remoting/RemoteDeploymentDocTest.java index af45808088..521f0629b4 100644 --- a/docs/src/test/java/jdocs/remoting/RemoteDeploymentDocTest.java +++ b/docs/src/test/java/jdocs/remoting/RemoteDeploymentDocTest.java @@ -4,8 +4,8 @@ package jdocs.remoting; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; -import org.apache.pekko.testkit.AkkaSpec; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoSpec; import jdocs.AbstractJavaTest; import org.junit.ClassRule; import org.junit.Test; @@ -36,15 +36,15 @@ public class RemoteDeploymentDocTest extends AbstractJavaTest { } @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource( + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource( "RemoteDeploymentDocTest", ConfigFactory.parseString( " pekko.actor.provider = remote\n" + " pekko.remote.classic.netty.tcp.port = 0\n" + " pekko.remote.artery.canonical.port = 0\n" + " pekko.remote.use-unsafe-remote-features-outside-cluster = on") - .withFallback(AkkaSpec.testConf())); + .withFallback(PekkoSpec.testConf())); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/docs/src/test/java/jdocs/routing/ConsistentHashingRouterDocTest.java b/docs/src/test/java/jdocs/routing/ConsistentHashingRouterDocTest.java index 0ea8b0b11e..5b7617e6d4 100644 --- a/docs/src/test/java/jdocs/routing/ConsistentHashingRouterDocTest.java +++ b/docs/src/test/java/jdocs/routing/ConsistentHashingRouterDocTest.java @@ -4,7 +4,7 @@ package jdocs.routing; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import jdocs.AbstractJavaTest; import org.apache.pekko.testkit.javadsl.TestKit; @@ -33,8 +33,8 @@ import org.apache.pekko.routing.ConsistentHashingRouter.ConsistentHashableEnvelo public class ConsistentHashingRouterDocTest extends AbstractJavaTest { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource("ConsistentHashingRouterDocTest"); + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource("ConsistentHashingRouterDocTest"); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/docs/src/test/java/jdocs/routing/CustomRouterDocTest.java b/docs/src/test/java/jdocs/routing/CustomRouterDocTest.java index f08a17f2bd..16f9a4444c 100644 --- a/docs/src/test/java/jdocs/routing/CustomRouterDocTest.java +++ b/docs/src/test/java/jdocs/routing/CustomRouterDocTest.java @@ -9,7 +9,7 @@ import org.apache.pekko.routing.RoundRobinRoutingLogic; import org.apache.pekko.routing.Routee; import org.apache.pekko.routing.RoutingLogic; import org.apache.pekko.routing.SeveralRoutees; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import jdocs.AbstractJavaTest; import org.apache.pekko.testkit.javadsl.TestKit; @@ -38,8 +38,8 @@ import java.util.List; public class CustomRouterDocTest extends AbstractJavaTest { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource( + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource( "CustomRouterDocTest", ConfigFactory.parseString(CustomRouterDocSpec.jconfig())); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/docs/src/test/java/jdocs/routing/RouterDocTest.java b/docs/src/test/java/jdocs/routing/RouterDocTest.java index 09972cd788..0074de6427 100644 --- a/docs/src/test/java/jdocs/routing/RouterDocTest.java +++ b/docs/src/test/java/jdocs/routing/RouterDocTest.java @@ -4,7 +4,7 @@ package jdocs.routing; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import jdocs.AbstractJavaTest; import org.apache.pekko.testkit.javadsl.TestKit; @@ -67,8 +67,8 @@ import org.apache.pekko.routing.TailChoppingPool; public class RouterDocTest extends AbstractJavaTest { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource( + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource( "RouterDocTest", ConfigFactory.parseString(docs.routing.RouterDocSpec.config())); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/docs/src/test/java/jdocs/testkit/ParentChildTest.java b/docs/src/test/java/jdocs/testkit/ParentChildTest.java index 997d8f4abf..238fa4c4cd 100644 --- a/docs/src/test/java/jdocs/testkit/ParentChildTest.java +++ b/docs/src/test/java/jdocs/testkit/ParentChildTest.java @@ -7,7 +7,7 @@ package jdocs.testkit; import org.apache.pekko.actor.*; import org.apache.pekko.japi.Creator; import org.apache.pekko.japi.Function; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import org.apache.pekko.testkit.TestProbe; import org.apache.pekko.testkit.javadsl.TestKit; import com.typesafe.config.ConfigFactory; @@ -18,8 +18,8 @@ import org.junit.Test; public class ParentChildTest extends AbstractJavaTest { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource( + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource( "TestKitDocTest", ConfigFactory.parseString( "pekko.loggers = [org.apache.pekko.testkit.TestEventListener]")); diff --git a/docs/src/test/java/jdocs/testkit/TestKitDocTest.java b/docs/src/test/java/jdocs/testkit/TestKitDocTest.java index ffde2ced2f..9e731915c4 100644 --- a/docs/src/test/java/jdocs/testkit/TestKitDocTest.java +++ b/docs/src/test/java/jdocs/testkit/TestKitDocTest.java @@ -10,7 +10,7 @@ import org.apache.pekko.pattern.Patterns; import jdocs.AbstractJavaTest; import org.junit.Assert; import org.apache.pekko.japi.JavaPartialFunction; -import org.apache.pekko.testkit.AkkaJUnitActorSystemResource; +import org.apache.pekko.testkit.PekkoJUnitActorSystemResource; import org.apache.pekko.testkit.CallingThreadDispatcher; import org.apache.pekko.testkit.TestActor; import org.apache.pekko.testkit.TestActorRef; @@ -41,8 +41,8 @@ import java.time.Duration; public class TestKitDocTest extends AbstractJavaTest { @ClassRule - public static AkkaJUnitActorSystemResource actorSystemResource = - new AkkaJUnitActorSystemResource( + public static PekkoJUnitActorSystemResource actorSystemResource = + new PekkoJUnitActorSystemResource( "TestKitDocTest", ConfigFactory.parseString( "pekko.loggers = [org.apache.pekko.testkit.TestEventListener]")); diff --git a/docs/src/test/scala/docs/actor/ActorDocSpec.scala b/docs/src/test/scala/docs/actor/ActorDocSpec.scala index 8d939fa474..b7a1102198 100644 --- a/docs/src/test/scala/docs/actor/ActorDocSpec.scala +++ b/docs/src/test/scala/docs/actor/ActorDocSpec.scala @@ -319,7 +319,7 @@ case class User(name: String) case class Register(user: User) //#immutable-message-definition -class ActorDocSpec extends AkkaSpec(""" +class ActorDocSpec extends PekkoSpec(""" pekko.loglevel = INFO pekko.loggers = [] """) { @@ -346,7 +346,7 @@ class ActorDocSpec extends AkkaSpec(""" // testing the actor - // TODO: convert docs to AkkaSpec(Map(...)) + // TODO: convert docs to PekkoSpec(Map(...)) val filter = EventFilter.custom { case e: Logging.Info => true case _ => false diff --git a/docs/src/test/scala/docs/actor/FSMDocSpec.scala b/docs/src/test/scala/docs/actor/FSMDocSpec.scala index 58f9aa6e00..7977c3b058 100644 --- a/docs/src/test/scala/docs/actor/FSMDocSpec.scala +++ b/docs/src/test/scala/docs/actor/FSMDocSpec.scala @@ -6,7 +6,7 @@ package docs.actor import language.postfixOps -import org.apache.pekko.testkit.{ AkkaSpec => MyFavoriteTestFrameWorkPlusAkkaTestKit } +import org.apache.pekko.testkit.{ PekkoSpec => MyFavoriteTestFrameWorkPlusPekkoTestKit } import org.apache.pekko.util.ByteString //#test-code @@ -40,7 +40,7 @@ object FSMDocSpec { // #test-code } -class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit { +class FSMDocSpec extends MyFavoriteTestFrameWorkPlusPekkoTestKit { import FSMDocSpec._ // #fsm-code-elided diff --git a/docs/src/test/scala/docs/actor/InitializationDocSpec.scala b/docs/src/test/scala/docs/actor/InitializationDocSpec.scala index b9760b049c..b6b7d5e16b 100644 --- a/docs/src/test/scala/docs/actor/InitializationDocSpec.scala +++ b/docs/src/test/scala/docs/actor/InitializationDocSpec.scala @@ -5,7 +5,7 @@ package docs.actor import org.apache.pekko.actor.{ Actor, Props } -import org.apache.pekko.testkit.{ AkkaSpec, ImplicitSender } +import org.apache.pekko.testkit.{ ImplicitSender, PekkoSpec } object InitializationDocSpec { @@ -52,7 +52,7 @@ object InitializationDocSpec { } } -class InitializationDocSpec extends AkkaSpec with ImplicitSender { +class InitializationDocSpec extends PekkoSpec with ImplicitSender { import InitializationDocSpec._ "Message based initialization example" must { diff --git a/docs/src/test/scala/docs/actor/SchedulerDocSpec.scala b/docs/src/test/scala/docs/actor/SchedulerDocSpec.scala index d825fd317c..cf6a52f4c5 100644 --- a/docs/src/test/scala/docs/actor/SchedulerDocSpec.scala +++ b/docs/src/test/scala/docs/actor/SchedulerDocSpec.scala @@ -16,7 +16,7 @@ import scala.concurrent.duration._ import pekko.testkit._ -class SchedulerDocSpec extends AkkaSpec(Map("pekko.loglevel" -> "INFO")) { +class SchedulerDocSpec extends PekkoSpec(Map("pekko.loglevel" -> "INFO")) { "schedule a one-off task" in { // #schedule-one-off-message // Use the system's dispatcher as ExecutionContext diff --git a/docs/src/test/scala/docs/cluster/ClusterDocSpec.scala b/docs/src/test/scala/docs/cluster/ClusterDocSpec.scala index aacf55e7ff..25218a31e9 100644 --- a/docs/src/test/scala/docs/cluster/ClusterDocSpec.scala +++ b/docs/src/test/scala/docs/cluster/ClusterDocSpec.scala @@ -6,7 +6,7 @@ package scala.docs.cluster import org.apache.pekko.cluster.Cluster -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import docs.CompileOnlySpec object ClusterDocSpec { @@ -18,7 +18,7 @@ object ClusterDocSpec { """ } -class ClusterDocSpec extends AkkaSpec(ClusterDocSpec.config) with CompileOnlySpec { +class ClusterDocSpec extends PekkoSpec(ClusterDocSpec.config) with CompileOnlySpec { "demonstrate leave" in compileOnlySpec { // #leave diff --git a/docs/src/test/scala/docs/coordination/LeaseDocSpec.scala b/docs/src/test/scala/docs/coordination/LeaseDocSpec.scala index b2e19e3875..4257c7dfe7 100644 --- a/docs/src/test/scala/docs/coordination/LeaseDocSpec.scala +++ b/docs/src/test/scala/docs/coordination/LeaseDocSpec.scala @@ -12,7 +12,7 @@ import org.apache.pekko.cluster.Cluster import org.apache.pekko.coordination.lease.LeaseSettings import org.apache.pekko.coordination.lease.scaladsl.Lease import org.apache.pekko.coordination.lease.scaladsl.LeaseProvider -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec //#lease-example class SampleLease(settings: LeaseSettings) extends Lease(settings) { @@ -62,7 +62,7 @@ object LeaseDocSpec { } } -class LeaseDocSpec extends AkkaSpec(LeaseDocSpec.config) { +class LeaseDocSpec extends PekkoSpec(LeaseDocSpec.config) { import LeaseDocSpec._ "A docs lease" should { diff --git a/docs/src/test/scala/docs/ddata/DistributedDataDocSpec.scala b/docs/src/test/scala/docs/ddata/DistributedDataDocSpec.scala index 000bb7eccc..43d087321f 100644 --- a/docs/src/test/scala/docs/ddata/DistributedDataDocSpec.scala +++ b/docs/src/test/scala/docs/ddata/DistributedDataDocSpec.scala @@ -8,7 +8,7 @@ import scala.concurrent.duration._ import org.apache.pekko.actor.Actor import org.apache.pekko.cluster.ddata._ -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import org.apache.pekko.testkit.TestProbe import org.apache.pekko.actor.ActorRef import org.apache.pekko.serialization.SerializationExtension @@ -99,7 +99,7 @@ object DistributedDataDocSpec { } -class DistributedDataDocSpec extends AkkaSpec(DistributedDataDocSpec.config) { +class DistributedDataDocSpec extends PekkoSpec(DistributedDataDocSpec.config) { import Replicator._ "demonstrate update" in { diff --git a/docs/src/test/scala/docs/discovery/DnsDiscoveryDocSpec.scala b/docs/src/test/scala/docs/discovery/DnsDiscoveryDocSpec.scala index 39755e16b2..89e5240aed 100644 --- a/docs/src/test/scala/docs/discovery/DnsDiscoveryDocSpec.scala +++ b/docs/src/test/scala/docs/discovery/DnsDiscoveryDocSpec.scala @@ -4,7 +4,7 @@ package docs.discovery -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import com.typesafe.config.ConfigFactory import scala.concurrent.duration._ @@ -22,7 +22,7 @@ object DnsDiscoveryDocSpec { """) } -class DnsDiscoveryDocSpec extends AkkaSpec(DnsDiscoveryDocSpec.config) { +class DnsDiscoveryDocSpec extends PekkoSpec(DnsDiscoveryDocSpec.config) { "DNS Discovery" should { "find pekko.io" in { diff --git a/docs/src/test/scala/docs/dispatcher/DispatcherDocSpec.scala b/docs/src/test/scala/docs/dispatcher/DispatcherDocSpec.scala index 6257bf692a..3b52700c5f 100644 --- a/docs/src/test/scala/docs/dispatcher/DispatcherDocSpec.scala +++ b/docs/src/test/scala/docs/dispatcher/DispatcherDocSpec.scala @@ -6,7 +6,7 @@ package docs.dispatcher import language.postfixOps -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import org.apache.pekko.event.Logging import org.apache.pekko.event.LoggingAdapter import org.apache.pekko.actor._ @@ -287,7 +287,7 @@ object DispatcherDocSpec { // #require-mailbox-on-actor } -class DispatcherDocSpec extends AkkaSpec(DispatcherDocSpec.config) { +class DispatcherDocSpec extends PekkoSpec(DispatcherDocSpec.config) { import DispatcherDocSpec._ diff --git a/docs/src/test/scala/docs/event/EventBusDocSpec.scala b/docs/src/test/scala/docs/event/EventBusDocSpec.scala index a49b870137..f0b4dcb208 100644 --- a/docs/src/test/scala/docs/event/EventBusDocSpec.scala +++ b/docs/src/test/scala/docs/event/EventBusDocSpec.scala @@ -5,7 +5,7 @@ package docs.event import scala.concurrent.duration._ -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import org.apache.pekko.actor.{ ActorRef, ActorSystem } import org.apache.pekko.testkit.TestProbe @@ -146,7 +146,7 @@ object EventBusDocSpec { } -class EventBusDocSpec extends AkkaSpec { +class EventBusDocSpec extends PekkoSpec { import EventBusDocSpec._ "demonstrate LookupClassification" in { diff --git a/docs/src/test/scala/docs/event/LoggingDocSpec.scala b/docs/src/test/scala/docs/event/LoggingDocSpec.scala index a75dec72a5..c5e7cba860 100644 --- a/docs/src/test/scala/docs/event/LoggingDocSpec.scala +++ b/docs/src/test/scala/docs/event/LoggingDocSpec.scala @@ -6,7 +6,7 @@ package docs.event import org.apache.pekko import pekko.actor.{ Actor, DeadLetter, Props } -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec object LoggingDocSpec { @@ -144,7 +144,7 @@ object LoggingDocSpec { } -class LoggingDocSpec extends AkkaSpec { +class LoggingDocSpec extends PekkoSpec { import LoggingDocSpec.{ MdcActor, MdcActorMixin, MyActor, Req } diff --git a/docs/src/test/scala/docs/extension/ExtensionDocSpec.scala b/docs/src/test/scala/docs/extension/ExtensionDocSpec.scala index de2d65a52c..fbd64e2f61 100644 --- a/docs/src/test/scala/docs/extension/ExtensionDocSpec.scala +++ b/docs/src/test/scala/docs/extension/ExtensionDocSpec.scala @@ -8,7 +8,7 @@ import java.util.concurrent.atomic.AtomicLong import org.apache.pekko.actor.Actor import org.apache.pekko.actor.ClassicActorSystemProvider -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec //#extension import org.apache.pekko.actor.Extension @@ -82,7 +82,7 @@ object ExtensionDocSpec { // #extension-usage-actor-trait } -class ExtensionDocSpec extends AkkaSpec(ExtensionDocSpec.config) { +class ExtensionDocSpec extends PekkoSpec(ExtensionDocSpec.config) { "demonstrate how to create an extension in Scala" in { // #extension-usage diff --git a/docs/src/test/scala/docs/extension/SettingsExtensionDocSpec.scala b/docs/src/test/scala/docs/extension/SettingsExtensionDocSpec.scala index 2feb8e7816..464fd6adaa 100644 --- a/docs/src/test/scala/docs/extension/SettingsExtensionDocSpec.scala +++ b/docs/src/test/scala/docs/extension/SettingsExtensionDocSpec.scala @@ -21,7 +21,7 @@ import pekko.actor.ClassicActorSystemProvider //#imports import org.apache.pekko.actor.Actor -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec //#extension class SettingsImpl(config: Config) extends Extension { @@ -80,7 +80,7 @@ object SettingsExtensionDocSpec { } -class SettingsExtensionDocSpec extends AkkaSpec(SettingsExtensionDocSpec.config) { +class SettingsExtensionDocSpec extends PekkoSpec(SettingsExtensionDocSpec.config) { "demonstrate how to create application specific settings extension in Scala" in { // #extension-usage diff --git a/docs/src/test/scala/docs/future/FutureDocSpec.scala b/docs/src/test/scala/docs/future/FutureDocSpec.scala index d452487cd4..fc1493a981 100644 --- a/docs/src/test/scala/docs/future/FutureDocSpec.scala +++ b/docs/src/test/scala/docs/future/FutureDocSpec.scala @@ -128,7 +128,7 @@ object FutureDocSpec { } -class FutureDocSpec extends AkkaSpec { +class FutureDocSpec extends PekkoSpec { import FutureDocSpec._ import system.dispatcher diff --git a/docs/src/test/scala/docs/io/IODocSpec.scala b/docs/src/test/scala/docs/io/IODocSpec.scala index 758623e126..54462e44a6 100644 --- a/docs/src/test/scala/docs/io/IODocSpec.scala +++ b/docs/src/test/scala/docs/io/IODocSpec.scala @@ -12,7 +12,7 @@ import pekko.util.ByteString import java.net.InetSocketAddress //#imports -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import scala.concurrent.duration._ class DemoActor extends Actor { @@ -104,7 +104,7 @@ class Client(remote: InetSocketAddress, listener: ActorRef) extends Actor { } //#client -class IODocSpec extends AkkaSpec { +class IODocSpec extends PekkoSpec { class Parent extends Actor { context.actorOf(Props[Server](), "server") diff --git a/docs/src/test/scala/docs/io/ReadBackPressure.scala b/docs/src/test/scala/docs/io/ReadBackPressure.scala index 95b457002d..874aa82b28 100644 --- a/docs/src/test/scala/docs/io/ReadBackPressure.scala +++ b/docs/src/test/scala/docs/io/ReadBackPressure.scala @@ -8,7 +8,7 @@ import org.apache.pekko.actor.{ Actor, ActorLogging, ActorRef, ActorSystem, Prop import org.apache.pekko.io.Tcp._ import org.apache.pekko.io.{ IO, Tcp } import java.net.InetSocketAddress -import org.apache.pekko.testkit.{ AkkaSpec, ImplicitSender, TestProbe } +import org.apache.pekko.testkit.{ ImplicitSender, PekkoSpec, TestProbe } import org.apache.pekko.util.ByteString import scala.concurrent.Await @@ -62,7 +62,7 @@ object PullReadingExample { } -class PullReadingSpec extends AkkaSpec with ImplicitSender { +class PullReadingSpec extends PekkoSpec with ImplicitSender { "demonstrate pull reading" in { val probe = TestProbe() diff --git a/docs/src/test/scala/docs/io/UdpDocSpec.scala b/docs/src/test/scala/docs/io/UdpDocSpec.scala index 5e8e5b4f49..1c84da463c 100644 --- a/docs/src/test/scala/docs/io/UdpDocSpec.scala +++ b/docs/src/test/scala/docs/io/UdpDocSpec.scala @@ -4,7 +4,7 @@ package docs.io -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import org.apache.pekko.actor.Actor import org.apache.pekko.io.IO import org.apache.pekko.io.Udp @@ -100,7 +100,7 @@ object ScalaUdpDocSpec { } -abstract class UdpDocSpec extends AkkaSpec { +abstract class UdpDocSpec extends PekkoSpec { def listenerProps(next: ActorRef): Props def simpleSenderProps(remote: InetSocketAddress): Props diff --git a/docs/src/test/scala/docs/persistence/PersistenceEventAdapterDocSpec.scala b/docs/src/test/scala/docs/persistence/PersistenceEventAdapterDocSpec.scala index cd5040dcc0..e204cae93f 100644 --- a/docs/src/test/scala/docs/persistence/PersistenceEventAdapterDocSpec.scala +++ b/docs/src/test/scala/docs/persistence/PersistenceEventAdapterDocSpec.scala @@ -7,12 +7,12 @@ package docs.persistence import org.apache.pekko.actor.{ ExtendedActorSystem, Props } import org.apache.pekko.persistence.journal.{ EventAdapter, EventSeq } import org.apache.pekko.persistence.{ PersistentActor, RecoveryCompleted } -import org.apache.pekko.testkit.{ AkkaSpec, TestProbe } +import org.apache.pekko.testkit.{ PekkoSpec, TestProbe } import com.google.gson.{ Gson, JsonElement } import scala.collection.immutable -class PersistenceEventAdapterDocSpec(config: String) extends AkkaSpec(config) { +class PersistenceEventAdapterDocSpec(config: String) extends PekkoSpec(config) { def this() = this(""" diff --git a/docs/src/test/scala/docs/persistence/query/LeveldbPersistenceQueryDocSpec.scala b/docs/src/test/scala/docs/persistence/query/LeveldbPersistenceQueryDocSpec.scala index 441c8a55bf..ee25fc2eb8 100644 --- a/docs/src/test/scala/docs/persistence/query/LeveldbPersistenceQueryDocSpec.scala +++ b/docs/src/test/scala/docs/persistence/query/LeveldbPersistenceQueryDocSpec.scala @@ -5,7 +5,7 @@ package docs.persistence.query import org.apache.pekko.NotUsed -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import org.apache.pekko.persistence.query.{ EventEnvelope, PersistenceQuery, Sequence } import org.apache.pekko.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal import org.apache.pekko.stream.scaladsl.Source @@ -34,7 +34,7 @@ object LeveldbPersistenceQueryDocSpec { } class LeveldbPersistenceQueryDocSpec - extends AkkaSpec("pekko.persistence.journal.plugin = pekko.persistence.journal.leveldb") { + extends PekkoSpec("pekko.persistence.journal.plugin = pekko.persistence.journal.leveldb") { "LeveldbPersistentQuery" must { "demonstrate how get ReadJournal" in { diff --git a/docs/src/test/scala/docs/persistence/query/PersistenceQueryDocSpec.scala b/docs/src/test/scala/docs/persistence/query/PersistenceQueryDocSpec.scala index b61a2ef08b..765b4de94a 100644 --- a/docs/src/test/scala/docs/persistence/query/PersistenceQueryDocSpec.scala +++ b/docs/src/test/scala/docs/persistence/query/PersistenceQueryDocSpec.scala @@ -10,7 +10,7 @@ import pekko.actor._ import pekko.persistence.query._ import pekko.stream.scaladsl.{ Flow, Sink, Source } import pekko.stream.javadsl -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import pekko.util.Timeout import org.reactivestreams.Subscriber import scala.collection.immutable @@ -174,7 +174,7 @@ object PersistenceQueryDocSpec { } -class PersistenceQueryDocSpec(s: String) extends AkkaSpec(s) { +class PersistenceQueryDocSpec(s: String) extends PekkoSpec(s) { import PersistenceQueryDocSpec._ def this() = diff --git a/docs/src/test/scala/docs/remoting/RemoteDeploymentDocSpec.scala b/docs/src/test/scala/docs/remoting/RemoteDeploymentDocSpec.scala index 5452c5e8a6..13ef36c48e 100644 --- a/docs/src/test/scala/docs/remoting/RemoteDeploymentDocSpec.scala +++ b/docs/src/test/scala/docs/remoting/RemoteDeploymentDocSpec.scala @@ -5,7 +5,7 @@ package docs.remoting import org.apache.pekko.actor.{ Actor, ActorRef, ActorSystem, ExtendedActorSystem } -import org.apache.pekko.testkit.{ AkkaSpec, ImplicitSender } +import org.apache.pekko.testkit.{ ImplicitSender, PekkoSpec } //#import import org.apache.pekko import pekko.actor.{ Address, AddressFromURIString, Deploy, Props } @@ -20,7 +20,7 @@ object RemoteDeploymentDocSpec { } -class RemoteDeploymentDocSpec extends AkkaSpec(""" +class RemoteDeploymentDocSpec extends PekkoSpec(""" pekko.actor.provider = remote pekko.remote.classic.netty.tcp.port = 0 pekko.remote.artery.canonical.port = 0 diff --git a/docs/src/test/scala/docs/routing/ConsistentHashingRouterDocSpec.scala b/docs/src/test/scala/docs/routing/ConsistentHashingRouterDocSpec.scala index ed17e8791e..e68686ef51 100644 --- a/docs/src/test/scala/docs/routing/ConsistentHashingRouterDocSpec.scala +++ b/docs/src/test/scala/docs/routing/ConsistentHashingRouterDocSpec.scala @@ -4,7 +4,7 @@ package docs.routing -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import org.apache.pekko.testkit.ImplicitSender import org.apache.pekko.routing.FromConfig import org.apache.pekko.actor.ActorRef @@ -37,7 +37,7 @@ object ConsistentHashingRouterDocSpec { } -class ConsistentHashingRouterDocSpec extends AkkaSpec with ImplicitSender { +class ConsistentHashingRouterDocSpec extends PekkoSpec with ImplicitSender { import ConsistentHashingRouterDocSpec._ diff --git a/docs/src/test/scala/docs/routing/CustomRouterDocSpec.scala b/docs/src/test/scala/docs/routing/CustomRouterDocSpec.scala index aa4ec76389..713f49233e 100644 --- a/docs/src/test/scala/docs/routing/CustomRouterDocSpec.scala +++ b/docs/src/test/scala/docs/routing/CustomRouterDocSpec.scala @@ -4,7 +4,7 @@ package docs.routing -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import org.apache.pekko.testkit.ImplicitSender import org.apache.pekko.actor.Actor import org.apache.pekko.actor.Props @@ -94,7 +94,7 @@ final case class RedundancyGroup(routeePaths: immutable.Iterable[String], nbrCop } //#group -class CustomRouterDocSpec extends AkkaSpec(CustomRouterDocSpec.config) with ImplicitSender { +class CustomRouterDocSpec extends PekkoSpec(CustomRouterDocSpec.config) with ImplicitSender { import CustomRouterDocSpec._ import org.apache.pekko.routing.SeveralRoutees diff --git a/docs/src/test/scala/docs/routing/RouterDocSpec.scala b/docs/src/test/scala/docs/routing/RouterDocSpec.scala index 5784ac4e16..d5efcc31f5 100644 --- a/docs/src/test/scala/docs/routing/RouterDocSpec.scala +++ b/docs/src/test/scala/docs/routing/RouterDocSpec.scala @@ -523,7 +523,7 @@ router-dispatcher {} } } -class RouterDocSpec extends AkkaSpec(RouterDocSpec.config) with ImplicitSender { +class RouterDocSpec extends PekkoSpec(RouterDocSpec.config) with ImplicitSender { import RouterDocSpec._ diff --git a/docs/src/test/scala/docs/serialization/SerializationDocSpec.scala b/docs/src/test/scala/docs/serialization/SerializationDocSpec.scala index 1f7b51265e..324504b5af 100644 --- a/docs/src/test/scala/docs/serialization/SerializationDocSpec.scala +++ b/docs/src/test/scala/docs/serialization/SerializationDocSpec.scala @@ -125,7 +125,7 @@ package docs.serialization { """ } - class SerializationDocSpec extends AkkaSpec { + class SerializationDocSpec extends PekkoSpec { "demonstrate configuration of serialize messages" in { val config = ConfigFactory.parseString(""" #//#serialize-messages-config diff --git a/docs/src/test/scala/docs/stream/BidiFlowDocSpec.scala b/docs/src/test/scala/docs/stream/BidiFlowDocSpec.scala index 4a684b6c8a..51587af632 100644 --- a/docs/src/test/scala/docs/stream/BidiFlowDocSpec.scala +++ b/docs/src/test/scala/docs/stream/BidiFlowDocSpec.scala @@ -4,7 +4,7 @@ package docs.stream -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import org.apache.pekko.stream.scaladsl._ import org.apache.pekko.stream._ import org.apache.pekko.util.ByteString @@ -147,7 +147,7 @@ object BidiFlowDocSpec { }) } -class BidiFlowDocSpec extends AkkaSpec { +class BidiFlowDocSpec extends PekkoSpec { import BidiFlowDocSpec._ "A BidiFlow" must { diff --git a/docs/src/test/scala/docs/stream/CompositionDocSpec.scala b/docs/src/test/scala/docs/stream/CompositionDocSpec.scala index 566acdc63b..8b5cb3f6a8 100644 --- a/docs/src/test/scala/docs/stream/CompositionDocSpec.scala +++ b/docs/src/test/scala/docs/stream/CompositionDocSpec.scala @@ -8,13 +8,13 @@ import org.apache.pekko.NotUsed import org.apache.pekko.stream._ import org.apache.pekko.stream.scaladsl.Tcp.OutgoingConnection import org.apache.pekko.stream.scaladsl._ -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import org.apache.pekko.util.ByteString import scala.concurrent.{ Future, Promise } import scala.concurrent.ExecutionContext -class CompositionDocSpec extends AkkaSpec { +class CompositionDocSpec extends PekkoSpec { implicit val ec: ExecutionContext = system.dispatcher diff --git a/docs/src/test/scala/docs/stream/FlowDocSpec.scala b/docs/src/test/scala/docs/stream/FlowDocSpec.scala index f2a79c14ad..5db2e7f130 100644 --- a/docs/src/test/scala/docs/stream/FlowDocSpec.scala +++ b/docs/src/test/scala/docs/stream/FlowDocSpec.scala @@ -11,14 +11,14 @@ import org.apache.pekko.stream.CompletionStrategy import org.apache.pekko.stream.Materializer import org.apache.pekko.stream.{ ClosedShape, FlowShape, OverflowStrategy } import org.apache.pekko.stream.scaladsl._ -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import docs.CompileOnlySpec import scala.concurrent.{ Future, Promise } import scala.util.{ Failure, Success } import scala.concurrent.ExecutionContext -class FlowDocSpec extends AkkaSpec with CompileOnlySpec { +class FlowDocSpec extends PekkoSpec with CompileOnlySpec { implicit val ec: ExecutionContext = system.dispatcher diff --git a/docs/src/test/scala/docs/stream/FlowErrorDocSpec.scala b/docs/src/test/scala/docs/stream/FlowErrorDocSpec.scala index 20cbc60ed5..4a09e99e25 100644 --- a/docs/src/test/scala/docs/stream/FlowErrorDocSpec.scala +++ b/docs/src/test/scala/docs/stream/FlowErrorDocSpec.scala @@ -7,11 +7,11 @@ package docs.stream import scala.concurrent.Await import org.apache.pekko.stream.Supervision import org.apache.pekko.stream.scaladsl._ -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import org.apache.pekko.stream.ActorAttributes import scala.concurrent.duration._ -class FlowErrorDocSpec extends AkkaSpec { +class FlowErrorDocSpec extends PekkoSpec { "demonstrate fail stream" in { // #stop diff --git a/docs/src/test/scala/docs/stream/FlowParallelismDocSpec.scala b/docs/src/test/scala/docs/stream/FlowParallelismDocSpec.scala index 0fa22d1a7a..f790fdf116 100644 --- a/docs/src/test/scala/docs/stream/FlowParallelismDocSpec.scala +++ b/docs/src/test/scala/docs/stream/FlowParallelismDocSpec.scala @@ -7,9 +7,9 @@ package docs.stream import org.apache.pekko.NotUsed import org.apache.pekko.stream.FlowShape import org.apache.pekko.stream.scaladsl.{ Balance, Flow, GraphDSL, Merge, Source } -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec -class FlowParallelismDocSpec extends AkkaSpec { +class FlowParallelismDocSpec extends PekkoSpec { import GraphDSL.Implicits._ diff --git a/docs/src/test/scala/docs/stream/FlowStreamRefsDocSpec.scala b/docs/src/test/scala/docs/stream/FlowStreamRefsDocSpec.scala index 9f1dccda4c..dccf7ac4f3 100644 --- a/docs/src/test/scala/docs/stream/FlowStreamRefsDocSpec.scala +++ b/docs/src/test/scala/docs/stream/FlowStreamRefsDocSpec.scala @@ -7,10 +7,10 @@ package docs.stream import org.apache.pekko.NotUsed import org.apache.pekko.actor.{ Actor, Props } import org.apache.pekko.stream.scaladsl._ -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import docs.CompileOnlySpec -class FlowStreamRefsDocSpec extends AkkaSpec with CompileOnlySpec { +class FlowStreamRefsDocSpec extends PekkoSpec with CompileOnlySpec { "offer a source ref" in compileOnlySpec { // #offer-source diff --git a/docs/src/test/scala/docs/stream/GraphCyclesSpec.scala b/docs/src/test/scala/docs/stream/GraphCyclesSpec.scala index 4bef7f7b78..130d2e2508 100644 --- a/docs/src/test/scala/docs/stream/GraphCyclesSpec.scala +++ b/docs/src/test/scala/docs/stream/GraphCyclesSpec.scala @@ -6,9 +6,9 @@ package docs.stream import org.apache.pekko.stream.{ ClosedShape, OverflowStrategy } import org.apache.pekko.stream.scaladsl._ -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec -class GraphCyclesSpec extends AkkaSpec { +class GraphCyclesSpec extends PekkoSpec { "Cycle demonstration" must { val source = Source.fromIterator(() => Iterator.from(0)) diff --git a/docs/src/test/scala/docs/stream/GraphDSLDocSpec.scala b/docs/src/test/scala/docs/stream/GraphDSLDocSpec.scala index 8f539e2b7d..dfe214ae20 100644 --- a/docs/src/test/scala/docs/stream/GraphDSLDocSpec.scala +++ b/docs/src/test/scala/docs/stream/GraphDSLDocSpec.scala @@ -7,14 +7,14 @@ package docs.stream import org.apache.pekko.NotUsed import org.apache.pekko.stream._ import org.apache.pekko.stream.scaladsl._ -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import scala.collection.immutable import scala.concurrent.duration._ import scala.concurrent.{ Await, Future } import scala.concurrent.ExecutionContext -class GraphDSLDocSpec extends AkkaSpec { +class GraphDSLDocSpec extends PekkoSpec { implicit val ec: ExecutionContext = system.dispatcher diff --git a/docs/src/test/scala/docs/stream/GraphStageDocSpec.scala b/docs/src/test/scala/docs/stream/GraphStageDocSpec.scala index 0449cdecf4..98c8c333bc 100644 --- a/docs/src/test/scala/docs/stream/GraphStageDocSpec.scala +++ b/docs/src/test/scala/docs/stream/GraphStageDocSpec.scala @@ -9,14 +9,14 @@ import org.apache.pekko.stream.scaladsl.{ Flow, Keep, Sink, Source } import org.apache.pekko.stream.stage._ import org.apache.pekko.stream._ import org.apache.pekko.stream.testkit.{ TestPublisher, TestSubscriber } -import org.apache.pekko.testkit.{ AkkaSpec, TestLatch } +import org.apache.pekko.testkit.{ PekkoSpec, TestLatch } import scala.collection.mutable import scala.concurrent.{ Await, Future, Promise } import scala.concurrent.duration._ import scala.collection.immutable.Iterable -class GraphStageDocSpec extends AkkaSpec { +class GraphStageDocSpec extends PekkoSpec { "Demonstrate creation of GraphStage boilerplate" in { // #boilerplate-example diff --git a/docs/src/test/scala/docs/stream/GraphStageLoggingDocSpec.scala b/docs/src/test/scala/docs/stream/GraphStageLoggingDocSpec.scala index 3cd8d46cc6..27f37eae90 100644 --- a/docs/src/test/scala/docs/stream/GraphStageLoggingDocSpec.scala +++ b/docs/src/test/scala/docs/stream/GraphStageLoggingDocSpec.scala @@ -8,10 +8,10 @@ import java.util.concurrent.ThreadLocalRandom import org.apache.pekko.stream._ import org.apache.pekko.stream.scaladsl._ -import org.apache.pekko.testkit.{ AkkaSpec, EventFilter } +import org.apache.pekko.testkit.{ EventFilter, PekkoSpec } import scala.concurrent.ExecutionContext -class GraphStageLoggingDocSpec extends AkkaSpec("pekko.loglevel = DEBUG") { +class GraphStageLoggingDocSpec extends PekkoSpec("pekko.loglevel = DEBUG") { implicit val ec: ExecutionContext = system.dispatcher diff --git a/docs/src/test/scala/docs/stream/HubsDocSpec.scala b/docs/src/test/scala/docs/stream/HubsDocSpec.scala index c9c98bb057..bcd81dd53b 100644 --- a/docs/src/test/scala/docs/stream/HubsDocSpec.scala +++ b/docs/src/test/scala/docs/stream/HubsDocSpec.scala @@ -7,13 +7,13 @@ package docs.stream import org.apache.pekko.NotUsed import org.apache.pekko.stream.{ KillSwitches, UniqueKillSwitch } import org.apache.pekko.stream.scaladsl._ -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import docs.CompileOnlySpec import scala.concurrent.duration._ import org.apache.pekko.stream.ThrottleMode -class HubsDocSpec extends AkkaSpec with CompileOnlySpec { +class HubsDocSpec extends PekkoSpec with CompileOnlySpec { "Hubs" must { diff --git a/docs/src/test/scala/docs/stream/IntegrationDocSpec.scala b/docs/src/test/scala/docs/stream/IntegrationDocSpec.scala index 10e4fc7947..0553e63bf8 100644 --- a/docs/src/test/scala/docs/stream/IntegrationDocSpec.scala +++ b/docs/src/test/scala/docs/stream/IntegrationDocSpec.scala @@ -8,7 +8,7 @@ import scala.concurrent.duration._ import org.apache.pekko.Done import org.apache.pekko.NotUsed -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import org.apache.pekko.stream.scaladsl._ import org.apache.pekko.stream._ @@ -132,7 +132,7 @@ object IntegrationDocSpec { } -class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) { +class IntegrationDocSpec extends PekkoSpec(IntegrationDocSpec.config) { import TwitterStreamQuickstartDocSpec._ import IntegrationDocSpec._ diff --git a/docs/src/test/scala/docs/stream/KillSwitchDocSpec.scala b/docs/src/test/scala/docs/stream/KillSwitchDocSpec.scala index 3ba3980bf9..d5aab19cca 100644 --- a/docs/src/test/scala/docs/stream/KillSwitchDocSpec.scala +++ b/docs/src/test/scala/docs/stream/KillSwitchDocSpec.scala @@ -6,13 +6,13 @@ package docs.stream import org.apache.pekko.stream.scaladsl._ import org.apache.pekko.stream.{ DelayOverflowStrategy, KillSwitches } -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import docs.CompileOnlySpec import scala.concurrent.Await import scala.concurrent.duration._ -class KillSwitchDocSpec extends AkkaSpec with CompileOnlySpec { +class KillSwitchDocSpec extends PekkoSpec with CompileOnlySpec { "Unique kill switch" must { diff --git a/docs/src/test/scala/docs/stream/MigrationsScala.scala b/docs/src/test/scala/docs/stream/MigrationsScala.scala index 14e50ffbc2..98ebab051b 100644 --- a/docs/src/test/scala/docs/stream/MigrationsScala.scala +++ b/docs/src/test/scala/docs/stream/MigrationsScala.scala @@ -5,9 +5,9 @@ package docs.stream import org.apache.pekko.stream.scaladsl._ -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec -class MigrationsScala extends AkkaSpec { +class MigrationsScala extends PekkoSpec { "Examples in migration guide" must { "compile" in { diff --git a/docs/src/test/scala/docs/stream/RateTransformationDocSpec.scala b/docs/src/test/scala/docs/stream/RateTransformationDocSpec.scala index 3a51c393db..ade87e9ec6 100644 --- a/docs/src/test/scala/docs/stream/RateTransformationDocSpec.scala +++ b/docs/src/test/scala/docs/stream/RateTransformationDocSpec.scala @@ -11,11 +11,11 @@ import scala.util.Random import scala.math._ import scala.concurrent.duration._ import scala.collection.immutable -import org.apache.pekko.testkit.{ AkkaSpec, TestLatch } +import org.apache.pekko.testkit.{ PekkoSpec, TestLatch } import scala.concurrent.Await -class RateTransformationDocSpec extends AkkaSpec { +class RateTransformationDocSpec extends PekkoSpec { "conflate should summarize" in { // #conflate-summarize diff --git a/docs/src/test/scala/docs/stream/ReactiveStreamsDocSpec.scala b/docs/src/test/scala/docs/stream/ReactiveStreamsDocSpec.scala index 6f9d5a58da..52a3f83711 100644 --- a/docs/src/test/scala/docs/stream/ReactiveStreamsDocSpec.scala +++ b/docs/src/test/scala/docs/stream/ReactiveStreamsDocSpec.scala @@ -7,9 +7,9 @@ package docs.stream import org.apache.pekko.NotUsed import org.apache.pekko.stream.scaladsl.{ Flow, Sink, Source } import org.apache.pekko.stream.testkit._ -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec -class ReactiveStreamsDocSpec extends AkkaSpec { +class ReactiveStreamsDocSpec extends PekkoSpec { import TwitterStreamQuickstartDocSpec._ // #imports diff --git a/docs/src/test/scala/docs/stream/RestartDocSpec.scala b/docs/src/test/scala/docs/stream/RestartDocSpec.scala index 1715a005d5..e063c9a3a4 100644 --- a/docs/src/test/scala/docs/stream/RestartDocSpec.scala +++ b/docs/src/test/scala/docs/stream/RestartDocSpec.scala @@ -7,13 +7,13 @@ package docs.stream import org.apache.pekko.NotUsed import org.apache.pekko.stream.{ KillSwitches, RestartSettings } import org.apache.pekko.stream.scaladsl._ -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import docs.CompileOnlySpec import scala.concurrent.duration._ import scala.concurrent._ -class RestartDocSpec extends AkkaSpec with CompileOnlySpec { +class RestartDocSpec extends PekkoSpec with CompileOnlySpec { import system.dispatcher // Mock akka-http interfaces diff --git a/docs/src/test/scala/docs/stream/StreamBuffersRateSpec.scala b/docs/src/test/scala/docs/stream/StreamBuffersRateSpec.scala index a63caeee42..b3c238e8d0 100644 --- a/docs/src/test/scala/docs/stream/StreamBuffersRateSpec.scala +++ b/docs/src/test/scala/docs/stream/StreamBuffersRateSpec.scala @@ -7,9 +7,9 @@ package docs.stream import org.apache.pekko.NotUsed import org.apache.pekko.stream._ import org.apache.pekko.stream.scaladsl._ -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec -class StreamBuffersRateSpec extends AkkaSpec { +class StreamBuffersRateSpec extends PekkoSpec { "Demonstrate pipelining" in { def println(s: Any) = () diff --git a/docs/src/test/scala/docs/stream/StreamPartialGraphDSLDocSpec.scala b/docs/src/test/scala/docs/stream/StreamPartialGraphDSLDocSpec.scala index e3970a875c..2b2b13732e 100644 --- a/docs/src/test/scala/docs/stream/StreamPartialGraphDSLDocSpec.scala +++ b/docs/src/test/scala/docs/stream/StreamPartialGraphDSLDocSpec.scala @@ -7,13 +7,13 @@ package docs.stream import org.apache.pekko.actor.ActorRef import org.apache.pekko.stream._ import org.apache.pekko.stream.scaladsl._ -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import scala.concurrent.{ Await, Future } import scala.concurrent.duration._ import scala.concurrent.ExecutionContext -class StreamPartialGraphDSLDocSpec extends AkkaSpec { +class StreamPartialGraphDSLDocSpec extends PekkoSpec { implicit val ec: ExecutionContext = system.dispatcher diff --git a/docs/src/test/scala/docs/stream/StreamTestKitDocSpec.scala b/docs/src/test/scala/docs/stream/StreamTestKitDocSpec.scala index a8ef8cc969..7a6b89779e 100644 --- a/docs/src/test/scala/docs/stream/StreamTestKitDocSpec.scala +++ b/docs/src/test/scala/docs/stream/StreamTestKitDocSpec.scala @@ -12,10 +12,10 @@ import org.apache.pekko.Done import org.apache.pekko.stream._ import org.apache.pekko.stream.scaladsl._ import org.apache.pekko.stream.testkit.scaladsl._ -import org.apache.pekko.testkit.{ AkkaSpec, TestProbe } +import org.apache.pekko.testkit.{ PekkoSpec, TestProbe } import org.apache.pekko.pattern -class StreamTestKitDocSpec extends AkkaSpec { +class StreamTestKitDocSpec extends PekkoSpec { "strict collection" in { // #strict-collection diff --git a/docs/src/test/scala/docs/stream/SubstreamDocSpec.scala b/docs/src/test/scala/docs/stream/SubstreamDocSpec.scala index 18ded56feb..236c39681d 100644 --- a/docs/src/test/scala/docs/stream/SubstreamDocSpec.scala +++ b/docs/src/test/scala/docs/stream/SubstreamDocSpec.scala @@ -6,9 +6,9 @@ package docs.stream import org.apache.pekko.stream.scaladsl.{ Sink, Source } import org.apache.pekko.stream.SubstreamCancelStrategy -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec -class SubstreamDocSpec extends AkkaSpec { +class SubstreamDocSpec extends PekkoSpec { "generate substreams by groupBy" in { // #groupBy1 diff --git a/docs/src/test/scala/docs/stream/TwitterStreamQuickstartDocSpec.scala b/docs/src/test/scala/docs/stream/TwitterStreamQuickstartDocSpec.scala index 4f6f069dd6..ee0bd78580 100644 --- a/docs/src/test/scala/docs/stream/TwitterStreamQuickstartDocSpec.scala +++ b/docs/src/test/scala/docs/stream/TwitterStreamQuickstartDocSpec.scala @@ -16,7 +16,7 @@ import scala.io.StdIn.readLine //#imports -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import scala.concurrent.ExecutionContext object TwitterStreamQuickstartDocSpec { @@ -69,7 +69,7 @@ object TwitterStreamQuickstartDocSpec { // #fiddle_code } -class TwitterStreamQuickstartDocSpec extends AkkaSpec { +class TwitterStreamQuickstartDocSpec extends PekkoSpec { import TwitterStreamQuickstartDocSpec._ implicit val executionContext: ExecutionContext = system.dispatcher diff --git a/docs/src/test/scala/docs/stream/cookbook/RecipeSpec.scala b/docs/src/test/scala/docs/stream/cookbook/RecipeSpec.scala index 96ac627989..68e00cacad 100644 --- a/docs/src/test/scala/docs/stream/cookbook/RecipeSpec.scala +++ b/docs/src/test/scala/docs/stream/cookbook/RecipeSpec.scala @@ -4,9 +4,9 @@ package docs.stream.cookbook -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec -trait RecipeSpec extends AkkaSpec { +trait RecipeSpec extends PekkoSpec { type Message = String diff --git a/docs/src/test/scala/docs/stream/io/StreamFileDocSpec.scala b/docs/src/test/scala/docs/stream/io/StreamFileDocSpec.scala index a0c11aed49..b71d25ef9f 100644 --- a/docs/src/test/scala/docs/stream/io/StreamFileDocSpec.scala +++ b/docs/src/test/scala/docs/stream/io/StreamFileDocSpec.scala @@ -10,12 +10,12 @@ import org.apache.pekko.stream._ import org.apache.pekko.stream.scaladsl.{ FileIO, Sink, Source } import org.apache.pekko.stream.testkit.Utils._ import org.apache.pekko.util.ByteString -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import scala.concurrent.Future import scala.concurrent.ExecutionContext -class StreamFileDocSpec extends AkkaSpec(UnboundedMailboxConfig) { +class StreamFileDocSpec extends PekkoSpec(UnboundedMailboxConfig) { implicit val ec: ExecutionContext = system.dispatcher diff --git a/docs/src/test/scala/docs/stream/io/StreamTcpDocSpec.scala b/docs/src/test/scala/docs/stream/io/StreamTcpDocSpec.scala index bb0dfac00d..a8ead35e92 100644 --- a/docs/src/test/scala/docs/stream/io/StreamTcpDocSpec.scala +++ b/docs/src/test/scala/docs/stream/io/StreamTcpDocSpec.scala @@ -8,7 +8,7 @@ import java.util.concurrent.atomic.AtomicReference import org.apache.pekko.stream.scaladsl.Tcp._ import org.apache.pekko.stream.scaladsl._ -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import org.apache.pekko.testkit.TestProbe import org.apache.pekko.util.ByteString @@ -16,7 +16,7 @@ import scala.concurrent.Future import org.apache.pekko.testkit.SocketUtil import scala.concurrent.ExecutionContext -class StreamTcpDocSpec extends AkkaSpec { +class StreamTcpDocSpec extends PekkoSpec { implicit val ec: ExecutionContext = system.dispatcher diff --git a/docs/src/test/scala/docs/stream/operators/WithContextSpec.scala b/docs/src/test/scala/docs/stream/operators/WithContextSpec.scala index a0ea8d2d35..288dd76187 100644 --- a/docs/src/test/scala/docs/stream/operators/WithContextSpec.scala +++ b/docs/src/test/scala/docs/stream/operators/WithContextSpec.scala @@ -4,9 +4,9 @@ package docs.stream.operators -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec -class WithContextSpec extends AkkaSpec { +class WithContextSpec extends PekkoSpec { "use asSourceWithContext" in { // #asSourceWithContext diff --git a/docs/src/test/scala/docs/stream/operators/converters/StreamConvertersToJava.scala b/docs/src/test/scala/docs/stream/operators/converters/StreamConvertersToJava.scala index 8f6c291d06..a80f600ccc 100644 --- a/docs/src/test/scala/docs/stream/operators/converters/StreamConvertersToJava.scala +++ b/docs/src/test/scala/docs/stream/operators/converters/StreamConvertersToJava.scala @@ -15,13 +15,13 @@ import pekko.stream.scaladsl.Sink import pekko.stream.scaladsl.Source import pekko.stream.scaladsl.StreamConverters // #import -import pekko.testkit.AkkaSpec +import pekko.testkit.PekkoSpec import org.scalatest.concurrent.Futures import scala.collection.immutable import scala.concurrent.Future -class StreamConvertersToJava extends AkkaSpec with Futures { +class StreamConvertersToJava extends PekkoSpec with Futures { "demonstrate materialization to Java8 streams" in { // #asJavaStream diff --git a/docs/src/test/scala/docs/stream/operators/converters/ToFromJavaIOStreams.scala b/docs/src/test/scala/docs/stream/operators/converters/ToFromJavaIOStreams.scala index 7e5ca16335..9306f90f35 100644 --- a/docs/src/test/scala/docs/stream/operators/converters/ToFromJavaIOStreams.scala +++ b/docs/src/test/scala/docs/stream/operators/converters/ToFromJavaIOStreams.scala @@ -14,12 +14,12 @@ import org.apache.pekko.util.ByteString import scala.util.Random // #import -import org.apache.pekko.testkit.AkkaSpec +import org.apache.pekko.testkit.PekkoSpec import org.scalatest.concurrent.Futures import scala.concurrent.Future -class ToFromJavaIOStreams extends AkkaSpec with Futures { +class ToFromJavaIOStreams extends PekkoSpec with Futures { "demonstrate conversion from java.io.streams" in { diff --git a/docs/src/test/scala/docs/testkit/TestkitDocSpec.scala b/docs/src/test/scala/docs/testkit/TestkitDocSpec.scala index df34b35d67..75b834ca31 100644 --- a/docs/src/test/scala/docs/testkit/TestkitDocSpec.scala +++ b/docs/src/test/scala/docs/testkit/TestkitDocSpec.scala @@ -100,7 +100,7 @@ object TestKitDocSpec { } } -class TestKitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { +class TestKitDocSpec extends PekkoSpec with DefaultTimeout with ImplicitSender { import TestKitDocSpec._ "demonstrate usage of TestActorRef" in { diff --git a/project/CopyrightHeader.scala b/project/CopyrightHeader.scala index edbe45f2b6..981eca848e 100644 --- a/project/CopyrightHeader.scala +++ b/project/CopyrightHeader.scala @@ -4,12 +4,12 @@ package org.apache.pekko -import org.apache.pekko.AkkaValidatePullRequest.additionalTasks +import org.apache.pekko.PekkoValidatePullRequest.additionalTasks import de.heikoseeberger.sbtheader.HeaderPlugin.autoImport._ import de.heikoseeberger.sbtheader.{ CommentCreator, HeaderPlugin } import com.typesafe.sbt.MultiJvmPlugin.MultiJvmKeys._ import sbt.Keys._ -import sbt.{ Def, _ } +import sbt._ trait CopyrightHeader extends AutoPlugin { diff --git a/project/Jdk9.scala b/project/Jdk9.scala index 9d8cdcbc2c..c166d29770 100644 --- a/project/Jdk9.scala +++ b/project/Jdk9.scala @@ -25,8 +25,8 @@ object Jdk9 extends AutoPlugin { Seq( (Compile / sourceDirectory).value / SCALA_SOURCE_DIRECTORY, (Compile / sourceDirectory).value / JAVA_SOURCE_DIRECTORY)), - scalacOptions := AkkaBuild.DefaultScalacOptions.value ++ notOnJdk8(Seq("-release", "11")), - javacOptions := AkkaBuild.DefaultJavacOptions ++ notOnJdk8(Seq("--release", "11"))) + scalacOptions := PekkoBuild.DefaultScalacOptions.value ++ notOnJdk8(Seq("-release", "11")), + javacOptions := PekkoBuild.DefaultJavacOptions ++ notOnJdk8(Seq("--release", "11"))) val testJdk9Settings = Seq( // following the scala-2.12, scala-sbt-1.0, ... convention @@ -34,8 +34,8 @@ object Jdk9 extends AutoPlugin { Seq( (Test / sourceDirectory).value / SCALA_TEST_SOURCE_DIRECTORY, (Test / sourceDirectory).value / JAVA_TEST_SOURCE_DIRECTORY)), - scalacOptions := AkkaBuild.DefaultScalacOptions.value ++ notOnJdk8(Seq("-release", "11")), - javacOptions := AkkaBuild.DefaultJavacOptions ++ notOnJdk8(Seq("--release", "11")), + scalacOptions := PekkoBuild.DefaultScalacOptions.value ++ notOnJdk8(Seq("-release", "11")), + javacOptions := PekkoBuild.DefaultJavacOptions ++ notOnJdk8(Seq("--release", "11")), compile := compile.dependsOn(CompileJdk9 / compile).value, classpathConfiguration := TestJdk9, externalDependencyClasspath := (Test / externalDependencyClasspath).value) diff --git a/project/AkkaBuild.scala b/project/PekkoBuild.scala similarity index 99% rename from project/AkkaBuild.scala rename to project/PekkoBuild.scala index 460d36f25c..bd734ab71d 100644 --- a/project/AkkaBuild.scala +++ b/project/PekkoBuild.scala @@ -17,7 +17,7 @@ import java.io.FileInputStream import java.io.InputStreamReader import java.util.Properties -object AkkaBuild { +object PekkoBuild { object CliOptions { // CI is the env var defined by Github Actions and Travis: diff --git a/project/AkkaDisciplinePlugin.scala b/project/PekkoDisciplinePlugin.scala similarity index 99% rename from project/AkkaDisciplinePlugin.scala rename to project/PekkoDisciplinePlugin.scala index 336ed93f9d..438eb110d4 100644 --- a/project/AkkaDisciplinePlugin.scala +++ b/project/PekkoDisciplinePlugin.scala @@ -8,7 +8,7 @@ import sbt._ import Keys.{ scalacOptions, _ } import sbt.plugins.JvmPlugin -object AkkaDisciplinePlugin extends AutoPlugin { +object PekkoDisciplinePlugin extends AutoPlugin { override def trigger: PluginTrigger = allRequirements override def requires: Plugins = JvmPlugin diff --git a/project/ValidatePullRequest.scala b/project/ValidatePullRequest.scala index be312bdc46..e3a1320803 100644 --- a/project/ValidatePullRequest.scala +++ b/project/ValidatePullRequest.scala @@ -14,7 +14,7 @@ import sbtunidoc.BaseUnidocPlugin.autoImport.unidoc import sbt.Keys._ import sbt._ -object AkkaValidatePullRequest extends AutoPlugin { +object PekkoValidatePullRequest extends AutoPlugin { object CliOptions { val mimaEnabled = CliOption("akka.mima.enabled", true) @@ -67,11 +67,11 @@ object AkkaValidatePullRequest extends AutoPlugin { * autoplugin would trigger only on projects which have both of these plugins enabled. */ object MultiNodeWithPrValidation extends AutoPlugin { - import AkkaValidatePullRequest._ + import PekkoValidatePullRequest._ import com.typesafe.sbt.MultiJvmPlugin.MultiJvmKeys.MultiJvm override def trigger = allRequirements - override def requires = AkkaValidatePullRequest && MultiNode + override def requires = PekkoValidatePullRequest && MultiNode override lazy val projectSettings = if (MultiNode.multiNodeTestInTest) Seq(additionalTasks += MultiNode.multiTest) else Seq.empty @@ -82,10 +82,10 @@ object MultiNodeWithPrValidation extends AutoPlugin { * when a project has MimaPlugin autoplugin enabled. */ object MimaWithPrValidation extends AutoPlugin { - import AkkaValidatePullRequest._ + import PekkoValidatePullRequest._ override def trigger = allRequirements - override def requires = AkkaValidatePullRequest && MimaPlugin + override def requires = PekkoValidatePullRequest && MimaPlugin override lazy val projectSettings = CliOptions.mimaEnabled.ifTrue(additionalTasks += mimaReportBinaryIssues).toList } @@ -95,15 +95,15 @@ object MimaWithPrValidation extends AutoPlugin { * when a project has ParadoxPlugin autoplugin enabled. */ object ParadoxWithPrValidation extends AutoPlugin { - import AkkaValidatePullRequest._ + import PekkoValidatePullRequest._ override def trigger = allRequirements - override def requires = AkkaValidatePullRequest && ParadoxPlugin + override def requires = PekkoValidatePullRequest && ParadoxPlugin override lazy val projectSettings = Seq(additionalTasks += Compile / paradox) } object UnidocWithPrValidation extends AutoPlugin { - import AkkaValidatePullRequest._ + import PekkoValidatePullRequest._ override def trigger = noTrigger override lazy val projectSettings = Seq(additionalTasks += Compile / unidoc)