diff --git a/akka-actor-tests/src/test/scala/akka/serialization/SerializationSetupSpec.scala b/akka-actor-tests/src/test/scala/akka/serialization/SerializationSetupSpec.scala index 50440d4c7c..8f34dece26 100644 --- a/akka-actor-tests/src/test/scala/akka/serialization/SerializationSetupSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/serialization/SerializationSetupSpec.scala @@ -4,12 +4,16 @@ package akka.serialization import akka.actor.setup.ActorSystemSetup -import akka.actor.{ ActorSystem, BootstrapSetup } -import akka.testkit.AkkaSpec +import akka.actor.{ ActorSystem, BootstrapSetup, ExtendedActorSystem, Terminated } +import akka.testkit.{ AkkaSpec, TestKit, TestProbe } import com.typesafe.config.ConfigFactory +import scala.concurrent.duration._ + class ConfigurationDummy class ProgrammaticDummy +case class ProgrammaticJavaDummy() +case class SerializableDummy() // since case classes are serializable object SerializationSetupSpec { @@ -17,13 +21,13 @@ object SerializationSetupSpec { val serializationSettings = SerializationSetup { _ ⇒ List( - SerializerDetails("test", testSerializer, List(classOf[ProgrammaticDummy])) - ) + SerializerDetails("test", testSerializer, List(classOf[ProgrammaticDummy]))) } val bootstrapSettings = BootstrapSetup(None, Some(ConfigFactory.parseString(""" akka { actor { serialize-messages = off + serialization-bindings { "akka.serialization.ConfigurationDummy" = test } @@ -32,11 +36,20 @@ object SerializationSetupSpec { """)), None) val actorSystemSettings = ActorSystemSetup(bootstrapSettings, serializationSettings) + val noJavaSerializationSystem = ActorSystem("SerializationSettingsSpec" + "NoJavaSerialization", ConfigFactory.parseString( + """ + akka { + actor { + allow-java-serialization = off + } + } + """.stripMargin)) + val noJavaSerializer = new DisabledJavaSerializer(noJavaSerializationSystem.asInstanceOf[ExtendedActorSystem]) + } class SerializationSetupSpec extends AkkaSpec( - ActorSystem("SerializationSettingsSpec", SerializationSetupSpec.actorSystemSettings) -) { + ActorSystem("SerializationSettingsSpec", SerializationSetupSpec.actorSystemSettings)) { import SerializationSetupSpec._ @@ -54,4 +67,70 @@ class SerializationSetupSpec extends AkkaSpec( } + // This is a weird edge case, someone creating a JavaSerializer manually and using it in a system means + // that they'd need a different actor system to be able to create it... someone MAY pick a system with + // allow-java-serialization=on to create the SerializationSetup and use that SerializationSetup + // in another system with allow-java-serialization=off + val addedJavaSerializationSettings = SerializationSetup { _ ⇒ + List( + SerializerDetails("test", testSerializer, List(classOf[ProgrammaticDummy])), + SerializerDetails("java-manual", new JavaSerializer(system.asInstanceOf[ExtendedActorSystem]), List(classOf[ProgrammaticJavaDummy]))) + } + val addedJavaSerializationProgramaticallyButDisabledSettings = BootstrapSetup(None, Some(ConfigFactory.parseString(""" + akka { + loglevel = debug + actor { + allow-java-serialization = off + } + } + """)), None) + + val addedJavaSerializationViaSettingsSystem = + ActorSystem("addedJavaSerializationSystem", ActorSystemSetup(addedJavaSerializationProgramaticallyButDisabledSettings, addedJavaSerializationSettings)) + + "Disabling java serialization" should { + + "throw if passed system to JavaSerializer has allow-java-serialization = off" in { + intercept[DisabledJavaSerializer.JavaSerializationException] { + new JavaSerializer(noJavaSerializationSystem.asInstanceOf[ExtendedActorSystem]) + }.getMessage should include("akka.actor.allow-java-serialization = off") + + intercept[DisabledJavaSerializer.JavaSerializationException] { + SerializationExtension(addedJavaSerializationViaSettingsSystem).findSerializerFor(new ProgrammaticJavaDummy).toBinary(new ProgrammaticJavaDummy) + } + } + + "have replaced java serializer" in { + val p = TestProbe()(addedJavaSerializationViaSettingsSystem) // only receiver has the serialization disabled + + p.ref ! new ProgrammaticJavaDummy + SerializationExtension(system).findSerializerFor(new ProgrammaticJavaDummy).toBinary(new ProgrammaticJavaDummy) + // should not receive this one, it would have been java serialization! + p.expectNoMsg(100.millis) + + p.ref ! new ProgrammaticDummy + p.expectMsgType[ProgrammaticDummy] + } + + "disable java serialization also for incoming messages if serializer id usually would have found the serializer" in { + val ser1 = SerializationExtension(system) + val msg = SerializableDummy() + val bytes = ser1.serialize(msg).get + val serId = ser1.findSerializerFor(msg).identifier + ser1.findSerializerFor(msg).includeManifest should ===(false) + + val ser2 = SerializationExtension(noJavaSerializationSystem) + ser2.findSerializerFor(new SerializableDummy) should ===(noJavaSerializer) + ser2.serializerByIdentity(serId) should ===(noJavaSerializer) + intercept[DisabledJavaSerializer.JavaSerializationException] { + ser2.deserialize(bytes, serId, "").get + } + } + } + + override def afterTermination(): Unit = { + TestKit.shutdownActorSystem(noJavaSerializationSystem) + TestKit.shutdownActorSystem(addedJavaSerializationViaSettingsSystem) + } + } diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index 9a56dcbe2d..074e0922d2 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -567,7 +567,28 @@ akka { # enable WARN logging of misconfigured routers router-misconfiguration = off } - + + # SECURITY BEST-PRACTICE is to disable java serialization for its multiple + # known attack surfaces. + # + # This setting is a short-cut to + # - using DisabledJavaSerializer instead of JavaSerializer + # - enable-additional-serialization-bindings = on + # + # Completely disable the use of `akka.serialization.JavaSerialization` by the + # Akka Serialization extension, instead DisabledJavaSerializer will + # be inserted which will fail explicitly if attempts to use java serialization are made. + # + # The log messages emitted by such serializer SHOULD be be treated as potential + # attacks which the serializer prevented, as they MAY indicate an external operator + # attempting to send malicious messages intending to use java serialization as attack vector. + # The attempts are logged with the SECURITY marker. + # + # Please note that this option does not stop you from manually invoking java serialization + # + # The default value for this might be changed to off in future versions of Akka. + allow-java-serialization = on + # Entries for pluggable serializers and their bindings. serializers { java = "akka.serialization.JavaSerializer" @@ -586,14 +607,17 @@ akka { "java.io.Serializable" = java } - # Set this to off to disable serialization-bindings define in - # additional-serialization-bindings. That should only be needed - # for backwards compatibility reasons. + # Additional serialization-bindings that are replacing Java serialization are + # defined in this section for backwards compatibility reasons. They are included + # by default but can be excluded for backwards compatibility with Akka 2.4.x. + # They can be disabled with enable-additional-serialization-bindings=off. + # + # This should only be needed for backwards compatibility reasons. enable-additional-serialization-bindings = on # Additional serialization-bindings that are replacing Java serialization are # defined in this section for backwards compatibility reasons. They are included - # by default but can be excluded for backwards compatibility with Akka 2.4.x. + # by default but can be excluded for backwards compatibility with Akka 2.4.x. # They can be disabled with enable-additional-serialization-bindings=off. additional-serialization-bindings { } diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 775690c7f9..85c7f4132b 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -114,6 +114,7 @@ final case class Terminated private[akka] (@BeanProperty actor: ActorRef)( @BeanProperty val existenceConfirmed: Boolean, @BeanProperty val addressTerminated: Boolean) extends AutoReceivedMessage with PossiblyHarmful with DeadLetterSuppression + with NoSerializationVerificationNeeded // local message, the remote one is DeathWatchNotification /** * INTERNAL API diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index e5f686ade4..7d87ca38a8 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -327,6 +327,9 @@ object ActorSystem { final val CreationTimeout: Timeout = Timeout(config.getMillisDuration("akka.actor.creation-timeout")) final val UnstartedPushTimeout: Timeout = Timeout(config.getMillisDuration("akka.actor.unstarted-push-timeout")) + final val AllowJavaSerialization: Boolean = getBoolean("akka.actor.allow-java-serialization") + final val EnableAdditionalSerializationBindings: Boolean = + !AllowJavaSerialization || getBoolean("akka.actor.enable-additional-serialization-bindings") final val SerializeAllMessages: Boolean = getBoolean("akka.actor.serialize-messages") final val SerializeAllCreators: Boolean = getBoolean("akka.actor.serialize-creators") diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 7a79219b75..763e3e6911 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -42,9 +42,12 @@ object Serialization { val defaultBindings = config.getConfig("akka.actor.serialization-bindings") val bindings = if (config.getBoolean("akka.actor.enable-additional-serialization-bindings") || - config.hasPath("akka.remote.artery.enabled") && config.getBoolean("akka.remote.artery.enabled")) + !config.getBoolean("akka.actor.allow-java-serialization") || + config.hasPath("akka.remote.artery.enabled") && config.getBoolean("akka.remote.artery.enabled")) { defaultBindings.withFallback(config.getConfig("akka.actor.additional-serialization-bindings")) - else defaultBindings + } else { + defaultBindings + } configToMap(bindings) } @@ -97,6 +100,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { import Serialization._ val settings = new Settings(system.settings.config) + val AllowJavaSerialization: Boolean = system.settings.AllowJavaSerialization + private[this] val _log = Logging.withMarker(system, getClass.getName) val log: LoggingAdapter = _log private val manifestCache = new AtomicReference[Map[String, Option[Class[_]]]](Map.empty[String, Option[Class[_]]]) @@ -222,14 +227,19 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { (possibilities forall (_._1 isAssignableFrom possibilities(0)._1)) || (possibilities forall (_._2 == possibilities(0)._2)) - val ser = bindings filter { _._1 isAssignableFrom clazz } match { - case Seq() ⇒ - throw new NotSerializableException("No configured serialization-bindings for class [%s]" format clazz.getName) - case possibilities ⇒ - if (!unique(possibilities)) - _log.warning(LogMarker.Security, "Multiple serializers found for " + clazz + ", choosing first: " + possibilities) - possibilities(0)._2 + val ser = { + bindings.filter { + case (c, _) ⇒ c isAssignableFrom clazz + } match { + case immutable.Seq() ⇒ + throw new NotSerializableException("No configured serialization-bindings for class [%s]" format clazz.getName) + case possibilities ⇒ + if (!unique(possibilities)) + _log.warning(LogMarker.Security, "Multiple serializers found for " + clazz + ", choosing first: " + possibilities) + possibilities(0)._2 + } } + serializerMap.putIfAbsent(clazz, ser) match { case null ⇒ if (shouldWarnAboutJavaSerializer(clazz, ser)) { @@ -237,7 +247,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { "performance implications. Use another serializer or disable this warning using the setting " + "'akka.actor.warn-about-java-serializer-usage'", clazz.getName) } - log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName) + log.debug("Using serializer [{}] for message [{}]", ser.getClass.getName, clazz.getName) ser case some ⇒ some } @@ -248,18 +258,34 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { * Tries to load the specified Serializer by the fully-qualified name; the actual * loading is performed by the system’s [[akka.actor.DynamicAccess]]. */ - def serializerOf(serializerFQN: String): Try[Serializer] = - system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, List(classOf[ExtendedActorSystem] → system)) recoverWith { - case _: NoSuchMethodException ⇒ system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Nil) + def serializerOf(serializerFQN: String): Try[Serializer] = { + // We override each instantiation of the JavaSerializer with the "disabled" serializer which will log warnings if used. + val fqn = + if (!system.settings.AllowJavaSerialization && serializerFQN == classOf[JavaSerializer].getName) { + log.debug("Replacing JavaSerializer with DisabledJavaSerializer, " + + "due to `akka.actor.allow-java-serialization = off`.") + classOf[DisabledJavaSerializer].getName + } else serializerFQN + + system.dynamicAccess.createInstanceFor[Serializer](fqn, List(classOf[ExtendedActorSystem] → system)) recoverWith { + case _: NoSuchMethodException ⇒ + system.dynamicAccess.createInstanceFor[Serializer](fqn, Nil) } + } /** * Programmatically defined serializers */ - private val serializerDetails = - system.settings.setup.get[SerializationSetup] match { + private val serializerDetails: immutable.Seq[SerializerDetails] = + (system.settings.setup.get[SerializationSetup] match { case None ⇒ Vector.empty case Some(setting) ⇒ setting.createSerializers(system) + }) collect { + case det: SerializerDetails if isDisallowedJavaSerializer(det.serializer) ⇒ + log.debug("Replacing JavaSerializer with DisabledJavaSerializer, " + + "due to `akka.actor.allow-java-serialization = off`.") + SerializerDetails(det.alias, new DisabledJavaSerializer(system), det.useFor) + case det ⇒ det } /** @@ -268,7 +294,9 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { */ 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) + val result = fromConfig ++ serializerDetails.map(d ⇒ d.alias → d.serializer) + ensureOnlyAllowedSerializers(result.map { case (_, ser) ⇒ ser }(collection.breakOut)) + result } /** @@ -285,7 +313,15 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { detail.useFor.map(clazz ⇒ clazz → detail.serializer) } - sort(fromConfig ++ fromSettings) + val result = sort(fromConfig ++ fromSettings) + ensureOnlyAllowedSerializers(result.map { case (_, ser) ⇒ ser }(collection.breakOut)) + result + } + + private def ensureOnlyAllowedSerializers(iter: Iterator[Serializer]): Unit = { + if (!system.settings.AllowJavaSerialization) { + require(iter.forall(!isDisallowedJavaSerializer(_)), "Disallowed JavaSerializer binding.") + } } // com.google.protobuf serialization binding is only used if the class can be loaded, @@ -349,6 +385,10 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { private val isJavaSerializationWarningEnabled = settings.config.getBoolean("akka.actor.warn-about-java-serializer-usage") private val isWarningOnNoVerificationEnabled = settings.config.getBoolean("akka.actor.warn-on-no-serialization-verification") + private def isDisallowedJavaSerializer(serializer: Serializer): Boolean = { + serializer.isInstanceOf[JavaSerializer] && !system.settings.AllowJavaSerialization + } + private def shouldWarnAboutJavaSerializer(serializedClass: Class[_], serializer: Serializer) = { def suppressWarningOnNonSerializationVerification(serializedClass: Class[_]) = { @@ -357,7 +397,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { } isJavaSerializationWarningEnabled && - serializer.isInstanceOf[JavaSerializer] && + (serializer.isInstanceOf[JavaSerializer] || serializer.isInstanceOf[DisabledJavaSerializer]) && !serializedClass.getName.startsWith("akka.") && !serializedClass.getName.startsWith("java.lang.") && !suppressWarningOnNonSerializationVerification(serializedClass) diff --git a/akka-actor/src/main/scala/akka/serialization/Serializer.scala b/akka-actor/src/main/scala/akka/serialization/Serializer.scala index 595ca4084b..928d25a5b4 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serializer.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serializer.scala @@ -4,12 +4,17 @@ package akka.serialization * Copyright (C) 2009-2017 Lightbend Inc. */ -import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ByteArrayInputStream } +import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream } import java.nio.ByteBuffer import java.util.concurrent.Callable + import akka.util.ClassLoaderObjectInputStream import akka.actor.ExtendedActorSystem +import akka.annotation.InternalApi +import akka.event.{ LogMarker, Logging } + import scala.util.DynamicVariable +import scala.util.control.NoStackTrace /** * A Serializer represents a bimap between an object and an array of bytes representing that object. @@ -212,7 +217,7 @@ trait BaseSerializer extends Serializer { * where `FQCN` is fully qualified class name of the serializer implementation * and `ID` is globally unique serializer identifier number. */ - final val SerializationIdentifiers = "akka.actor.serialization-identifiers" + final val SerializationIdentifiers = BaseSerializer.SerializationIdentifiers /** * Globally unique serialization identifier configured in the `reference.conf`. @@ -224,8 +229,25 @@ trait BaseSerializer extends Serializer { /** * INTERNAL API */ + @InternalApi private[akka] def identifierFromConfig: Int = - system.settings.config.getInt(s"""${SerializationIdentifiers}."${getClass.getName}"""") + BaseSerializer.identifierFromConfig(getClass, system) +} +object BaseSerializer { + /** + * Configuration namespace of serialization identifiers in the `reference.conf`. + * + * Each serializer implementation must have an entry in the following format: + * `akka.actor.serialization-identifiers."FQCN" = ID` + * where `FQCN` is fully qualified class name of the serializer implementation + * and `ID` is globally unique serializer identifier number. + */ + final val SerializationIdentifiers = "akka.actor.serialization-identifiers" + + /** INTERNAL API */ + @InternalApi + private[akka] def identifierFromConfig(clazz: Class[_], system: ExtendedActorSystem): Int = + system.settings.config.getInt(s"""${SerializationIdentifiers}."${clazz.getName}"""") } /** @@ -280,6 +302,8 @@ object JavaSerializer { * This Serializer uses standard Java Serialization */ class JavaSerializer(val system: ExtendedActorSystem) extends BaseSerializer { + if (!system.settings.AllowJavaSerialization) + throw new DisabledJavaSerializer.JavaSerializationException("Attempted creation of `JavaSerializer` while `akka.actor.allow-java-serialization = off` was set!") def includeManifest: Boolean = false @@ -299,6 +323,50 @@ class JavaSerializer(val system: ExtendedActorSystem) extends BaseSerializer { } } +/** + * This Serializer is used when `akka.actor.java-serialization = off` + */ +final case class DisabledJavaSerializer(system: ExtendedActorSystem) extends Serializer with ByteBufferSerializer { + import DisabledJavaSerializer._ + + // use same identifier as JavaSerializer, since it's a replacement + override val identifier: Int = BaseSerializer.identifierFromConfig(classOf[JavaSerializer], system) + + private[this] val empty = Array.empty[Byte] + + private[this] val log = Logging.withMarker(system, getClass) + + def includeManifest: Boolean = false + + override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = { + log.warning(LogMarker.Security, "Outgoing message attempted to use Java Serialization even though `akka.actor.allow-java-serialization = off` was set! " + + "Message type was: [{}]", o.getClass) + throw IllegalSerialization + } + + override def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = { + log.warning(LogMarker.Security, "Incoming message attempted to use Java Serialization even though `akka.actor.allow-java-serialization = off` was set! " + + "Message class was: [{}]", clazz) + throw IllegalDeserialization + } + + override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = { + this.fromBinary(empty, None) + } + + override def toBinary(o: AnyRef): Array[Byte] = { + toBinary(o, null) + empty // won't return, toBinary throws + } + +} + +object DisabledJavaSerializer { + final class JavaSerializationException(msg: String) extends RuntimeException(msg) with NoStackTrace + final val IllegalSerialization = new JavaSerializationException("Attempted to serialize message using Java serialization while `akka.actor.allow-java-serialization` was disabled. Check WARNING logs for more details.") + final val IllegalDeserialization = new JavaSerializationException("Attempted to deserialize message using Java serialization while `akka.actor.allow-java-serialization` was disabled. Check WARNING logs for more details.") +} + /** * This is a special Serializer that Serializes and deserializes nulls only */ diff --git a/akka-docs/rst/java.rst b/akka-docs/rst/java.rst index 750bc3e8f5..27034e28ec 100644 --- a/akka-docs/rst/java.rst +++ b/akka-docs/rst/java.rst @@ -6,6 +6,7 @@ Java Documentation .. toctree:: :maxdepth: 2 + security/index intro/index-java general/index java/index-actors diff --git a/akka-docs/rst/java/remoting-artery.rst b/akka-docs/rst/java/remoting-artery.rst index c9046d1bd1..c51455c8b8 100644 --- a/akka-docs/rst/java/remoting-artery.rst +++ b/akka-docs/rst/java/remoting-artery.rst @@ -236,6 +236,37 @@ you can advise the system to create a child on that remote node like so: .. includecode:: code/docs/remoting/RemoteDeploymentDocTest.java#deploy +Remote deployment whitelist +^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +As remote deployment can potentially be abused by both users and even attackers a whitelist feature +is available to guard the ActorSystem from deploying unexpected actors. Please note that remote deployment +is *not* remote code loading, the Actors class to be deployed onto a remote system needs to be present on that +remote system. This still however may pose a security risk, and one may want to restrict remote deployment to +only a specific set of known actors by enabling the whitelist feature. + +To enable remote deployment whitelisting set the ``akka.remote.deployment.enable-whitelist`` value to ``on``. +The list of allowed classes has to be configured on the "remote" system, in other words on the system onto which +others will be attempting to remote deploy Actors. That system, locally, knows best which Actors it should or +should not allow others to remote deploy onto it. The full settings section may for example look like this: + +.. includecode:: ../../../akka-remote/src/test/scala/akka/remote/RemoteDeploymentWhitelistSpec.scala#whitelist-config + +Actor classes not included in the whitelist will not be allowed to be remote deployed onto this system. + +.. _remote-security-java-artery: + +Remote Security +--------------- + +An ``ActorSystem`` should not be exposed via Akka Remote (Artery) over plain Aeron/UDP to an untrusted network (e.g. internet). +It should be protected by network security, such as a firewall. There is currently no support for encryption with Artery +so if network security is not considered as enough protection the classic remoting with +:ref:`TLS and mutual authentication ` should be used. + +It is also security best-practice to :ref:`disable the Java serializer ` because of +its multiple `known attack surfaces `_. + Untrusted Mode ^^^^^^^^^^^^^^ @@ -255,6 +286,14 @@ a denial of service attack). :class:`PossiblyHarmful` covers the predefined messages like :class:`PoisonPill` and :class:`Kill`, but it can also be added as a marker trait to user-defined messages. +.. warning:: + + Untrusted mode does not give full protection against attacks by itself. + It makes it slightly harder to perform malicious or unintended actions but + it should be complemented with :ref:`disabled Java serializer `. + Additional protection can be achieved when running in an untrusted network by + network security (e.g. firewalls). + Messages sent with actor selection are by default discarded in untrusted mode, but permission to receive actor selection messages can be granted to specific actors defined in configuration:: @@ -441,24 +480,27 @@ Note that the array based methods can be implemented by delegation like this: .. includecode:: code/docs/actor/ByteBufferSerializerDocTest.java#bytebufserializer-with-manifest +.. _disable-java-serializer-java-artery: + Disabling the Java Serializer ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ It is possible to completely disable Java Serialization for the entire Actor system. -Java serialization is known to be slow and prone to attacks of various kinds - it never was designed for high -throughput messaging after all. However it is very convenient to use, thus it remained the default serialization -mechanism that Akka used to serialize user messages as well as some of its internal messages in previous versions. - -Akka internals do not rely on Java serialization (exceptions to that being ``java.lang.Throwable`` and "remote deployment"). +Java serialization is known to be slow and `prone to attacks +`_ +of various kinds - it never was designed for high throughput messaging after all. However, it is very +convenient to use, thus it remained the default serialization mechanism that Akka used to +serialize user messages as well as some of its internal messages in previous versions. +Since the release of Artery, Akka internals do not rely on Java serialization anymore (exceptions to that being ``java.lang.Throwable`` and "remote deployment"). .. note:: - Akka does not use Java Serialization for any of it's internal messages. + Akka does not use Java Serialization for any of its internal messages. It is highly encouraged to disable java serialization, so please plan to do so at the earliest possibility you have in your project. One may think that network bandwidth and latency limit the performance of remote messaging, but serialization is a more typical bottleneck. -For user messages the default serializer implemented using Java serialization remains available and enabled by default. +For user messages, the default serializer, implemented using Java serialization, remains available and enabled. We do however recommend to disable it entirely and utilise a proper serialization library instead in order effectively utilise the improved performance and ability for rolling deployments using Artery. Libraries that we recommend to use include, but are not limited to, `Kryo`_ by using the `akka-kryo-serialization`_ library or `Google Protocol Buffers`_ if you want @@ -469,13 +511,18 @@ your ``application.conf``: .. code-block:: ruby - akka { - actor { - serialization-bindings { - "java.io.Serializable" = none - } - } - } + akka.actor.allow-java-serialization = off + +This will completely disable the use of ``akka.serialization.JavaSerialization`` by the +Akka Serialization extension, instead ``DisabledJavaSerializer`` will +be inserted which will fail explicitly if attempts to use java serialization are made. + +The log messages emitted by such serializer SHOULD be be treated as potential +attacks which the serializer prevented, as they MAY indicate an external operator +attempting to send malicious messages intending to use java serialization as attack vector. +The attempts are logged with the SECURITY marker. + +Please note that this option does not stop you from manually invoking java serialization. Please note that this means that you will have to configure different serializers which will able to handle all of your remote messages. Please refer to the :ref:`serialization-java` documentation as well as :ref:`ByteBuffer based serialization ` to learn how to do this. diff --git a/akka-docs/rst/java/remoting.rst b/akka-docs/rst/java/remoting.rst index e10f861036..49d264734c 100644 --- a/akka-docs/rst/java/remoting.rst +++ b/akka-docs/rst/java/remoting.rst @@ -278,24 +278,33 @@ those actors are serializable. Failing to do so will cause the system to behave For more information please see :ref:`serialization-java`. +.. _disable-java-serializer-java: + Disabling the Java Serializer ----------------------------- -It is possible to completely disable Java Serialization for the entire Actor system. +Java serialization is known to be slow and `prone to attacks +`_ +of various kinds - it never was designed for high throughput messaging after all. However, it is very +convenient to use, thus it remained the default serialization mechanism that Akka used to +serialize user messages as well as some of its internal messages in previous versions. +Since the release of Artery, Akka internals do not rely on Java serialization anymore (one exception being ``java.lang.Throwable``). -Java serialization is known to be slow and prone to attacks of various kinds - it never was designed for high -throughput messaging after all. However it is very convenient to use, thus it remained the default serialization -mechanism that Akka used to serialize user messages as well as some of its internal messages in previous versions. - -Akka internals do not rely on Java serialization (exceptions to that being ``java.lang.Throwable`` and "remote deployment"). +.. warning:: + Please note Akka 2.5 by default does not use any Java Serialization for its own internal messages, unlike 2.4 where + by default it sill did for a few of the messages. If you want an 2.4.x system to communicate with a 2.5.x series, for + example during a rolling deployment you should first enable ``additional-serialization-bindings`` on the old systems. + You must do so on all nodes participating in a cluster, otherwise the mis-aligned serialization + configurations will cause deserialization errors on the receiving nodes. These additional serialization bindings are + enabled by default in Akka 2.5.x. .. note:: - Akka does not use Java Serialization for any of it's internal messages. + When using the new remoting implementation (codename Artery), Akka does not use Java Serialization for any of its internal messages. It is highly encouraged to disable java serialization, so please plan to do so at the earliest possibility you have in your project. One may think that network bandwidth and latency limit the performance of remote messaging, but serialization is a more typical bottleneck. - -For user messages the default serializer implemented using Java serialization remains available and enabled by default. + +For user messages, the default serializer, implemented using Java serialization, remains available and enabled. We do however recommend to disable it entirely and utilise a proper serialization library instead in order effectively utilise the improved performance and ability for rolling deployments using Artery. Libraries that we recommend to use include, but are not limited to, `Kryo`_ by using the `akka-kryo-serialization`_ library or `Google Protocol Buffers`_ if you want @@ -306,16 +315,21 @@ your ``application.conf``: .. code-block:: ruby - akka { - actor { - serialization-bindings { - "java.io.Serializable" = none - } - } - } + akka.actor.allow-java-serialization = off + +This will completely disable the use of ``akka.serialization.JavaSerialization`` by the +Akka Serialization extension, instead ``DisabledJavaSerializer`` will +be inserted which will fail explicitly if attempts to use java serialization are made. + +The log messages emitted by such serializer SHOULD be be treated as potential +attacks which the serializer prevented, as they MAY indicate an external operator +attempting to send malicious messages intending to use java serialization as attack vector. +The attempts are logged with the SECURITY marker. + +Please note that this option does not stop you from manually invoking java serialization. Please note that this means that you will have to configure different serializers which will able to handle all of your -remote messages. Please refer to the :ref:`serialization-scala` documentation as well as :ref:`ByteBuffer based serialization ` to learn how to do this. +remote messages. Please refer to the :ref:`serialization-java` documentation as well as :ref:`ByteBuffer based serialization ` to learn how to do this. .. _Kryo: https://github.com/EsotericSoftware/kryo .. _akka-kryo-serialization: https://github.com/romix/akka-kryo-serialization @@ -401,58 +415,19 @@ which includes the addresses of local and remote ActorSystems. To intercept generic remoting related errors, listen to ``RemotingErrorEvent`` which holds the ``Throwable`` cause. +.. _remote-security-java: + Remote Security ^^^^^^^^^^^^^^^ -Untrusted Mode --------------- +An ``ActorSystem`` should not be exposed via Akka Remote over plain TCP to an untrusted network (e.g. internet). +It should be protected by network security, such as a firewall. If that is not considered as enough protection +:ref:`TLS with mutual authentication ` should be enabled. -As soon as an actor system can connect to another remotely, it may in principle -send any possible message to any actor contained within that remote system. One -example may be sending a :class:`PoisonPill` to the system guardian, shutting -that system down. This is not always desired, and it can be disabled with the -following setting:: +It is also security best-practice to :ref:`disable the Java serializer ` because of +its multiple `known attack surfaces `_. - akka.remote.untrusted-mode = on - -This disallows sending of system messages (actor life-cycle commands, -DeathWatch, etc.) and any message extending :class:`PossiblyHarmful` to the -system on which this flag is set. Should a client send them nonetheless they -are dropped and logged (at DEBUG level in order to reduce the possibilities for -a denial of service attack). :class:`PossiblyHarmful` covers the predefined -messages like :class:`PoisonPill` and :class:`Kill`, but it can also be added -as a marker trait to user-defined messages. - -Messages sent with actor selection are by default discarded in untrusted mode, but -permission to receive actor selection messages can be granted to specific actors -defined in configuration:: - - akka.remote.trusted-selection-paths = ["/user/receptionist", "/user/namingService"] - -The actual message must still not be of type :class:`PossiblyHarmful`. - -In summary, the following operations are ignored by a system configured in -untrusted mode when incoming via the remoting layer: - -* remote deployment (which also means no remote supervision) -* remote DeathWatch -* ``system.stop()``, :class:`PoisonPill`, :class:`Kill` -* sending any message which extends from the :class:`PossiblyHarmful` marker - interface, which includes :class:`Terminated` -* messages sent with actor selection, unless destination defined in ``trusted-selection-paths``. - -.. note:: - - Enabling the untrusted mode does not remove the capability of the client to - freely choose the target of its message sends, which means that messages not - prohibited by the above rules can be sent to any actor in the remote system. - It is good practice for a client-facing system to only contain a well-defined - set of entry point actors, which then forward requests (possibly after - performing validation) to another actor system containing the actual worker - actors. If messaging between these two server-side systems is done using - local :class:`ActorRef` (they can be exchanged safely between actor systems - within the same JVM), you can restrict the messages on this interface by - marking them :class:`PossiblyHarmful` so that a client cannot forge them. +.. _remote-tls-java: Configuring SSL/TLS for Akka Remoting ------------------------------------- @@ -506,6 +481,13 @@ as well as the `Oracle documentation on creating KeyStore and TrustStores `. + Additional protection can be achieved when running in an untrusted network by + network security (e.g. firewalls) and/or enabling + :ref:`TLS with mutual authentication `. + +Messages sent with actor selection are by default discarded in untrusted mode, but +permission to receive actor selection messages can be granted to specific actors +defined in configuration:: + + akka.remote.trusted-selection-paths = ["/user/receptionist", "/user/namingService"] + +The actual message must still not be of type :class:`PossiblyHarmful`. + +In summary, the following operations are ignored by a system configured in +untrusted mode when incoming via the remoting layer: + +* remote deployment (which also means no remote supervision) +* remote DeathWatch +* ``system.stop()``, :class:`PoisonPill`, :class:`Kill` +* sending any message which extends from the :class:`PossiblyHarmful` marker + interface, which includes :class:`Terminated` +* messages sent with actor selection, unless destination defined in ``trusted-selection-paths``. + +.. note:: + + Enabling the untrusted mode does not remove the capability of the client to + freely choose the target of its message sends, which means that messages not + prohibited by the above rules can be sent to any actor in the remote system. + It is good practice for a client-facing system to only contain a well-defined + set of entry point actors, which then forward requests (possibly after + performing validation) to another actor system containing the actual worker + actors. If messaging between these two server-side systems is done using + local :class:`ActorRef` (they can be exchanged safely between actor systems + within the same JVM), you can restrict the messages on this interface by + marking them :class:`PossiblyHarmful` so that a client cannot forge them. + .. _remote-configuration-java: Remote Configuration diff --git a/akka-docs/rst/scala.rst b/akka-docs/rst/scala.rst index bce3187bef..e0ad6d5651 100644 --- a/akka-docs/rst/scala.rst +++ b/akka-docs/rst/scala.rst @@ -6,6 +6,7 @@ Scala Documentation .. toctree:: :maxdepth: 2 + security/index intro/index-scala general/index scala/index-actors diff --git a/akka-docs/rst/scala/remoting-artery.rst b/akka-docs/rst/scala/remoting-artery.rst index b9566f95c9..90f90ea2fc 100644 --- a/akka-docs/rst/scala/remoting-artery.rst +++ b/akka-docs/rst/scala/remoting-artery.rst @@ -236,6 +236,37 @@ you can advise the system to create a child on that remote node like so: .. includecode:: code/docs/remoting/RemoteDeploymentDocSpec.scala#deploy +Remote deployment whitelist +^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +As remote deployment can potentially be abused by both users and even attackers a whitelist feature +is available to guard the ActorSystem from deploying unexpected actors. Please note that remote deployment +is *not* remote code loading, the Actors class to be deployed onto a remote system needs to be present on that +remote system. This still however may pose a security risk, and one may want to restrict remote deployment to +only a specific set of known actors by enabling the whitelist feature. + +To enable remote deployment whitelisting set the ``akka.remote.deployment.enable-whitelist`` value to ``on``. +The list of allowed classes has to be configured on the "remote" system, in other words on the system onto which +others will be attempting to remote deploy Actors. That system, locally, knows best which Actors it should or +should not allow others to remote deploy onto it. The full settings section may for example look like this: + +.. includecode:: ../../../akka-remote/src/test/scala/akka/remote/RemoteDeploymentWhitelistSpec.scala#whitelist-config + +Actor classes not included in the whitelist will not be allowed to be remote deployed onto this system. + +.. _remote-security-scala-artery: + +Remote Security +--------------- + +An ``ActorSystem`` should not be exposed via Akka Remote (Artery) over plain Aeron/UDP to an untrusted network (e.g. internet). +It should be protected by network security, such as a firewall. There is currently no support for encryption with Artery +so if network security is not considered as enough protection the classic remoting with +:ref:`TLS and mutual authentication ` should be used. + +It is also security best-practice to :ref:`disable the Java serializer ` because of +its multiple `known attack surfaces `_. + Untrusted Mode ^^^^^^^^^^^^^^ @@ -255,6 +286,14 @@ a denial of service attack). :class:`PossiblyHarmful` covers the predefined messages like :class:`PoisonPill` and :class:`Kill`, but it can also be added as a marker trait to user-defined messages. +.. warning:: + + Untrusted mode does not give full protection against attacks by itself. + It makes it slightly harder to perform malicious or unintended actions but + it should be complemented with :ref:`disabled Java serializer `. + Additional protection can be achieved when running in an untrusted network by + network security (e.g. firewalls). + Messages sent with actor selection are by default discarded in untrusted mode, but permission to receive actor selection messages can be granted to specific actors defined in configuration:: @@ -441,24 +480,27 @@ Note that the array based methods can be implemented by delegation like this: .. includecode:: code/docs/actor/ByteBufferSerializerDocSpec.scala#bytebufserializer-with-manifest +.. _disable-java-serializer-scala-artery: + Disabling the Java Serializer ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ It is possible to completely disable Java Serialization for the entire Actor system. -Java serialization is known to be slow and prone to attacks of various kinds - it never was designed for high -throughput messaging after all. However it is very convenient to use, thus it remained the default serialization -mechanism that Akka used to serialize user messages as well as some of its internal messages in previous versions. - -Akka internals do not rely on Java serialization (exceptions to that being ``java.lang.Throwable`` and "remote deployment"). +Java serialization is known to be slow and `prone to attacks +`_ +of various kinds - it never was designed for high throughput messaging after all. However, it is very +convenient to use, thus it remained the default serialization mechanism that Akka used to +serialize user messages as well as some of its internal messages in previous versions. +Since the release of Artery, Akka internals do not rely on Java serialization anymore (exceptions to that being ``java.lang.Throwable`` and "remote deployment"). .. note:: - Akka does not use Java Serialization for any of it's internal messages. + Akka does not use Java Serialization for any of its internal messages. It is highly encouraged to disable java serialization, so please plan to do so at the earliest possibility you have in your project. One may think that network bandwidth and latency limit the performance of remote messaging, but serialization is a more typical bottleneck. -For user messages the default serializer implemented using Java serialization remains available and enabled by default. +For user messages, the default serializer, implemented using Java serialization, remains available and enabled. We do however recommend to disable it entirely and utilise a proper serialization library instead in order effectively utilise the improved performance and ability for rolling deployments using Artery. Libraries that we recommend to use include, but are not limited to, `Kryo`_ by using the `akka-kryo-serialization`_ library or `Google Protocol Buffers`_ if you want @@ -469,13 +511,20 @@ your ``application.conf``: .. code-block:: ruby - akka { - actor { - serialization-bindings { - "java.io.Serializable" = none - } - } - } + akka.actor.allow-java-serialization = off + +This will completely disable the use of ``akka.serialization.JavaSerialization`` by the +Akka Serialization extension, instead ``DisabledJavaSerializer`` will +be inserted which will fail explicitly if attempts to use java serialization are made. + +It will also enable the above mentioned `enable-additional-serialization-bindings`. + +The log messages emitted by such serializer SHOULD be be treated as potential +attacks which the serializer prevented, as they MAY indicate an external operator +attempting to send malicious messages intending to use java serialization as attack vector. +The attempts are logged with the SECURITY marker. + +Please note that this option does not stop you from manually invoking java serialization. Please note that this means that you will have to configure different serializers which will able to handle all of your remote messages. Please refer to the :ref:`serialization-scala` documentation as well as :ref:`ByteBuffer based serialization ` to learn how to do this. diff --git a/akka-docs/rst/scala/remoting.rst b/akka-docs/rst/scala/remoting.rst index 23c64941cd..4cdb48b6ad 100644 --- a/akka-docs/rst/scala/remoting.rst +++ b/akka-docs/rst/scala/remoting.rst @@ -284,24 +284,54 @@ those actors are serializable. Failing to do so will cause the system to behave For more information please see :ref:`serialization-scala`. +.. _disable-java-serializer-scala: + Disabling the Java Serializer ----------------------------- -It is possible to completely disable Java Serialization for the entire Actor system. +Since the ``2.4.11`` release of Akka it is possible to entirely disable the default Java Serialization mechanism. +Please note that :ref:`new remoting implementation (codename Artery) ` does not use Java +serialization for internal messages by default. For compatibility reasons, the current remoting still uses Java +serialization for some classes, however you can disable it in this remoting implementation as well by following +the steps below. -Java serialization is known to be slow and prone to attacks of various kinds - it never was designed for high -throughput messaging after all. However it is very convenient to use, thus it remained the default serialization -mechanism that Akka used to serialize user messages as well as some of its internal messages in previous versions. +The first step is to enable some additional serializers that replace previous Java serialization of some internal +messages. This is recommended also when you can't disable Java serialization completely. Those serializers are +enabled with this configuration: -Akka internals do not rely on Java serialization (exceptions to that being ``java.lang.Throwable`` and "remote deployment"). +.. code-block:: ruby + + akka.actor { + # Set this to on to enable serialization-bindings define in + # additional-serialization-bindings. Those are by default not included + # for backwards compatibility reasons. They are enabled by default if + # akka.remote.artery.enabled=on. + enable-additional-serialization-bindings = on + } + +The reason these are not enabled by default is wire-level compatibility between any 2.4.x Actor Systems. +If you roll out a new cluster, all on the same Akka version that can enable these serializers it is recommended to +enable this setting. When using :ref:`remoting-artery-scala` these serializers are enabled by default. + +.. warning:: + Please note that when enabling the additional-serialization-bindings when using the old remoting, + you must do so on all nodes participating in a cluster, otherwise the mis-aligned serialization + configurations will cause deserialization errors on the receiving nodes. + +Java serialization is known to be slow and `prone to attacks +`_ +of various kinds - it never was designed for high throughput messaging after all. However, it is very +convenient to use, thus it remained the default serialization mechanism that Akka used to +serialize user messages as well as some of its internal messages in previous versions. +Since the release of Artery, Akka internals do not rely on Java serialization anymore (one exception being ``java.lang.Throwable``). .. note:: - Akka does not use Java Serialization for any of it's internal messages. + When using the new remoting implementation (codename Artery), Akka does not use Java Serialization for any of its internal messages. It is highly encouraged to disable java serialization, so please plan to do so at the earliest possibility you have in your project. One may think that network bandwidth and latency limit the performance of remote messaging, but serialization is a more typical bottleneck. - -For user messages the default serializer implemented using Java serialization remains available and enabled by default. + +For user messages, the default serializer, implemented using Java serialization, remains available and enabled. We do however recommend to disable it entirely and utilise a proper serialization library instead in order effectively utilise the improved performance and ability for rolling deployments using Artery. Libraries that we recommend to use include, but are not limited to, `Kryo`_ by using the `akka-kryo-serialization`_ library or `Google Protocol Buffers`_ if you want @@ -312,13 +342,20 @@ your ``application.conf``: .. code-block:: ruby - akka { - actor { - serialization-bindings { - "java.io.Serializable" = none - } - } - } + akka.actor.allow-java-serialization = off + +This will completely disable the use of ``akka.serialization.JavaSerialization`` by the +Akka Serialization extension, instead ``DisabledJavaSerializer`` will +be inserted which will fail explicitly if attempts to use java serialization are made. + +It will also enable the above mentioned ``enable-additional-serialization-bindings``. + +The log messages emitted by such serializer SHOULD be be treated as potential +attacks which the serializer prevented, as they MAY indicate an external operator +attempting to send malicious messages intending to use java serialization as attack vector. +The attempts are logged with the SECURITY marker. + +Please note that this option does not stop you from manually invoking java serialization. Please note that this means that you will have to configure different serializers which will able to handle all of your remote messages. Please refer to the :ref:`serialization-scala` documentation as well as :ref:`ByteBuffer based serialization ` to learn how to do this. @@ -407,58 +444,19 @@ To be notified when the remoting subsystem has been shut down, listen to ``Remo To intercept generic remoting related errors, listen to ``RemotingErrorEvent`` which holds the ``Throwable`` cause. +.. _remote-security-scala: + Remote Security ^^^^^^^^^^^^^^^ -Untrusted Mode --------------- +An ``ActorSystem`` should not be exposed via Akka Remote over plain TCP to an untrusted network (e.g. internet). +It should be protected by network security, such as a firewall. If that is not considered as enough protection +:ref:`TLS with mutual authentication ` should be enabled. -As soon as an actor system can connect to another remotely, it may in principle -send any possible message to any actor contained within that remote system. One -example may be sending a :class:`PoisonPill` to the system guardian, shutting -that system down. This is not always desired, and it can be disabled with the -following setting:: +It is also security best-practice to :ref:`disable the Java serializer ` because of +its multiple `known attack surfaces `_. - akka.remote.untrusted-mode = on - -This disallows sending of system messages (actor life-cycle commands, -DeathWatch, etc.) and any message extending :class:`PossiblyHarmful` to the -system on which this flag is set. Should a client send them nonetheless they -are dropped and logged (at DEBUG level in order to reduce the possibilities for -a denial of service attack). :class:`PossiblyHarmful` covers the predefined -messages like :class:`PoisonPill` and :class:`Kill`, but it can also be added -as a marker trait to user-defined messages. - -Messages sent with actor selection are by default discarded in untrusted mode, but -permission to receive actor selection messages can be granted to specific actors -defined in configuration:: - - akka.remote.trusted-selection-paths = ["/user/receptionist", "/user/namingService"] - -The actual message must still not be of type :class:`PossiblyHarmful`. - -In summary, the following operations are ignored by a system configured in -untrusted mode when incoming via the remoting layer: - -* remote deployment (which also means no remote supervision) -* remote DeathWatch -* ``system.stop()``, :class:`PoisonPill`, :class:`Kill` -* sending any message which extends from the :class:`PossiblyHarmful` marker - interface, which includes :class:`Terminated` -* messages sent with actor selection, unless destination defined in ``trusted-selection-paths``. - -.. note:: - - Enabling the untrusted mode does not remove the capability of the client to - freely choose the target of its message sends, which means that messages not - prohibited by the above rules can be sent to any actor in the remote system. - It is good practice for a client-facing system to only contain a well-defined - set of entry point actors, which then forward requests (possibly after - performing validation) to another actor system containing the actual worker - actors. If messaging between these two server-side systems is done using - local :class:`ActorRef` (they can be exchanged safely between actor systems - within the same JVM), you can restrict the messages on this interface by - marking them :class:`PossiblyHarmful` so that a client cannot forge them. +.. _remote-tls-scala: Configuring SSL/TLS for Akka Remoting ------------------------------------- @@ -514,6 +512,13 @@ as well as the `Oracle documentation on creating KeyStore and TrustStores `. + Additional protection can be achieved when running in an untrusted network by + network security (e.g. firewalls) and/or enabling :ref:`TLS with mutual + authentication `. + +Messages sent with actor selection are by default discarded in untrusted mode, but +permission to receive actor selection messages can be granted to specific actors +defined in configuration:: + + akka.remote.trusted-selection-paths = ["/user/receptionist", "/user/namingService"] + +The actual message must still not be of type :class:`PossiblyHarmful`. + +In summary, the following operations are ignored by a system configured in +untrusted mode when incoming via the remoting layer: + +* remote deployment (which also means no remote supervision) +* remote DeathWatch +* ``system.stop()``, :class:`PoisonPill`, :class:`Kill` +* sending any message which extends from the :class:`PossiblyHarmful` marker + interface, which includes :class:`Terminated` +* messages sent with actor selection, unless destination defined in ``trusted-selection-paths``. + +.. note:: + + Enabling the untrusted mode does not remove the capability of the client to + freely choose the target of its message sends, which means that messages not + prohibited by the above rules can be sent to any actor in the remote system. + It is good practice for a client-facing system to only contain a well-defined + set of entry point actors, which then forward requests (possibly after + performing validation) to another actor system containing the actual worker + actors. If messaging between these two server-side systems is done using + local :class:`ActorRef` (they can be exchanged safely between actor systems + within the same JVM), you can restrict the messages on this interface by + marking them :class:`PossiblyHarmful` so that a client cannot forge them. + .. _remote-configuration-scala: Remote Configuration diff --git a/akka-docs/rst/security/2017-02-10-java-serialization.rst b/akka-docs/rst/security/2017-02-10-java-serialization.rst new file mode 100644 index 0000000000..c42ba54574 --- /dev/null +++ b/akka-docs/rst/security/2017-02-10-java-serialization.rst @@ -0,0 +1,61 @@ +Java Serialization, Fixed in Akka 2.4.17 +======================================== + +Date +---- + +10 Feburary 2017 + +Description of Vulnerability +~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +An attacker that can connect to an ``ActorSystem`` exposed via Akka Remote over TCP can gain remote code execution +capabilities in the context of the JVM process that runs the ActorSystem if: + +* ``JavaSerializer`` is enabled (default in Akka 2.4.x) +* and TLS is disabled *or* TLS is enabled with ``akka.remote.netty.ssl.security.require-mutual-authentication = false`` + (which is still the default in Akka 2.4.x) +* regardless of whether ``untrusted`` mode is enabled or not + +Java deserialization is `known to be vulnerable `_ to attacks when attacker can provide arbitrary types. + +Akka Remoting uses Java serialiser as default configuration which makes it vulnerable in its default form. The documentation of how to disable Java serializer was not complete. The documentation of how to enable mutual authentication was missing (only described in reference.conf). + +To protect against such attacks the system should be updated to Akka `2.4.17` or later and be configured with +:ref:`disabled Java serializer `. Additional protection can be achieved when running in an +untrusted network by enabling :ref:`TLS with mutual authentication `. + +Please subscribe to the `akka-security `_ mailing list to be notified promptly about future security issues. + +Severity +~~~~~~~~ + +The `CVSS `_ score of this vulnerability is 3.6 (Low), based on vector `AV:A/AC:H/Au:N/C:P/I:P/A:P/E:F/RL:OF/RC:C `_. + +Rationale for the score: + +* AV:A - Best practice is that Akka remoting nodes should only be accessible from the adjacent network, so in good setups, this will be adjacent. +* AC:H - In order to exploit, you first need to be able to connect to the Akka system. This will usually mean exploiting some other system that connects to it first. +* C:P, I:P, A:P - Partial impact for each of confidentiality, integrity and availability, due to the already high impact to these that being able to connect to a remote actor system in the first place has. + +Affected Versions +~~~~~~~~~~~~~~~~~ + +- Akka `2.4.16` and prior +- Akka `2.5-M1` (milestone not intended for production) + +Fixed Versions +~~~~~~~~~~~~~~ + +We have prepared patches for the affected versions, and have released the following versions which resolve the issue: + +- Akka `2.4.17` (Scala 2.11, 2.12) + +Binary and source compatibility has been maintained for the patched releases so the upgrade procedure is as simple as changing the library dependency. + +It will also be fixed in 2.5-M2 or 2.5.0-RC1. + +Acknowledgements +~~~~~~~~~~~~~~~~ + +We would like to thank Alvaro Munoz & Adrian Bravo for their thorough investigation and bringing this issue to our attention. \ No newline at end of file diff --git a/akka-docs/rst/security/index.rst b/akka-docs/rst/security/index.rst new file mode 100644 index 0000000000..aff893029f --- /dev/null +++ b/akka-docs/rst/security/index.rst @@ -0,0 +1,36 @@ +Security Announcements +====================== + +Receiving Security Advisories +----------------------------- + +The best way to receive any and all security announcements is to subscribe to the `Akka security list `_. + +The mailing list is very low traffic, and receives notifications only after security reports have been managed by the core team and fixes are publicly available. + +Reporting Vulnerabilities +------------------------- + +We strongly encourage people to report such problems to our private security mailing list first, before disclosing them in a public forum. + +Following best-practice, we strongly encourage anyone to report potential security +vulnerabilities to security@akka.io before disclosing them in a public forum like the mailing list or as a Github issue. + +Reports to this email address will be handled by our security team, who will work together with you +to ensure that a fix can be provided without delay. + +Security Related Documentation +------------------------------ + +* :ref:`disable-java-serializer-scala` +* :ref:`remote-deployment-whitelist-scala` +* :ref:`remote-security-scala` + + +Fixed Security Vulnerabilities +------------------------------ + +.. toctree:: + :maxdepth: 1 + + 2017-02-10-java-serialization \ No newline at end of file diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 3c3328702e..7c8e2b8fe2 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -53,7 +53,7 @@ akka { # Additional serialization-bindings that are replacing Java serialization are # defined in this section for backwards compatibility reasons. They are included - # by default but can be excluded for backwards compatibility with Akka 2.4.x. + # by default but can be excluded for backwards compatibility with Akka 2.4.x. # They can be disabled with enable-additional-serialization-bindings=off. additional-serialization-bindings { "akka.actor.Identify" = akka-misc @@ -620,7 +620,7 @@ akka { # "SHA1PRNG" => Can be slow because of blocking issues on Linux # "AES128CounterSecureRNG" => fastest startup and based on AES encryption # algorithm - # "AES256CounterSecureRNG" (Install JCE Unlimited Strength Jurisdiction + # "AES256CounterSecureRNG" (Install JCE Unlimited Strength Jurisdiction # Policy Files first) # # Setting a value here may require you to supply the appropriate cipher diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 311d93240b..a54ff4b139 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -630,9 +630,13 @@ private[remote] class Deserializer( push(out, envelopeWithMessage) } catch { case NonFatal(e) ⇒ + val from = envelope.association match { + case OptionVal.Some(a) ⇒ a.remoteAddress + case OptionVal.None ⇒ "unknown" + } log.warning( - "Failed to deserialize message with serializer id [{}] and manifest [{}]. {}", - envelope.serializer, envelope.classManifest, e.getMessage) + "Failed to deserialize message from [{}] with serializer id [{}] and manifest [{}]. {}", + from, envelope.serializer, envelope.classManifest, e.getMessage) pull(in) } finally { val buf = envelope.envelopeBuffer diff --git a/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala index ce7173a65c..84244642ac 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala @@ -61,7 +61,7 @@ class SerializationErrorSpec extends ArteryMultiNodeSpec(ArterySpecSupport.defau expectMsg("ping") EventFilter.warning( - start = "Failed to deserialize message with serializer id [4]", occurrences = 1).intercept { + pattern = """Failed to deserialize message from \[.*\] with serializer id \[4\]""", occurrences = 1).intercept { remoteRef ! "boom".getBytes("utf-8") }(systemB) diff --git a/akka-remote/src/test/scala/akka/remote/serialization/AllowJavaSerializationOffSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/AllowJavaSerializationOffSpec.scala new file mode 100644 index 0000000000..0dff15f267 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/serialization/AllowJavaSerializationOffSpec.scala @@ -0,0 +1,133 @@ +/* + * Copyright (C) 2016-2017 Lightbend Inc. + */ +package akka.remote.serialization + +import akka.actor.setup.ActorSystemSetup +import akka.actor.{ ActorSystem, BootstrapSetup, ExtendedActorSystem, Terminated } +import akka.serialization._ +import akka.testkit.{ AkkaSpec, TestKit, TestProbe } +import com.typesafe.config.ConfigFactory +import scala.concurrent.duration._ +import akka.actor.actorRef2Scala + +class ConfigurationDummy +class ProgrammaticDummy +case class ProgrammaticJavaDummy() +case class SerializableDummy() // since case classes are serializable + +object AllowJavaSerializationOffSpec { + + 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) + + val noJavaSerializationSystem = ActorSystem("AllowJavaSerializationOffSpec" + "NoJavaSerialization", ConfigFactory.parseString( + """ + akka { + actor { + allow-java-serialization = off + } + } + """.stripMargin)) + val noJavaSerializer = new DisabledJavaSerializer(noJavaSerializationSystem.asInstanceOf[ExtendedActorSystem]) + +} + +class AllowJavaSerializationOffSpec extends AkkaSpec( + ActorSystem("AllowJavaSerializationOffSpec", AllowJavaSerializationOffSpec.actorSystemSettings)) { + + import AllowJavaSerializationOffSpec._ + + // This is a weird edge case, someone creating a JavaSerializer manually and using it in a system means + // that they'd need a different actor system to be able to create it... someone MAY pick a system with + // allow-java-serialization=on to create the SerializationSetup and use that SerializationSetup + // in another system with allow-java-serialization=off + val addedJavaSerializationSettings = SerializationSetup { _ ⇒ + List( + SerializerDetails("test", testSerializer, List(classOf[ProgrammaticDummy])), + SerializerDetails("java-manual", new JavaSerializer(system.asInstanceOf[ExtendedActorSystem]), List(classOf[ProgrammaticJavaDummy]))) + } + val addedJavaSerializationProgramaticallyButDisabledSettings = BootstrapSetup(None, Some(ConfigFactory.parseString(""" + akka { + loglevel = debug + actor { + enable-additional-serialization-bindings = off # this should be overriden by the setting below, which should force it to be on + allow-java-serialization = off + } + } + """)), None) + + val dontAllowJavaSystem = + ActorSystem("addedJavaSerializationSystem", ActorSystemSetup(addedJavaSerializationProgramaticallyButDisabledSettings, addedJavaSerializationSettings)) + + "Disabling java serialization" should { + + "throw if passed system to JavaSerializer has allow-java-serialization = off" in { + intercept[DisabledJavaSerializer.JavaSerializationException] { + new JavaSerializer(noJavaSerializationSystem.asInstanceOf[ExtendedActorSystem]) + }.getMessage should include("akka.actor.allow-java-serialization = off") + + intercept[DisabledJavaSerializer.JavaSerializationException] { + SerializationExtension(dontAllowJavaSystem).findSerializerFor(new ProgrammaticJavaDummy).toBinary(new ProgrammaticJavaDummy) + } + } + + "enable additional-serialization-bindings" in { + val some = Some("foo") + val ser = SerializationExtension(dontAllowJavaSystem).findSerializerFor(some).asInstanceOf[MiscMessageSerializer] + val bytes = ser.toBinary(some) + ser.fromBinary(bytes, ser.manifest(some)) should ===(Some("foo")) + SerializationExtension(dontAllowJavaSystem).deserialize(bytes, ser.identifier, ser.manifest(some)) + .get should ===(Some("foo")) + } + + "have replaced java serializer" in { + val p = TestProbe()(dontAllowJavaSystem) // only receiver has the serialization disabled + + p.ref ! new ProgrammaticJavaDummy + SerializationExtension(system).findSerializerFor(new ProgrammaticJavaDummy).toBinary(new ProgrammaticJavaDummy) + // should not receive this one, it would have been java serialization! + p.expectNoMsg(100.millis) + + p.ref ! new ProgrammaticDummy + p.expectMsgType[ProgrammaticDummy] + } + + "disable java serialization also for incoming messages if serializer id usually would have found the serializer" in { + val ser1 = SerializationExtension(system) + val msg = SerializableDummy() + val bytes = ser1.serialize(msg).get + val serId = ser1.findSerializerFor(msg).identifier + ser1.findSerializerFor(msg).includeManifest should ===(false) + + val ser2 = SerializationExtension(noJavaSerializationSystem) + ser2.findSerializerFor(new SerializableDummy) should ===(noJavaSerializer) + ser2.serializerByIdentity(serId) should ===(noJavaSerializer) + intercept[DisabledJavaSerializer.JavaSerializationException] { + ser2.deserialize(bytes, serId, "").get + } + } + } + + override def afterTermination(): Unit = { + TestKit.shutdownActorSystem(noJavaSerializationSystem) + TestKit.shutdownActorSystem(dontAllowJavaSystem) + } + +}