actor: allow seamless access to untyped extensions given typed ActorSystem (#28294)

* actor: allow seamless access to untyped extensions given typed ActorSystem

* add overrides with concrete type for Java API everywhere
This commit is contained in:
Johannes Rudolph 2019-12-05 16:01:23 +01:00 committed by Arnout Engelen
parent 2c96a57d89
commit 702b6a7f41
70 changed files with 192 additions and 49 deletions

View file

@ -5,7 +5,6 @@
package akka.actor.typed.internal
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.ActorRef
import akka.serialization.{ JavaSerializer, SerializationExtension }
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
@ -32,7 +31,7 @@ class ActorRefSerializationSpec
with WordSpecLike
with LogCapturing {
val serialization = SerializationExtension(system.toClassic)
val serialization = SerializationExtension(system)
"ActorRef[T]" must {
"be serialized and deserialized by MiscMessageSerializer" in {

View file

@ -14,7 +14,6 @@ import akka.actor.{ Address, CoordinatedShutdown, InvalidMessageException }
import akka.actor.testkit.typed.scaladsl.TestInbox
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
import org.scalatest._
import org.scalatest.concurrent.Eventually
import org.scalatest.concurrent.ScalaFutures
@ -62,8 +61,7 @@ class ActorSystemSpec
}
inbox.receiveAll() should ===("hello" :: Nil)
sys.whenTerminated.futureValue
CoordinatedShutdown(sys.toClassic).shutdownReason() should ===(
Some(CoordinatedShutdown.ActorSystemTerminateReason))
CoordinatedShutdown(sys).shutdownReason() should ===(Some(CoordinatedShutdown.ActorSystemTerminateReason))
}
}
@ -98,8 +96,7 @@ class ActorSystemSpec
// now we know that the guardian has started, and should receive PostStop
sys.terminate()
sys.whenTerminated.futureValue
CoordinatedShutdown(sys.toClassic).shutdownReason() should ===(
Some(CoordinatedShutdown.ActorSystemTerminateReason))
CoordinatedShutdown(sys).shutdownReason() should ===(Some(CoordinatedShutdown.ActorSystemTerminateReason))
inbox.receiveAll() should ===("done" :: Nil)
}

View file

@ -6,7 +6,6 @@ package akka.actor.typed.internal.receptionist
import akka.actor.typed.internal.ActorRefSerializationSpec
import akka.actor.typed.receptionist.ServiceKey
import akka.actor.typed.scaladsl.adapter._
import akka.serialization.SerializationExtension
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
@ -17,7 +16,7 @@ class ServiceKeySerializationSpec
with WordSpecLike
with LogCapturing {
val serialization = SerializationExtension(system.toClassic)
val serialization = SerializationExtension(system)
"ServiceKey[T]" must {
"be serialized and deserialized by ServiceKeySerializer" in {

View file

@ -5,7 +5,6 @@
package akka.actor.typed.scaladsl.adapter
import scala.util.control.NoStackTrace
import akka.actor.InvalidMessageException
import akka.actor.testkit.typed.TestException
import akka.actor.typed.scaladsl.Behaviors
@ -20,6 +19,7 @@ import akka.actor.ActorInitializationException
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.LoggingTestKit
import akka.actor.typed.internal.adapter.SchedulerAdapter
import akka.serialization.SerializationExtension
import akka.{ actor => classic }
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
@ -183,6 +183,7 @@ class AdapterSpec extends WordSpec with Matchers with BeforeAndAfterAll with Log
import AdapterSpec._
implicit val system = akka.actor.ActorSystem("AdapterSpec")
def typedSystem: ActorSystem[Nothing] = system.toTyped
"ActorSystem adaption" must {
"only happen once for a given actor system" in {
@ -224,6 +225,10 @@ class AdapterSpec extends WordSpec with Matchers with BeforeAndAfterAll with Log
typedScheduler.getClass should ===(classOf[SchedulerAdapter])
(typedScheduler.toClassic should be).theSameInstanceAs(system.scheduler)
}
"allow seamless access to untyped extensions" in {
SerializationExtension(typedSystem) should not be (null)
}
}
"Adapted actors" must {

View file

@ -0,0 +1,12 @@
# Incompatibilities against Akka < 2.5.17 where extra static bridge methods were generated that
# Mima now wrongly correlates with the new ones added in the PR
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.util.ManifestInfo.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.actor.CoordinatedShutdown.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.actor.TypedActor.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.event.AddressTerminatedTopic.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.io.Dns.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.io.Tcp.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.io.Udp.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.io.UdpConnected.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.serialization.SerializationExtension.get")

View file

@ -181,6 +181,8 @@ object CoordinatedShutdown extends ExtensionId[CoordinatedShutdown] with Extensi
override def get(system: ActorSystem): CoordinatedShutdown = super.get(system)
override def get(system: ClassicActorSystemProvider): CoordinatedShutdown = super.get(system)
override def lookup = CoordinatedShutdown
override def createExtension(system: ExtendedActorSystem): CoordinatedShutdown = {

View file

@ -31,6 +31,7 @@ package akka.actor
*
* // Java API: retrieve the extension for the given system.
* override def get(system: ActorSystem): UdpExt = super.get(system)
* override def get(system: ClassicActorSystemProvider): UdpExt = super.get(system)
* }
*
* class Ext(system: ExtendedActorSystem) extends Extension {
@ -77,6 +78,11 @@ trait ExtensionId[T <: Extension] {
java.util.Objects.requireNonNull(system, "system must not be null!").registerExtension(this)
}
/**
* Returns an instance of the extension identified by this ExtensionId instance.
*/
def apply(system: ClassicActorSystemProvider): T = apply(system.classicSystem)
/**
* Returns an instance of the extension identified by this ExtensionId instance.
* Java API
@ -89,6 +95,18 @@ trait ExtensionId[T <: Extension] {
*/
def get(system: ActorSystem): T = apply(system)
/**
* Returns an instance of the extension identified by this ExtensionId instance.
* Java API
* For extensions written in Scala that are to be used from Java also,
* this method should be overridden to get correct return type.
* {{{
* override def get(system: ClassicActorSystemProvider): TheExtension = super.get(system)
* }}}
*
*/
def get(system: ClassicActorSystemProvider): T = apply(system)
/**
* Is used by Akka to instantiate the Extension identified by this ExtensionId,
* internal use only.

View file

@ -106,6 +106,7 @@ trait TypedActorFactory {
@deprecated("Use 'akka.actor.typed' API.", since = "2.6.0")
object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvider {
override def get(system: ActorSystem): TypedActorExtension = super.get(system)
override def get(system: ClassicActorSystemProvider): TypedActorExtension = super.get(system)
def lookup() = this
def createExtension(system: ExtendedActorSystem): TypedActorExtension = new TypedActorExtension(system)

View file

@ -5,10 +5,12 @@
package akka.event
import java.util.concurrent.atomic.AtomicReference
import scala.annotation.tailrec
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.AddressTerminated
import akka.actor.ClassicActorSystemProvider
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
import akka.actor.ExtensionId
@ -24,6 +26,7 @@ import akka.actor.ExtensionIdProvider
*/
private[akka] object AddressTerminatedTopic extends ExtensionId[AddressTerminatedTopic] with ExtensionIdProvider {
override def get(system: ActorSystem): AddressTerminatedTopic = super.get(system)
override def get(system: ClassicActorSystemProvider): AddressTerminatedTopic = super.get(system)
override def lookup = AddressTerminatedTopic

View file

@ -145,6 +145,7 @@ object Dns extends ExtensionId[DnsExt] with ExtensionIdProvider {
* Java API: retrieve the Udp extension for the given system.
*/
override def get(system: ActorSystem): DnsExt = super.get(system)
override def get(system: ClassicActorSystemProvider): DnsExt = super.get(system)
}
class DnsExt private[akka] (val system: ExtendedActorSystem, resolverName: String, managerName: String)

View file

@ -48,6 +48,8 @@ object Tcp extends ExtensionId[TcpExt] with ExtensionIdProvider {
*/
override def get(system: ActorSystem): TcpExt = super.get(system)
override def get(system: ClassicActorSystemProvider): TcpExt = super.get(system)
/**
* Scala API: this object contains all applicable socket options for TCP.
*

View file

@ -40,6 +40,7 @@ object Udp extends ExtensionId[UdpExt] with ExtensionIdProvider {
* Java API: retrieve the Udp extension for the given system.
*/
override def get(system: ActorSystem): UdpExt = super.get(system)
override def get(system: ClassicActorSystemProvider): UdpExt = super.get(system)
/**
* The common interface for [[Command]] and [[Event]].

View file

@ -38,6 +38,7 @@ object UdpConnected extends ExtensionId[UdpConnectedExt] with ExtensionIdProvide
* Java API: retrieve the UdpConnected extension for the given system.
*/
override def get(system: ActorSystem): UdpConnectedExt = super.get(system)
override def get(system: ClassicActorSystemProvider): UdpConnectedExt = super.get(system)
/**
* The common interface for [[Command]] and [[Event]].

View file

@ -4,6 +4,7 @@
package akka.serialization
import akka.actor.ClassicActorSystemProvider
import akka.actor.{ ActorSystem, ExtendedActorSystem, ExtensionId, ExtensionIdProvider }
/**
@ -12,6 +13,7 @@ import akka.actor.{ ActorSystem, ExtendedActorSystem, ExtensionId, ExtensionIdPr
*/
object SerializationExtension extends ExtensionId[Serialization] with ExtensionIdProvider {
override def get(system: ActorSystem): Serialization = super.get(system)
override def get(system: ClassicActorSystemProvider): Serialization = super.get(system)
override def lookup = SerializationExtension
override def createExtension(system: ExtendedActorSystem): Serialization = new Serialization(system)
}

View file

@ -11,6 +11,7 @@ import java.util.jar.Attributes
import java.util.jar.Manifest
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
import akka.actor.ExtensionId
@ -40,6 +41,7 @@ object ManifestInfo extends ExtensionId[ManifestInfo] with ExtensionIdProvider {
"com.typesafe.play")
override def get(system: ActorSystem): ManifestInfo = super.get(system)
override def get(system: ClassicActorSystemProvider): ManifestInfo = super.get(system)
override def lookup(): ManifestInfo.type = ManifestInfo

View file

@ -0,0 +1,4 @@
# Incompatibilities against Akka < 2.5.17 where extra static bridge methods were generated that
# Mima now wrongly correlates with the new ones added in the PR
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.cluster.metrics.ClusterMetricsExtension.get")

View file

@ -10,6 +10,7 @@ import akka.actor.SupervisorStrategy
import akka.event.LoggingAdapter
import akka.event.Logging
import com.typesafe.config.Config
import scala.collection.immutable
import akka.actor.Props
import akka.actor.Deploy
@ -17,6 +18,7 @@ import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
import akka.actor.ActorSystem
import akka.actor.ActorRef
import akka.actor.ClassicActorSystemProvider
/**
* Cluster metrics extension.
@ -87,6 +89,7 @@ class ClusterMetricsExtension(system: ExtendedActorSystem) extends Extension {
object ClusterMetricsExtension extends ExtensionId[ClusterMetricsExtension] with ExtensionIdProvider {
override def lookup = ClusterMetricsExtension
override def get(system: ActorSystem): ClusterMetricsExtension = super.get(system)
override def get(system: ClassicActorSystemProvider): ClusterMetricsExtension = super.get(system)
override def createExtension(system: ExtendedActorSystem): ClusterMetricsExtension =
new ClusterMetricsExtension(system)
}

View file

@ -8,7 +8,6 @@ import akka.actor.testkit.typed.scaladsl.ActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.typed.ActorSystem
import akka.actor.typed.scaladsl.adapter._
import akka.cluster.{ Cluster => ClassicCluster }
import akka.testkit.LongRunningTest
import com.typesafe.config.{ Config, ConfigFactory }
@ -50,14 +49,14 @@ abstract class JoinConfigCompatCheckerClusterShardingSpec
protected def join(sys: ActorSystem[_]): ClassicCluster = {
if (sys eq system) {
configured(system) should ===(JoinConfig.Shards)
val seedNode = ClassicCluster(system.toClassic)
val seedNode = ClassicCluster(system)
seedNode.join(seedNode.selfAddress)
val probe = createTestProbe()
probe.awaitAssert(seedNode.readView.isSingletonCluster should ===(true), clusterWaitDuration)
seedNode
} else {
val joiningNode = ClassicCluster(sys.toClassic)
joiningNode.joinSeedNodes(im.Seq(ClassicCluster(system.toClassic).selfAddress))
val joiningNode = ClassicCluster(sys)
joiningNode.joinSeedNodes(im.Seq(ClassicCluster(system).selfAddress))
joiningNode
}
}

View file

@ -0,0 +1,4 @@
# Incompatibilities against Akka < 2.5.17 where extra static bridge methods were generated that
# Mima now wrongly correlates with the new ones added in the PR
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.cluster.sharding.ClusterSharding.get")

View file

@ -11,11 +11,11 @@ import java.util.concurrent.ConcurrentHashMap
import scala.collection.immutable
import scala.concurrent.Await
import scala.util.control.NonFatal
import akka.util.ccompat.JavaConverters._
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
import akka.actor.Deploy
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
@ -153,6 +153,7 @@ import akka.util.ByteString
object ClusterSharding extends ExtensionId[ClusterSharding] with ExtensionIdProvider {
override def get(system: ActorSystem): ClusterSharding = super.get(system)
override def get(system: ClassicActorSystemProvider): ClusterSharding = super.get(system)
override def lookup = ClusterSharding

View file

@ -0,0 +1,5 @@
# Incompatibilities against Akka < 2.5.17 where extra static bridge methods were generated that
# Mima now wrongly correlates with the new ones added in the PR
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.cluster.client.ClusterClientReceptionist.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.cluster.pubsub.DistributedPubSub.get")

View file

@ -8,7 +8,6 @@ import java.net.URLEncoder
import scala.collection.immutable
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.ActorLogging
@ -17,6 +16,7 @@ import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.Address
import akka.actor.Cancellable
import akka.actor.ClassicActorSystemProvider
import akka.actor.DeadLetterSuppression
import akka.actor.Deploy
import akka.actor.ExtendedActorSystem
@ -40,6 +40,7 @@ import com.typesafe.config.Config
import akka.remote.DeadlineFailureDetector
import akka.util.MessageBuffer
import akka.util.ccompat._
import scala.collection.immutable.{ HashMap, HashSet }
@ccompatUsedUntil213
@ -557,6 +558,7 @@ final class ClusterClient(settings: ClusterClientSettings) extends Actor with Ac
since = "2.6.0")
object ClusterClientReceptionist extends ExtensionId[ClusterClientReceptionist] with ExtensionIdProvider {
override def get(system: ActorSystem): ClusterClientReceptionist = super.get(system)
override def get(system: ClassicActorSystemProvider): ClusterClientReceptionist = super.get(system)
override def lookup() = ClusterClientReceptionist

View file

@ -920,6 +920,8 @@ class DistributedPubSubMediator(settings: DistributedPubSubSettings)
object DistributedPubSub extends ExtensionId[DistributedPubSub] with ExtensionIdProvider {
override def get(system: ActorSystem): DistributedPubSub = super.get(system)
override def get(system: ClassicActorSystemProvider): DistributedPubSub = super.get(system)
override def lookup = DistributedPubSub
override def createExtension(system: ExtendedActorSystem): DistributedPubSub =

View file

@ -8,11 +8,11 @@ import java.util.concurrent.atomic.AtomicReference
import scala.concurrent.Future
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
import akka.actor.ExtensionId
@ -73,6 +73,7 @@ class TestLeaseActor extends Actor with ActorLogging {
object TestLeaseActorClientExt extends ExtensionId[TestLeaseActorClientExt] with ExtensionIdProvider {
override def get(system: ActorSystem): TestLeaseActorClientExt = super.get(system)
override def get(system: ClassicActorSystemProvider): TestLeaseActorClientExt = super.get(system)
override def lookup = TestLeaseActorClientExt
override def createExtension(system: ExtendedActorSystem): TestLeaseActorClientExt =
new TestLeaseActorClientExt(system)

View file

@ -7,6 +7,7 @@ package akka.cluster
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.atomic.AtomicReference
import akka.actor.ClassicActorSystemProvider
import akka.actor.{ ActorSystem, ExtendedActorSystem, Extension, ExtensionId, ExtensionIdProvider }
import akka.coordination.lease.LeaseSettings
import akka.coordination.lease.scaladsl.Lease
@ -19,6 +20,7 @@ import akka.util.ccompat.JavaConverters._
object TestLeaseExt extends ExtensionId[TestLeaseExt] with ExtensionIdProvider {
override def get(system: ActorSystem): TestLeaseExt = super.get(system)
override def get(system: ClassicActorSystemProvider): TestLeaseExt = super.get(system)
override def lookup = TestLeaseExt
override def createExtension(system: ExtendedActorSystem): TestLeaseExt = new TestLeaseExt(system)
}

View file

@ -203,7 +203,6 @@ abstract class ClusterSingleton extends Extension {
}
object ClusterSingletonManagerSettings {
import akka.actor.typed.scaladsl.adapter._
/**
* Create settings from the default configuration
@ -211,7 +210,7 @@ object ClusterSingletonManagerSettings {
*/
def apply(system: ActorSystem[_]): ClusterSingletonManagerSettings =
apply(system.settings.config.getConfig("akka.cluster.singleton"))
.withRemovalMargin(akka.cluster.Cluster(system.toClassic).downingProvider.downRemovalMargin)
.withRemovalMargin(akka.cluster.Cluster(system).downingProvider.downRemovalMargin)
/**
* Create settings from a configuration with the same layout as

View file

@ -142,7 +142,7 @@ private[akka] final class AdapterClusterImpl(system: ActorSystem[_]) extends Clu
import AdapterClusterImpl._
require(system.isInstanceOf[ActorSystemAdapter[_]], "only adapted actor systems can be used for cluster features")
private val classicCluster = akka.cluster.Cluster(system.toClassic)
private val classicCluster = akka.cluster.Cluster(system)
override def selfMember: Member = classicCluster.selfMember
override def isTerminated: Boolean = classicCluster.isTerminated

View file

@ -72,7 +72,7 @@ class ActorSystemSpec
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.remote.artery.canonical.hostname = 127.0.0.1
serializers {
test = "akka.cluster.typed.ActorSystemSpec$$TestSerializer"
}
@ -114,8 +114,7 @@ class ActorSystemSpec
}
inbox.receiveAll() should ===("hello" :: Nil)
sys.whenTerminated.futureValue
CoordinatedShutdown(sys.toClassic).shutdownReason() should ===(
Some(CoordinatedShutdown.ActorSystemTerminateReason))
CoordinatedShutdown(sys).shutdownReason() should ===(Some(CoordinatedShutdown.ActorSystemTerminateReason))
}
}
@ -150,8 +149,7 @@ class ActorSystemSpec
// now we know that the guardian has started, and should receive PostStop
sys.terminate()
sys.whenTerminated.futureValue
CoordinatedShutdown(sys.toClassic).shutdownReason() should ===(
Some(CoordinatedShutdown.ActorSystemTerminateReason))
CoordinatedShutdown(sys).shutdownReason() should ===(Some(CoordinatedShutdown.ActorSystemTerminateReason))
inbox.receiveAll() should ===("done" :: Nil)
}

View file

@ -17,7 +17,6 @@ import akka.actor.typed.ActorRef
import akka.actor.typed.receptionist.Receptionist
import akka.actor.typed.receptionist.ServiceKey
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
import akka.cluster.MemberStatus
import akka.cluster.typed.Cluster
import akka.cluster.typed.Down
@ -210,7 +209,7 @@ class ClusterReceptionistSpec extends WordSpec with Matchers with LogCapturing {
regProbe2.expectMessageType[Listing].serviceInstances(PingKey).size should ===(2)
akka.cluster.Cluster(system1.toClassic).shutdown()
akka.cluster.Cluster(system1).shutdown()
regProbe2.expectNoMessage(3.seconds)

View file

@ -87,8 +87,7 @@ object Publisher {
Behaviors.setup[AnyRef] { context =>
import akka.cluster.pubsub.DistributedPubSub
import akka.cluster.pubsub.DistributedPubSubMediator
import akka.actor.typed.scaladsl.adapter._
val mediator = DistributedPubSub(context.system.toClassic).mediator
val mediator = DistributedPubSub(context.system).mediator
var registry: Map[DataKey, DataType] = Map.empty
@ -263,8 +262,7 @@ object DataPlatform {
def apply(): Behavior[ProvisionCommand] = {
Behaviors.setup { context =>
//#mediator
import akka.actor.typed.scaladsl.adapter._
val mediator = DistributedPubSub(context.system.toClassic).mediator
val mediator = DistributedPubSub(context.system).mediator
//#mediator
val service = context.spawn(DataService(mediator), "data")
@ -337,7 +335,7 @@ object DistributedPubSubExample {
// provision new data type
val platformProbe = TestProbe[DataApi]()(system)
val mediator = DistributedPubSub(system.toClassic).mediator
val mediator = DistributedPubSub(system).mediator
mediator ! DistributedPubSubMediator.Subscribe(IngestionTopic, platformProbe.ref.toClassic)
system ! ProvisionDataType(key, "dummy-schema", platformProbe.ref)

View file

@ -0,0 +1,4 @@
# Incompatibilities against Akka < 2.5.17 where extra static bridge methods were generated that
# Mima now wrongly correlates with the new ones added in the PR
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.cluster.Cluster.get")

View file

@ -35,6 +35,8 @@ import com.github.ghik.silencer.silent
object Cluster extends ExtensionId[Cluster] with ExtensionIdProvider {
override def get(system: ActorSystem): Cluster = super.get(system)
override def get(system: ClassicActorSystemProvider): Cluster = super.get(system)
override def lookup = Cluster
override def createExtension(system: ExtendedActorSystem): Cluster = new Cluster(system)

View file

@ -4,12 +4,14 @@
package akka.coordination.lease.javadsl
import akka.actor.ClassicActorSystemProvider
import akka.actor.{ ActorSystem, ExtendedActorSystem, Extension, ExtensionId, ExtensionIdProvider }
import akka.coordination.lease.internal.LeaseAdapter
import akka.coordination.lease.scaladsl.{ LeaseProvider => ScalaLeaseProvider }
object LeaseProvider extends ExtensionId[LeaseProvider] with ExtensionIdProvider {
override def get(system: ActorSystem): LeaseProvider = super.get(system)
override def get(system: ClassicActorSystemProvider): LeaseProvider = super.get(system)
override def lookup = LeaseProvider

View file

@ -10,6 +10,7 @@ import java.util.function.{ Function => JFunction }
import scala.collection.immutable
import scala.util.{ Failure, Success, Try }
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
import akka.actor.ExtensionId
@ -19,6 +20,7 @@ import akka.coordination.lease.LeaseSettings
object LeaseProvider extends ExtensionId[LeaseProvider] with ExtensionIdProvider {
override def get(system: ActorSystem): LeaseProvider = super.get(system)
override def get(system: ClassicActorSystemProvider): LeaseProvider = super.get(system)
override def lookup = LeaseProvider

View file

@ -103,6 +103,8 @@ object Discovery extends ExtensionId[Discovery] with ExtensionIdProvider {
override def get(system: ActorSystem): Discovery = super.get(system)
override def get(system: ClassicActorSystemProvider): Discovery = super.get(system)
override def createExtension(system: ExtendedActorSystem): Discovery = new Discovery()(system)
/**

View file

@ -0,0 +1,4 @@
# Incompatibilities against Akka < 2.5.17 where extra static bridge methods were generated that
# Mima now wrongly correlates with the new ones added in the PR
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.cluster.ddata.DistributedData.get")

View file

@ -6,6 +6,7 @@ package akka.cluster.ddata
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
import akka.actor.ExtensionId
@ -15,6 +16,7 @@ import akka.event.Logging
object DistributedData extends ExtensionId[DistributedData] with ExtensionIdProvider {
override def get(system: ActorSystem): DistributedData = super.get(system)
override def get(system: ClassicActorSystemProvider): DistributedData = super.get(system)
override def lookup = DistributedData

View file

@ -10,7 +10,6 @@ import akka.actor.typed.ActorSystem;
import akka.actor.typed.Behavior;
import akka.actor.typed.javadsl.AbstractBehavior;
import akka.actor.typed.javadsl.ActorContext;
import akka.actor.typed.javadsl.Adapter;
import akka.actor.typed.javadsl.Behaviors;
import akka.actor.typed.javadsl.Receive;
import akka.actor.typed.javadsl.AskPattern;
@ -31,7 +30,7 @@ public interface ResumableProjectionExample {
throws Exception {
final MyJavadslReadJournal readJournal =
PersistenceQuery.get(Adapter.toClassic(system))
PersistenceQuery.get(system)
.getReadJournalFor(
MyJavadslReadJournal.class, "akka.persistence.query.my-read-journal");

View file

@ -5,7 +5,9 @@
package docs.extension
import java.util.concurrent.atomic.AtomicLong
import akka.actor.Actor
import akka.actor.ClassicActorSystemProvider
import akka.testkit.AkkaSpec
//#extension
@ -42,6 +44,7 @@ object CountExtension extends ExtensionId[CountExtensionImpl] with ExtensionIdPr
* Java API: retrieve the Count extension for the given system.
*/
override def get(system: ActorSystem): CountExtensionImpl = super.get(system)
override def get(system: ClassicActorSystemProvider): CountExtensionImpl = super.get(system)
}
//#extensionid

View file

@ -10,10 +10,13 @@ import akka.actor.Extension
import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
import akka.actor.ExtendedActorSystem
import scala.concurrent.duration.Duration
import com.typesafe.config.Config
import java.util.concurrent.TimeUnit
import akka.actor.ClassicActorSystemProvider
//#imports
import akka.actor.Actor
@ -39,6 +42,7 @@ object Settings extends ExtensionId[SettingsImpl] with ExtensionIdProvider {
* Java API: retrieve the Settings extension for the given system.
*/
override def get(system: ActorSystem): SettingsImpl = super.get(system)
override def get(system: ClassicActorSystemProvider): SettingsImpl = super.get(system)
}
//#extensionid

View file

@ -0,0 +1,4 @@
# Incompatibilities against Akka < 2.5.17 where extra static bridge methods were generated that
# Mima now wrongly correlates with the new ones added in the PR
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.remote.testconductor.TestConductor.get")

View file

@ -4,6 +4,7 @@
package akka.remote.testconductor
import akka.actor.ClassicActorSystemProvider
import akka.actor.{ ActorContext, ActorSystem, ExtendedActorSystem, Extension, ExtensionId, ExtensionIdProvider }
import akka.remote.RemoteActorRefProvider
import akka.util.Timeout
@ -30,6 +31,7 @@ object TestConductor extends ExtensionId[TestConductorExt] with ExtensionIdProvi
* Java API: retrieve the TestConductor extension for the given system.
*/
override def get(system: ActorSystem): TestConductorExt = super.get(system)
override def get(system: ClassicActorSystemProvider): TestConductorExt = super.get(system)
def apply()(implicit ctx: ActorContext): TestConductorExt = apply(ctx.system)

View file

@ -0,0 +1,4 @@
# Incompatibilities against Akka < 2.5.17 where extra static bridge methods were generated that
# Mima now wrongly correlates with the new ones added in the PR
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.persistence.query.PersistenceQuery.get")

View file

@ -18,10 +18,8 @@ import scala.reflect.ClassTag
*/
object PersistenceQuery extends ExtensionId[PersistenceQuery] with ExtensionIdProvider {
/**
* Java API.
*/
override def get(system: ActorSystem): PersistenceQuery = super.get(system)
override def get(system: ClassicActorSystemProvider): PersistenceQuery = super.get(system)
def createExtension(system: ExtendedActorSystem): PersistenceQuery = new PersistenceQuery(system)

View file

@ -283,10 +283,9 @@ class EventSourcedBehaviorSpec
with LogCapturing {
import EventSourcedBehaviorSpec._
import akka.actor.typed.scaladsl.adapter._
val queries: LeveldbReadJournal =
PersistenceQuery(system.toClassic).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
val pidCounter = new AtomicInteger(0)
private def nextPid(): PersistenceId = PersistenceId.ofUniqueId(s"c${pidCounter.incrementAndGet()})")

View file

@ -96,13 +96,11 @@ class EventSourcedEventAdapterSpec
State
}
import akka.actor.typed.scaladsl.adapter._
val pidCounter = new AtomicInteger(0)
private def nextPid(): PersistenceId = PersistenceId.ofUniqueId(s"c${pidCounter.incrementAndGet()})")
val queries: LeveldbReadJournal =
PersistenceQuery(system.toClassic).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
private def behavior(pid: PersistenceId, probe: ActorRef[String]): EventSourcedBehavior[String, String, String] =
EventSourcedBehavior(pid, "", commandHandler = { (_, command) =>

View file

@ -12,7 +12,6 @@ import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
import akka.persistence.serialization.Snapshot
import akka.persistence.typed.PersistenceId
import akka.serialization.Serialization
@ -64,7 +63,7 @@ class SnapshotRecoveryWithEmptyJournalSpec
val snapshotsDir: File = new File(survivingSnapshotPath)
val serializationExtension: Serialization = SerializationExtension(system.toClassic)
val serializationExtension: Serialization = SerializationExtension(system)
val persistenceId: String = system.name

View file

@ -0,0 +1,6 @@
# Incompatibilities against Akka < 2.5.17 where extra static bridge methods were generated that
# Mima now wrongly correlates with the new ones added in the PR
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.persistence.fsm.SnapshotAfter.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.persistence.Persistence.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.persistence.journal.PersistencePluginProxyExtension.get")

View file

@ -158,6 +158,8 @@ object Persistence extends ExtensionId[Persistence] with ExtensionIdProvider {
/** Java API. */
override def get(system: ActorSystem): Persistence = super.get(system)
override def get(system: ClassicActorSystemProvider): Persistence = super.get(system)
def createExtension(system: ExtendedActorSystem): Persistence = new Persistence(system)
def lookup() = Persistence

View file

@ -24,6 +24,8 @@ import scala.reflect.ClassTag
private[akka] object SnapshotAfter extends ExtensionId[SnapshotAfter] with ExtensionIdProvider {
override def get(system: ActorSystem): SnapshotAfter = super.get(system)
override def get(system: ClassicActorSystemProvider): SnapshotAfter = super.get(system)
override def lookup = SnapshotAfter
override def createExtension(system: ExtendedActorSystem): SnapshotAfter = new SnapshotAfter(system.settings.config)

View file

@ -68,6 +68,7 @@ object PersistencePluginProxyExtension
new PersistencePluginProxyExtensionImpl(system)
override def lookup(): ExtensionId[_ <: Extension] = PersistencePluginProxyExtension
override def get(system: ActorSystem): PersistencePluginProxyExtensionImpl = super.get(system)
override def get(system: ClassicActorSystemProvider): PersistencePluginProxyExtensionImpl = super.get(system)
}
final class PersistencePluginProxy(config: Config) extends Actor with Stash with ActorLogging {

View file

@ -82,6 +82,7 @@ object JournalPuppet extends ExtensionId[JournalProbe] with ExtensionIdProvider
new JournalProbe()(system)
override def get(system: ActorSystem): JournalProbe = super.get(system)
override def get(system: ClassicActorSystemProvider): JournalProbe = super.get(system)
}
class JournalProbe(implicit private val system: ExtendedActorSystem) extends Extension {
val probe = TestProbe()

View file

@ -0,0 +1,7 @@
# Incompatibilities against Akka < 2.5.17 where extra static bridge methods were generated that
# Mima now wrongly correlates with the new ones added in the PR
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.remote.AddressUidExtension.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.remote.BoundAddressesExtension.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.remote.RemoteMetricsExtension.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.remote.serialization.ActorRefResolveThreadLocalCache.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.remote.transport.TransportAdaptersExtension.get")

View file

@ -5,7 +5,9 @@
package akka.remote
import java.util.concurrent.ThreadLocalRandom
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
import akka.actor.ExtensionId
@ -21,6 +23,7 @@ import akka.actor.ExtensionIdProvider
*/
object AddressUidExtension extends ExtensionId[AddressUidExtension] with ExtensionIdProvider {
override def get(system: ActorSystem): AddressUidExtension = super.get(system)
override def get(system: ClassicActorSystemProvider): AddressUidExtension = super.get(system)
override def lookup = AddressUidExtension

View file

@ -6,6 +6,7 @@ package akka.remote
import akka.actor.ActorSystem
import akka.actor.Address
import akka.actor.ClassicActorSystemProvider
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
import akka.actor.ExtensionId
@ -17,6 +18,7 @@ import akka.remote.artery.ArteryTransport
*/
object BoundAddressesExtension extends ExtensionId[BoundAddressesExtension] with ExtensionIdProvider {
override def get(system: ActorSystem): BoundAddressesExtension = super.get(system)
override def get(system: ClassicActorSystemProvider): BoundAddressesExtension = super.get(system)
override def lookup = BoundAddressesExtension

View file

@ -7,9 +7,9 @@ package akka.remote
import java.util.concurrent.ConcurrentHashMap
import scala.annotation.tailrec
import akka.actor.ActorSelectionMessage
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
import akka.actor.ExtensionId
@ -26,6 +26,7 @@ import com.github.ghik.silencer.silent
@silent("deprecated")
private[akka] object RemoteMetricsExtension extends ExtensionId[RemoteMetrics] with ExtensionIdProvider {
override def get(system: ActorSystem): RemoteMetrics = super.get(system)
override def get(system: ClassicActorSystemProvider): RemoteMetrics = super.get(system)
override def lookup = RemoteMetricsExtension

View file

@ -6,6 +6,7 @@ package akka.remote.serialization
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
import akka.actor.EmptyLocalActorRef
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
@ -23,6 +24,7 @@ private[akka] object ActorRefResolveThreadLocalCache
with ExtensionIdProvider {
override def get(system: ActorSystem): ActorRefResolveThreadLocalCache = super.get(system)
override def get(system: ClassicActorSystemProvider): ActorRefResolveThreadLocalCache = super.get(system)
override def lookup = ActorRefResolveThreadLocalCache

View file

@ -49,6 +49,7 @@ class TransportAdapters(system: ExtendedActorSystem) extends Extension {
@deprecated("Classic remoting is deprecated, use Artery", "2.6.0")
object TransportAdaptersExtension extends ExtensionId[TransportAdapters] with ExtensionIdProvider {
override def get(system: ActorSystem): TransportAdapters = super.get(system)
override def get(system: ClassicActorSystemProvider): TransportAdapters = super.get(system)
override def lookup = TransportAdaptersExtension
override def createExtension(system: ExtendedActorSystem): TransportAdapters =
new TransportAdapters(system)

View file

@ -18,6 +18,7 @@ import akka.testkit.JavaSerializable
object MetadataCarryingSpy extends ExtensionId[MetadataCarryingSpy] with ExtensionIdProvider {
override def get(system: ActorSystem): MetadataCarryingSpy = super.get(system)
override def get(system: ClassicActorSystemProvider): MetadataCarryingSpy = super.get(system)
override def lookup = MetadataCarryingSpy
override def createExtension(system: ExtendedActorSystem): MetadataCarryingSpy = new MetadataCarryingSpy

View file

@ -12,6 +12,7 @@ import scala.compat.java8.OptionConverters._
import scala.util.Failure
import scala.util.Success
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
import akka.actor.DynamicAccess
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
@ -44,6 +45,7 @@ import com.fasterxml.jackson.databind.json.JsonMapper
object JacksonObjectMapperProvider extends ExtensionId[JacksonObjectMapperProvider] with ExtensionIdProvider {
override def get(system: ActorSystem): JacksonObjectMapperProvider = super.get(system)
override def get(system: ClassicActorSystemProvider): JacksonObjectMapperProvider = super.get(system)
override def lookup = JacksonObjectMapperProvider

View file

@ -0,0 +1,8 @@
# Incompatibilities against Akka < 2.5.17 where extra static bridge methods were generated that
# Mima now wrongly correlates with the new ones added in the PR
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.stream.javadsl.Tcp.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.stream.scaladsl.Tcp.apply")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.stream.scaladsl.Tcp.get")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("com.typesafe.sslconfig.akka.AkkaSSLConfig.apply")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("com.typesafe.sslconfig.akka.AkkaSSLConfig.get")

View file

@ -7,6 +7,7 @@ package akka.stream
import akka.NotUsed
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
import akka.actor.ExtensionId
@ -120,11 +121,8 @@ final case class InvalidPartnerActorException(expectedRef: ActorRef, gotRef: Act
* The stream ref resolver extension provides a way to serialize and deserialize streamrefs in user serializers.
*/
object StreamRefResolver extends ExtensionId[StreamRefResolver] {
/**
* Java API
*/
override def get(system: ActorSystem): StreamRefResolver = super.get(system)
override def get(system: ClassicActorSystemProvider): StreamRefResolver = super.get(system)
override def createExtension(system: ExtendedActorSystem): StreamRefResolver =
new StreamRefResolverImpl(system)

View file

@ -5,6 +5,7 @@
package akka.stream
import akka.actor.ActorSystem
import akka.actor.ClassicActorSystemProvider
import akka.actor.Deploy
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
@ -29,6 +30,7 @@ import com.github.ghik.silencer.silent
*/
object SystemMaterializer extends ExtensionId[SystemMaterializer] with ExtensionIdProvider {
override def get(system: ActorSystem): SystemMaterializer = super.get(system)
override def get(system: ClassicActorSystemProvider): SystemMaterializer = super.get(system)
override def lookup = SystemMaterializer

View file

@ -174,6 +174,7 @@ private[akka] class SubFusingActorMaterializerImpl(
*/
@InternalApi private[akka] object FlowNames extends ExtensionId[FlowNames] with ExtensionIdProvider {
override def get(system: ActorSystem): FlowNames = super.get(system)
override def get(system: ClassicActorSystemProvider): FlowNames = super.get(system)
override def lookup() = FlowNames
override def createExtension(system: ExtendedActorSystem): FlowNames = new FlowNames
}

View file

@ -4,6 +4,7 @@
package akka.stream.impl.streamref
import akka.actor.ClassicActorSystemProvider
import akka.actor.{ ActorSystem, ExtendedActorSystem, Extension, ExtensionId, ExtensionIdProvider }
import akka.annotation.InternalApi
import akka.stream.impl.SeqActorName
@ -18,6 +19,7 @@ private[stream] object StreamRefsMaster extends ExtensionId[StreamRefsMaster] wi
override def lookup(): StreamRefsMaster.type = this
override def get(system: ActorSystem): StreamRefsMaster = super.get(system)
override def get(system: ClassicActorSystemProvider): StreamRefsMaster = super.get(system)
}
/** INTERNAL API */

View file

@ -130,6 +130,8 @@ object Tcp extends ExtensionId[Tcp] with ExtensionIdProvider {
override def get(system: ActorSystem): Tcp = super.get(system)
override def get(system: ClassicActorSystemProvider): Tcp = super.get(system)
def lookup() = Tcp
def createExtension(system: ExtendedActorSystem): Tcp = new Tcp(system)

View file

@ -81,6 +81,7 @@ object Tcp extends ExtensionId[Tcp] with ExtensionIdProvider {
def apply()(implicit system: ActorSystem): Tcp = super.apply(system)
override def get(system: ActorSystem): Tcp = super.get(system)
override def get(system: ClassicActorSystemProvider): Tcp = super.get(system)
def lookup() = Tcp

View file

@ -22,6 +22,7 @@ object AkkaSSLConfig extends ExtensionId[AkkaSSLConfig] 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 lookup() = AkkaSSLConfig

View file

@ -0,0 +1,4 @@
# Incompatibilities against Akka < 2.5.17 where extra static bridge methods were generated that
# Mima now wrongly correlates with the new ones added in the PR
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.testkit.TestKitExtension.get")

View file

@ -4,13 +4,16 @@
package akka.testkit
import akka.actor.ClassicActorSystemProvider
import com.typesafe.config.Config
import akka.util.Timeout
import akka.actor.{ ActorSystem, ExtendedActorSystem, Extension, ExtensionId }
import scala.concurrent.duration.FiniteDuration
object TestKitExtension extends ExtensionId[TestKitSettings] {
override def get(system: ActorSystem): TestKitSettings = super.get(system)
override def get(system: ClassicActorSystemProvider): TestKitSettings = super.get(system)
def createExtension(system: ExtendedActorSystem): TestKitSettings = new TestKitSettings(system.settings.config)
}