diff --git a/akka-actor-tests/src/test/java/akka/actor/setup/ActorSystemSetupTest.java b/akka-actor-tests/src/test/java/akka/actor/setup/ActorSystemSetupTest.java new file mode 100644 index 0000000000..ee969698d9 --- /dev/null +++ b/akka-actor-tests/src/test/java/akka/actor/setup/ActorSystemSetupTest.java @@ -0,0 +1,36 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.actor.setup; + +import akka.actor.setup.ActorSystemSetup; +import akka.actor.setup.Setup; +import org.junit.Test; +import org.scalatest.junit.JUnitSuite; + +import java.util.Optional; + +import static org.junit.Assert.*; + +public class ActorSystemSetupTest extends JUnitSuite { + + static class JavaSetup extends Setup { + public final String name; + public JavaSetup(String name) { + this.name = name; + } + } + + @Test + public void apiMustBeUsableFromJava() { + final JavaSetup javaSetting = new JavaSetup("Jasmine Rice"); + final Optional result = ActorSystemSetup.create() + .withSetup(javaSetting) + .get(JavaSetup.class); + + assertTrue(result.isPresent()); + assertEquals(result.get(), javaSetting); + + } + +} diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala index c21abb7e59..5e5eb7e56e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala @@ -6,9 +6,12 @@ package akka.actor import language.postfixOps import akka.testkit._ import com.typesafe.config.ConfigFactory -import scala.concurrent.{ ExecutionContext, Await, Future } + +import scala.concurrent.{ Await, ExecutionContext, Future } import scala.concurrent.duration._ -import java.util.concurrent.{ RejectedExecutionException, ConcurrentLinkedQueue } +import java.util.concurrent.{ ConcurrentLinkedQueue, RejectedExecutionException } + +import akka.actor.setup.ActorSystemSetup import akka.util.Timeout import akka.japi.Util.immutableSeq import akka.pattern.ask @@ -353,7 +356,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend "not allow top-level actor creation with custom guardian" in { val sys = new ActorSystemImpl("custom", ConfigFactory.defaultReference(), - getClass.getClassLoader, None, Some(Props.empty)) + getClass.getClassLoader, None, Some(Props.empty), ActorSystemSetup.empty) sys.start() try { intercept[UnsupportedOperationException] { diff --git a/akka-actor-tests/src/test/scala/akka/actor/setup/ActorSystemSetupSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/setup/ActorSystemSetupSpec.scala new file mode 100644 index 0000000000..400675eab3 --- /dev/null +++ b/akka-actor-tests/src/test/scala/akka/actor/setup/ActorSystemSetupSpec.scala @@ -0,0 +1,74 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.actor.setup + +import akka.actor.ActorSystem +import akka.testkit.TestKit +import org.scalatest.{ Matchers, WordSpec } + +case class DummySetup(name: String) extends Setup +case class DummySetup2(name: String) extends Setup +case class DummySetup3(name: String) extends Setup + +class ActorSystemSetupSpec extends WordSpec with Matchers { + + "The ActorSystemSettings" should { + + "store and retrieve a setup" in { + val setup = DummySetup("Al Dente") + val setups = ActorSystemSetup() + .withSetup(setup) + + setups.get[DummySetup] should ===(Some(setup)) + setups.get[DummySetup2] should ===(None) + } + + "replace setup if already defined" in { + val setup1 = DummySetup("Al Dente") + val setup2 = DummySetup("Earl E. Bird") + val setups = ActorSystemSetup() + .withSetup(setup1) + .withSetup(setup2) + + setups.get[DummySetup] should ===(Some(setup2)) + } + + "provide a fluent creation alternative" in { + val a = DummySetup("Al Dente") + val b = DummySetup("Earl E. Bird") // same type again + val c = DummySetup2("Amanda Reckonwith") + val setups = a and b and c + + setups.get[DummySetup] should ===(Some(b)) + setups.get[DummySetup2] should ===(Some(c)) + } + + "be created with a set of setups" in { + val setup1 = DummySetup("Manny Kin") + val setup2 = DummySetup2("Pepe Roni") + val setups = ActorSystemSetup(setup1, setup2) + + setups.get[DummySetup].isDefined shouldBe true + setups.get[DummySetup2].isDefined shouldBe true + setups.get[DummySetup3].isDefined shouldBe false + } + + "be available from the ExtendedActorSystem" in { + var system: ActorSystem = null + try { + val setup = DummySetup("Tad Moore") + system = ActorSystem("name", ActorSystemSetup(setup)) + + system + .settings + .setup + .get[DummySetup] should ===(Some(setup)) + + } finally { + TestKit.shutdownActorSystem(system) + } + } + } + +} diff --git a/akka-actor-tests/src/test/scala/akka/serialization/SerializationSetupSpec.scala b/akka-actor-tests/src/test/scala/akka/serialization/SerializationSetupSpec.scala new file mode 100644 index 0000000000..f260b7774a --- /dev/null +++ b/akka-actor-tests/src/test/scala/akka/serialization/SerializationSetupSpec.scala @@ -0,0 +1,57 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.serialization + +import akka.actor.setup.ActorSystemSetup +import akka.actor.{ ActorSystem, BootstrapSetup } +import akka.testkit.AkkaSpec +import com.typesafe.config.ConfigFactory + +class ConfigurationDummy +class ProgrammaticDummy + +object SerializationSetupSpec { + + val testSerializer = new TestSerializer + + val serializationSettings = SerializationSetup { _ ⇒ + List( + SerializerDetails("test", testSerializer, List(classOf[ProgrammaticDummy])) + ) + } + val bootstrapSettings = BootstrapSetup(None, Some(ConfigFactory.parseString(""" + akka { + actor { + serialize-messages = off + serialization-bindings { + "akka.serialization.ConfigurationDummy" = test + } + } + } + """)), None) + val actorSystemSettings = ActorSystemSetup(bootstrapSettings, serializationSettings) + +} + +class SerializationSetupSpec extends AkkaSpec( + ActorSystem("SerializationSettingsSpec", SerializationSetupSpec.actorSystemSettings) +) { + + import SerializationSetupSpec._ + + "The serialization settings" should { + + "allow for programmatic configuration of serializers" in { + val serializer = SerializationExtension(system).findSerializerFor(new ProgrammaticDummy) + serializer shouldBe theSameInstanceAs(testSerializer) + } + + "allow a configured binding to hook up to a programmatic serializer" in { + val serializer = SerializationExtension(system).findSerializerFor(new ConfigurationDummy) + serializer shouldBe theSameInstanceAs(testSerializer) + } + + } + +} diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index fed18c491f..ff5069548a 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -5,8 +5,9 @@ package akka.actor import java.io.Closeable -import java.util.concurrent.{ ConcurrentHashMap, ThreadFactory, CountDownLatch, RejectedExecutionException } -import java.util.concurrent.atomic.{ AtomicReference } +import java.util.concurrent.{ ConcurrentHashMap, CountDownLatch, RejectedExecutionException, ThreadFactory } +import java.util.concurrent.atomic.AtomicReference + import com.typesafe.config.{ Config, ConfigFactory } import akka.event._ import akka.dispatch._ @@ -14,13 +15,123 @@ import akka.japi.Util.immutableSeq import akka.actor.dungeon.ChildrenContainer import akka.util._ import akka.util.Helpers.toRootLowerCase + import scala.annotation.tailrec import scala.collection.immutable -import scala.concurrent.duration.{ Duration } -import scala.concurrent.{ Await, Future, Promise, ExecutionContext, ExecutionContextExecutor } +import scala.concurrent.duration.Duration +import scala.concurrent.{ Await, ExecutionContext, ExecutionContextExecutor, Future, Promise } import scala.util.{ Failure, Success, Try } -import scala.util.control.{ NonFatal, ControlThrowable } -import java.util.Locale +import scala.util.control.{ ControlThrowable, NonFatal } +import java.util.Optional + +import akka.actor.setup.{ ActorSystemSetup, Setup } + +import scala.compat.java8.OptionConverters._ + +object BootstrapSetup { + + /** + * Scala API: Construct a bootstrap settings with default values. Note that passing that to the actor system is the + * same as not passing any [[BootstrapSetup]] at all. You can use the returned instance to derive + * one that has other values than defaults using the various `with`-methods. + */ + def apply(): BootstrapSetup = { + new BootstrapSetup() + } + + /** + * Scala API: Create bootstrap settings needed for starting the actor system + * + * @see [[BootstrapSetup]] for description of the properties + */ + def apply(classLoader: Option[ClassLoader], config: Option[Config], defaultExecutionContext: Option[ExecutionContext]): BootstrapSetup = + new BootstrapSetup(classLoader, config, defaultExecutionContext) + + /** + * Scala API: Short for using custom config but keeping default classloader and default execution context + */ + def apply(config: Config): BootstrapSetup = apply(None, Some(config), None) + + /** + * Java API: Create bootstrap settings needed for starting the actor system + * + * @see [[BootstrapSetup]] for description of the properties + */ + def create(classLoader: Optional[ClassLoader], config: Optional[Config], defaultExecutionContext: Optional[ExecutionContext]): BootstrapSetup = + apply(classLoader.asScala, config.asScala, defaultExecutionContext.asScala) + + /** + * Java API: Short for using custom config but keeping default classloader and default execution context + */ + def create(config: Config): BootstrapSetup = apply(config) + + /** + * Java API: Construct a bootstrap settings with default values. Note that passing that to the actor system is the + * same as not passing any [[BootstrapSetup]] at all. You can use the returned instance to derive + * one that has other values than defaults using the various `with`-methods. + */ + def create(): BootstrapSetup = { + new BootstrapSetup() + } + +} + +abstract class ProviderSelection private (private[akka] val identifier: String) +object ProviderSelection { + case object Local extends ProviderSelection("local") + case object Remote extends ProviderSelection("remote") + case object Cluster extends ProviderSelection("cluster") + + /** + * JAVA API + */ + def local(): ProviderSelection = Local + + /** + * JAVA API + */ + def remote(): ProviderSelection = Remote + + /** + * JAVA API + */ + def cluster(): ProviderSelection = Cluster + +} + +/** + * Core bootstrap settings of the actor system, create using one of the factories in [[BootstrapSetup]], + * constructor is *Internal API*. + * + * @param classLoader If no ClassLoader is given, it obtains the current ClassLoader by first inspecting the current + * threads' getContextClassLoader, then tries to walk the stack to find the callers class loader, then + * falls back to the ClassLoader associated with the ActorSystem class. + * @param config Configuration to use for the actor system. If no Config is given, the default reference config will be obtained from the ClassLoader. + * @param defaultExecutionContext If defined the ExecutionContext will be used as the default executor inside this ActorSystem. + * If no ExecutionContext is given, the system will fallback to the executor configured under + * "akka.actor.default-dispatcher.default-executor.fallback". + * @param actorRefProvider Overrides the `akka.actor.provider` setting in config, can be `local` (default), `remote` or + * `cluster`. It can also be a fully qualified class name of a provider. + */ +final class BootstrapSetup private ( + val classLoader: Option[ClassLoader] = None, + val config: Option[Config] = None, + val defaultExecutionContext: Option[ExecutionContext] = None, + val actorRefProvider: Option[ProviderSelection] = None) extends Setup { + + def withClassloader(classLoader: ClassLoader): BootstrapSetup = + new BootstrapSetup(Some(classLoader), config, defaultExecutionContext, actorRefProvider) + + def withConfig(config: Config): BootstrapSetup = + new BootstrapSetup(classLoader, Some(config), defaultExecutionContext, actorRefProvider) + + def withDefaultExecutionContext(executionContext: ExecutionContext): BootstrapSetup = + new BootstrapSetup(classLoader, config, Some(executionContext), actorRefProvider) + + def withActorRefProvider(name: ProviderSelection): BootstrapSetup = + new BootstrapSetup(classLoader, config, defaultExecutionContext, Some(name)) + +} object ActorSystem { @@ -56,6 +167,19 @@ object ActorSystem { */ def create(name: String): ActorSystem = apply(name) + /** + * Java API: Creates a new actor system with the specified name and settings + * The core actor system settings are defined in [[BootstrapSetup]] + */ + def create(name: String, setups: ActorSystemSetup): ActorSystem = apply(name, setups) + + /** + * Java API: Shortcut for creating an actor system with custom bootstrap settings. + * Same behaviour as calling `ActorSystem.create(name, ActorSystemSetup.create(bootstrapSettings))` + */ + def create(name: String, bootstrapSetup: BootstrapSetup): ActorSystem = + create(name, ActorSystemSetup.create(bootstrapSetup)) + /** * Creates a new ActorSystem with the specified name, and the specified Config, then * obtains the current ClassLoader by first inspecting the current threads' getContextClassLoader, @@ -108,6 +232,26 @@ object ActorSystem { */ def apply(name: String): ActorSystem = apply(name, None, None, None) + /** + * Scala API: Creates a new actor system with the specified name and settings + * The core actor system settings are defined in [[BootstrapSetup]] + */ + def apply(name: String, setup: ActorSystemSetup): ActorSystem = { + val bootstrapSettings = setup.get[BootstrapSetup] + val cl = bootstrapSettings.flatMap(_.classLoader).getOrElse(findClassLoader()) + val appConfig = bootstrapSettings.flatMap(_.config).getOrElse(ConfigFactory.load(cl)) + val defaultEC = bootstrapSettings.flatMap(_.defaultExecutionContext) + + new ActorSystemImpl(name, appConfig, cl, defaultEC, None, setup).start() + } + + /** + * Scala API: Shortcut for creating an actor system with custom bootstrap settings. + * Same behaviour as calling `ActorSystem(name, ActorSystemSetup(bootstrapSetup))` + */ + def apply(name: String, bootstrapSetup: BootstrapSetup): ActorSystem = + create(name, ActorSystemSetup.create(bootstrapSetup)) + /** * Creates a new ActorSystem with the specified name, and the specified Config, then * obtains the current ClassLoader by first inspecting the current threads' getContextClassLoader, @@ -136,11 +280,12 @@ object ActorSystem { * * @see The Typesafe Config Library API Documentation */ - def apply(name: String, config: Option[Config] = None, classLoader: Option[ClassLoader] = None, defaultExecutionContext: Option[ExecutionContext] = None): ActorSystem = { - val cl = classLoader.getOrElse(findClassLoader()) - val appConfig = config.getOrElse(ConfigFactory.load(cl)) - new ActorSystemImpl(name, appConfig, cl, defaultExecutionContext, None).start() - } + def apply( + name: String, + config: Option[Config] = None, + classLoader: Option[ClassLoader] = None, + defaultExecutionContext: Option[ExecutionContext] = None): ActorSystem = + apply(name, ActorSystemSetup(BootstrapSetup(classLoader, config, defaultExecutionContext))) /** * Settings are the overall ActorSystem Settings which also provides a convenient access to the Config object. @@ -149,7 +294,9 @@ object ActorSystem { * * @see The Typesafe Config Library API Documentation */ - class Settings(classLoader: ClassLoader, cfg: Config, final val name: String) { + class Settings(classLoader: ClassLoader, cfg: Config, final val name: String, val setup: ActorSystemSetup) { + + def this(classLoader: ClassLoader, cfg: Config, name: String) = this(classLoader, cfg, name, ActorSystemSetup()) /** * The backing Config of this ActorSystem's Settings @@ -167,13 +314,15 @@ object ActorSystem { final val ConfigVersion: String = getString("akka.version") final val ProviderClass: String = - getString("akka.actor.provider") match { - case "local" ⇒ classOf[LocalActorRefProvider].getName - // these two cannot be referenced by class as they may not be on the classpath - case "remote" ⇒ "akka.remote.RemoteActorRefProvider" - case "cluster" ⇒ "akka.cluster.ClusterActorRefProvider" - case fqcn ⇒ fqcn - } + setup.get[BootstrapSetup] + .flatMap(_.actorRefProvider).map(_.identifier) + .getOrElse(getString("akka.actor.provider")) match { + case "local" ⇒ classOf[LocalActorRefProvider].getName + // these two cannot be referenced by class as they may not be on the classpath + case "remote" ⇒ "akka.remote.RemoteActorRefProvider" + case "cluster" ⇒ "akka.cluster.ClusterActorRefProvider" + case fqcn ⇒ fqcn + } final val SupervisorStrategyClass: String = getString("akka.actor.guardian-supervisor-strategy") final val CreationTimeout: Timeout = Timeout(config.getMillisDuration("akka.actor.creation-timeout")) @@ -517,7 +666,8 @@ private[akka] class ActorSystemImpl( applicationConfig: Config, classLoader: ClassLoader, defaultExecutionContext: Option[ExecutionContext], - val guardianProps: Option[Props]) extends ExtendedActorSystem { + val guardianProps: Option[Props], + setup: ActorSystemSetup) extends ExtendedActorSystem { if (!name.matches("""^[a-zA-Z0-9][a-zA-Z0-9-_]*$""")) throw new IllegalArgumentException( @@ -527,7 +677,7 @@ private[akka] class ActorSystemImpl( import ActorSystem._ @volatile private var logDeadLetterListener: Option[ActorRef] = None - final val settings: Settings = new Settings(classLoader, applicationConfig, name) + final val settings: Settings = new Settings(classLoader, applicationConfig, name, setup) protected def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = new Thread.UncaughtExceptionHandler() { diff --git a/akka-actor/src/main/scala/akka/actor/setup/ActorSystemSetup.scala b/akka-actor/src/main/scala/akka/actor/setup/ActorSystemSetup.scala new file mode 100644 index 0000000000..edf8c84b8f --- /dev/null +++ b/akka-actor/src/main/scala/akka/actor/setup/ActorSystemSetup.scala @@ -0,0 +1,84 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.actor.setup + +import java.util.Optional + +import scala.annotation.varargs +import scala.compat.java8.OptionConverters._ +import scala.reflect.ClassTag + +/** + * Marker supertype for a setup part that can be put inside [[ActorSystemSetup]], if a specific concrete setup + * is not specified in the actor system setup that means defaults are used (usually from the config file) - no concrete + * setup instance should be mandatory in the [[ActorSystemSetup]] that an actor system is created with. + */ +abstract class Setup { + + /** + * Construct an [[ActorSystemSetup]] with this setup combined with another one. Allows for + * fluent creation of settings. If `other` is a setting of the same concrete [[Setup]] as this + * it will replace this. + */ + final def and(other: Setup): ActorSystemSetup = ActorSystemSetup(this, other) + +} + +object ActorSystemSetup { + + val empty = new ActorSystemSetup(Map.empty) + + /** + * Scala API: Create an [[ActorSystemSetup]] containing all the provided settings + */ + def apply(settings: Setup*): ActorSystemSetup = + new ActorSystemSetup(settings.map(s ⇒ s.getClass → s).toMap) + + /** + * Java API: Create an [[ActorSystemSetup]] containing all the provided settings + */ + @varargs + def create(settings: Setup*): ActorSystemSetup = apply(settings: _*) +} + +/** + * A set of setup settings for programmatic configuration of the actor system. + * + * Constructor is *Internal API*. Use the factory methods [[ActorSystemSetup#create]] and [[akka.actor.Actor#apply]] to create + * instances. + */ +final class ActorSystemSetup private[akka] (setups: Map[Class[_], AnyRef]) { + + /** + * Java API: Extract a concrete [[Setup]] of type `T` if it is defined in the settings. + */ + def get[T <: Setup](clazz: Class[T]): Optional[T] = { + setups.get(clazz).map(_.asInstanceOf[T]).asJava + } + + /** + * Scala API: Extract a concrete [[Setup]] of type `T` if it is defined in the settings. + */ + def get[T <: Setup: ClassTag]: Option[T] = { + val clazz = implicitly[ClassTag[T]].runtimeClass + setups.get(clazz).map(_.asInstanceOf[T]) + } + + /** + * Add a concrete [[Setup]]. If a setting of the same concrete [[Setup]] already is + * present it will be replaced. + */ + def withSetup[T <: Setup](t: T): ActorSystemSetup = { + new ActorSystemSetup(setups + (t.getClass → t)) + } + + /** + * alias for `withSetting` allowing for fluent combination of settings: `a and b and c`, where `a`, `b` and `c` are + * concrete [[Setup]] instances. If a setting of the same concrete [[Setup]] already is + * present it will be replaced. + */ + def and[T <: Setup](t: T): ActorSystemSetup = withSetup(t) + + override def toString: String = s"""ActorSystemSettings(${setups.keys.map(_.getName).mkString(",")})""" +} diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index c7814ab093..e98d6a0c0e 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -253,20 +253,40 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { case _: NoSuchMethodException ⇒ system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Nil) } + /** + * Programmatically defined serializers + */ + private val serializerDetails = + system.settings.setup.get[SerializationSetup] match { + case None ⇒ Vector.empty + case Some(setting) ⇒ setting.createSerializers(system) + } + /** * A Map of serializer from alias to implementation (class implementing akka.serialization.Serializer) * By default always contains the following mapping: "java" -> akka.serialization.JavaSerializer */ - private val serializers: Map[String, Serializer] = - for ((k: String, v: String) ← settings.Serializers) yield k → serializerOf(v).get + private val serializers: Map[String, Serializer] = { + val fromConfig = for ((k: String, v: String) ← settings.Serializers) yield k → serializerOf(v).get + fromConfig ++ serializerDetails.map(d ⇒ d.alias → d.serializer) + } /** * bindings is a Seq of tuple representing the mapping from Class to Serializer. * It is primarily ordered by the most specific classes first, and secondly in the configured order. */ - private[akka] val bindings: immutable.Seq[ClassSerializer] = - sort(for ((k: String, v: String) ← settings.SerializationBindings if v != "none" && checkGoogleProtobuf(k)) - yield (system.dynamicAccess.getClassFor[Any](k).get, serializers(v))).to[immutable.Seq] + private[akka] val bindings: immutable.Seq[ClassSerializer] = { + val fromConfig = for { + (className: String, alias: String) ← settings.SerializationBindings + if alias != "none" && checkGoogleProtobuf(className) + } yield (system.dynamicAccess.getClassFor[Any](className).get, serializers(alias)) + + val fromSettings = serializerDetails.flatMap { detail ⇒ + detail.useFor.map(clazz ⇒ clazz → detail.serializer) + } + + sort(fromConfig ++ fromSettings) + } // com.google.protobuf serialization binding is only used if the class can be loaded, // i.e. com.google.protobuf dependency has been added in the application project. diff --git a/akka-actor/src/main/scala/akka/serialization/SerializationSetup.scala b/akka-actor/src/main/scala/akka/serialization/SerializationSetup.scala new file mode 100644 index 0000000000..da69d6f390 --- /dev/null +++ b/akka-actor/src/main/scala/akka/serialization/SerializationSetup.scala @@ -0,0 +1,69 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.serialization + +import akka.actor.ExtendedActorSystem +import akka.actor.setup.Setup + +import scala.collection.immutable +import scala.collection.JavaConverters._ + +object SerializationSetup { + + /** + * Scala API: Programmatic definition of serializers + * @param createSerializers create pairs of serializer and the set of classes it should be used for + */ + def apply(createSerializers: ExtendedActorSystem ⇒ immutable.Seq[SerializerDetails]): SerializationSetup = { + new SerializationSetup(createSerializers) + } + + /** + * Java API: Programmatic definition of serializers + * @param createSerializers create pairs of serializer and the set of classes it should be used for + */ + def create( + createSerializers: akka.japi.Function[ExtendedActorSystem, java.util.List[SerializerDetails]]): SerializationSetup = + apply(sys ⇒ createSerializers(sys).asScala.toVector) + +} + +/** + * Setup for the serialization subsystem, constructor is *Internal API*, use factories in [[SerializationSetup()]] + */ +final class SerializationSetup private ( + val createSerializers: ExtendedActorSystem ⇒ immutable.Seq[SerializerDetails] +) extends Setup + +object SerializerDetails { + /** + * Scala API: factory for details about one programmatically setup serializer + * + * @param alias Register the serializer under this alias (this allows it to be used by bindings in the config) + * @param useFor A set of classes or superclasses to bind to the serializer, selection works just as if + * the classes, the alias and the serializer had been in the config. + */ + def apply(alias: String, serializer: Serializer, useFor: immutable.Seq[Class[_]]): SerializerDetails = + new SerializerDetails(alias, serializer, useFor) + + /** + * Java API: factory for details about one programmatically setup serializer + * + * @param alias Register the serializer under this alias (this allows it to be used by bindings in the config) + * @param useFor A set of classes or superclasses to bind to the serializer, selection works just as if + * the classes, the alias and the serializer had been in the config. + */ + def create(alias: String, serializer: Serializer, useFor: java.util.List[Class[_]]): SerializerDetails = + apply(alias, serializer, useFor.asScala.toVector) + +} + +/** + * Constructor is internal API: Use the factories [[SerializerDetails#create]] or [[SerializerDetails#apply]] + * to construct + */ +final class SerializerDetails private ( + val alias: String, + val serializer: Serializer, + val useFor: immutable.Seq[Class[_]]) \ No newline at end of file diff --git a/akka-typed/src/main/scala/akka/typed/ActorSystem.scala b/akka-typed/src/main/scala/akka/typed/ActorSystem.scala index 059033cec5..20a0423cce 100644 --- a/akka-typed/src/main/scala/akka/typed/ActorSystem.scala +++ b/akka-typed/src/main/scala/akka/typed/ActorSystem.scala @@ -6,9 +6,12 @@ package akka.typed import scala.concurrent.ExecutionContext import akka.{ actor ⇒ a, event ⇒ e } import java.util.concurrent.ThreadFactory + +import akka.actor.setup.ActorSystemSetup import com.typesafe.config.{ Config, ConfigFactory } + import scala.concurrent.{ ExecutionContextExecutor, Future } -import akka.typed.adapter.{ PropsAdapter, ActorSystemAdapter } +import akka.typed.adapter.{ ActorSystemAdapter, PropsAdapter } import akka.util.Timeout /** @@ -163,14 +166,15 @@ object ActorSystem { * system typed and untyped actors can coexist. */ def adapter[T](name: String, guardianBehavior: Behavior[T], - guardianDeployment: DeploymentConfig = EmptyDeploymentConfig, - config: Option[Config] = None, - classLoader: Option[ClassLoader] = None, - executionContext: Option[ExecutionContext] = None): ActorSystem[T] = { + guardianDeployment: DeploymentConfig = EmptyDeploymentConfig, + config: Option[Config] = None, + classLoader: Option[ClassLoader] = None, + executionContext: Option[ExecutionContext] = None, + actorSystemSettings: ActorSystemSetup = ActorSystemSetup.empty): ActorSystem[T] = { Behavior.validateAsInitial(guardianBehavior) val cl = classLoader.getOrElse(akka.actor.ActorSystem.findClassLoader()) val appConfig = config.getOrElse(ConfigFactory.load(cl)) - val untyped = new a.ActorSystemImpl(name, appConfig, cl, executionContext, Some(PropsAdapter(guardianBehavior, guardianDeployment))) + val untyped = new a.ActorSystemImpl(name, appConfig, cl, executionContext, Some(PropsAdapter(guardianBehavior, guardianDeployment)), actorSystemSettings) untyped.start() new ActorSystemAdapter(untyped) } diff --git a/project/MiMa.scala b/project/MiMa.scala index 98abd66957..a0e2178ca8 100644 --- a/project/MiMa.scala +++ b/project/MiMa.scala @@ -657,10 +657,14 @@ object MiMa extends AutoPlugin { // https://github.com/akka/akka/pull/21688 ProblemFilters.exclude[MissingClassProblem]("akka.stream.Fusing$StructuralInfo$"), ProblemFilters.exclude[MissingClassProblem]("akka.stream.Fusing$StructuralInfo"), - + // https://github.com/akka/akka/pull/21989 - add more information in tcp connection shutdown logs (add mapError) - ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.mapError") + ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.mapError"), + + // #21894 Programmatic configuration of the ActorSystem + ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorSystemImpl.this") ) ++ bcIssuesBetween24and25) + // Entries should be added to a section keyed with the latest released version before the change ) } }