=doc Make it easier to disable Java serialization, #22283 (#22315)

* one single config option to disable it completely
* improve security documentation

Co-authored-by: Patrik Nordwall <patrik.nordwall@gmail.com>
This commit is contained in:
Konrad `ktoso` Malawski 2017-02-16 14:09:04 +01:00 committed by GitHub
parent 552481b511
commit d306700feb
18 changed files with 843 additions and 191 deletions

View file

@ -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)
}
}

View file

@ -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 {
}

View file

@ -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

View file

@ -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")

View file

@ -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 systems [[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)

View file

@ -4,12 +4,17 @@ package akka.serialization
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
*/
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
*/

View file

@ -6,6 +6,7 @@ Java Documentation
.. toctree::
:maxdepth: 2
security/index
intro/index-java
general/index
java/index-actors

View file

@ -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 <remote-tls-java>` should be used.
It is also security best-practice to :ref:`disable the Java serializer <disable-java-serializer-java-artery>` because of
its multiple `known attack surfaces <https://community.hpe.com/t5/Security-Research/The-perils-of-Java-deserialization/ba-p/6838995>`_.
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 <disable-java-serializer-java-artery>`.
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
<https://community.hpe.com/t5/Security-Research/The-perils-of-Java-deserialization/ba-p/6838995>`_
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 <remote-bytebuffer-serialization-java>` to learn how to do this.

View file

@ -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
<https://community.hpe.com/t5/Security-Research/The-perils-of-Java-deserialization/ba-p/6838995>`_
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 <remote-bytebuffer-serialization-scala>` to learn how to do this.
remote messages. Please refer to the :ref:`serialization-java` documentation as well as :ref:`ByteBuffer based serialization <remote-bytebuffer-serialization-java>` 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 <remote-tls-java>` 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 <disable-java-serializer-java>` because of
its multiple `known attack surfaces <https://community.hpe.com/t5/Security-Research/The-perils-of-Java-deserialization/ba-p/6838995>`_.
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 <https
are both great resources to research when setting up security on the JVM. Please consult those resources when troubleshooting
and configuring SSL.
Since Akka 2.5.0 mutual authentication between TLS peers is enabled by default.
Mutual authentication means that the the passive side (the TLS server side) of a connection will also request and verify
a certificate from the connecting peer. Without this mode only the client side is requesting and verifying certificates.
While Akka is a peer-to-peer technology, each connection between nodes starts out from one side (the "client") towards
the other (the "server").
See also a description of the settings in the :ref:`remote-configuration-scala` section.
.. note::
@ -513,6 +495,65 @@ See also a description of the settings in the :ref:`remote-configuration-scala`
When using SHA1PRNG on Linux it's recommended specify ``-Djava.security.egd=file:/dev/urandom`` as argument
to the JVM to prevent blocking. It is NOT as secure because it reuses the seed.
Untrusted Mode
--------------
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::
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.
.. 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 <disable-java-serializer-java>`.
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 <remote-tls-java>`.
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

View file

@ -6,6 +6,7 @@ Scala Documentation
.. toctree::
:maxdepth: 2
security/index
intro/index-scala
general/index
scala/index-actors

View file

@ -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 <remote-tls-scala>` should be used.
It is also security best-practice to :ref:`disable the Java serializer <disable-java-serializer-java-artery>` because of
its multiple `known attack surfaces <https://community.hpe.com/t5/Security-Research/The-perils-of-Java-deserialization/ba-p/6838995>`_.
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 <disable-java-serializer-scala-artery>`.
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
<https://community.hpe.com/t5/Security-Research/The-perils-of-Java-deserialization/ba-p/6838995>`_
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 <remote-bytebuffer-serialization-scala>` to learn how to do this.

View file

@ -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) <remoting-artery-scala>` 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
<https://community.hpe.com/t5/Security-Research/The-perils-of-Java-deserialization/ba-p/6838995>`_
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 <remote-bytebuffer-serialization-scala>` 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 <remote-tls-scala>` 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 <disable-java-serializer-scala>` because of
its multiple `known attack surfaces <https://community.hpe.com/t5/Security-Research/The-perils-of-Java-deserialization/ba-p/6838995>`_.
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 <https
are both great resources to research when setting up security on the JVM. Please consult those resources when troubleshooting
and configuring SSL.
Since Akka 2.5.0 mutual authentication between TLS peers is enabled by default.
Mutual authentication means that the the passive side (the TLS server side) of a connection will also request and verify
a certificate from the connecting peer. Without this mode only the client side is requesting and verifying certificates.
While Akka is a peer-to-peer technology, each connection between nodes starts out from one side (the "client") towards
the other (the "server").
See also a description of the settings in the :ref:`remote-configuration-scala` section.
.. note::
@ -521,6 +526,65 @@ See also a description of the settings in the :ref:`remote-configuration-scala`
When using SHA1PRNG on Linux it's recommended specify ``-Djava.security.egd=file:/dev/urandom`` as argument
to the JVM to prevent blocking. It is NOT as secure because it reuses the seed.
Untrusted Mode
--------------
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::
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.
.. 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 <disable-java-serializer-scala>`.
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 <remote-tls-scala>`.
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

View file

@ -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 <https://community.hpe.com/t5/Security-Research/The-perils-of-Java-deserialization/ba-p/6838995>`_ 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 <disable-java-serializer-scala>`. Additional protection can be achieved when running in an
untrusted network by enabling :ref:`TLS with mutual authentication <remote-tls-scala>`.
Please subscribe to the `akka-security <https://groups.google.com/forum/#!forum/akka-security>`_ mailing list to be notified promptly about future security issues.
Severity
~~~~~~~~
The `CVSS <https://en.wikipedia.org/wiki/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 <https://nvd.nist.gov/cvss.cfm?calculator&version=2&vector=%28AV:A/AC:H/Au:N/C:P/I:P/A:P/E:F/RL:OF/RC:C%29>`_.
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.

View file

@ -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 <https://groups.google.com/forum/#!forum/akka-security>`_.
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

View file

@ -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

View file

@ -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

View file

@ -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)

View file

@ -0,0 +1,133 @@
/*
* Copyright (C) 2016-2017 Lightbend Inc. <http://www.lightbend.com>
*/
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)
}
}