+act,rem,str add SECURITY marker to logs (#21749)
* +act,rem,str add SECURITY marker to logs * Update TlsSpec.scala
This commit is contained in:
parent
9be7df1527
commit
92671e6d98
14 changed files with 132 additions and 66 deletions
|
|
@ -7,13 +7,15 @@ package akka.actor
|
|||
import scala.collection.immutable
|
||||
import akka.dispatch._
|
||||
import akka.dispatch.sysmsg._
|
||||
import java.lang.{ UnsupportedOperationException, IllegalStateException }
|
||||
import akka.serialization.{ Serialization, JavaSerializer }
|
||||
import akka.event.EventStream
|
||||
import java.lang.{ IllegalStateException, UnsupportedOperationException }
|
||||
|
||||
import akka.serialization.{ JavaSerializer, Serialization }
|
||||
import akka.event.{ EventStream, Logging, LoggingAdapter, MarkerLoggingAdapter }
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import akka.event.{ Logging, LoggingAdapter }
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
object ActorRef {
|
||||
|
|
@ -606,7 +608,7 @@ private[akka] class VirtualPathContainer(
|
|||
override val provider: ActorRefProvider,
|
||||
override val path: ActorPath,
|
||||
override val getParent: InternalActorRef,
|
||||
val log: LoggingAdapter) extends MinimalActorRef {
|
||||
val log: MarkerLoggingAdapter) extends MinimalActorRef {
|
||||
|
||||
private val children = new ConcurrentHashMap[String, InternalActorRef]
|
||||
|
||||
|
|
|
|||
|
|
@ -499,7 +499,7 @@ private[akka] class LocalActorRefProvider private[akka] (
|
|||
|
||||
override val rootPath: ActorPath = RootActorPath(Address("akka", _systemName))
|
||||
|
||||
private[akka] val log: LoggingAdapter = Logging(eventStream, getClass.getName + "(" + rootPath.address + ")")
|
||||
private[akka] val log: MarkerLoggingAdapter = Logging.withMarker(eventStream, getClass.getName + "(" + rootPath.address + ")")
|
||||
|
||||
override val deadLetters: InternalActorRef =
|
||||
_deadLetters.getOrElse((p: ActorPath) ⇒ new DeadLetterActorRef(this, p, eventStream)).apply(rootPath / "deadLetters")
|
||||
|
|
|
|||
|
|
@ -537,7 +537,7 @@ private[akka] class ActorSystemImpl(
|
|||
case _ ⇒
|
||||
if (settings.JvmExitOnFatalError) {
|
||||
try {
|
||||
log.error(cause, "Uncaught error from thread [{}] shutting down JVM since 'akka.jvm-exit-on-fatal-error' is enabled", thread.getName)
|
||||
markerLogging.error(LogMarker.Security, cause, "Uncaught error from thread [{}] shutting down JVM since 'akka.jvm-exit-on-fatal-error' is enabled", thread.getName)
|
||||
import System.err
|
||||
err.print("Uncaught error from thread [")
|
||||
err.print(thread.getName)
|
||||
|
|
@ -550,7 +550,7 @@ private[akka] class ActorSystemImpl(
|
|||
System.exit(-1)
|
||||
}
|
||||
} else {
|
||||
log.error(cause, "Uncaught fatal error from thread [{}] shutting down ActorSystem [{}]", thread.getName, name)
|
||||
markerLogging.error(LogMarker.Security, cause, "Uncaught fatal error from thread [{}] shutting down ActorSystem [{}]", thread.getName, name)
|
||||
terminate()
|
||||
}
|
||||
}
|
||||
|
|
@ -605,7 +605,8 @@ private[akka] class ActorSystemImpl(
|
|||
dynamicAccess.createInstanceFor[LoggingFilter](LoggingFilter, arguments).get
|
||||
}
|
||||
|
||||
val log: LoggingAdapter = new BusLogging(eventStream, getClass.getName + "(" + name + ")", this.getClass, logFilter)
|
||||
private[this] val markerLogging = new MarkerLoggingAdapter(eventStream, getClass.getName + "(" + name + ")", this.getClass, logFilter)
|
||||
val log: LoggingAdapter = markerLogging
|
||||
|
||||
val scheduler: Scheduler = createScheduler()
|
||||
|
||||
|
|
|
|||
|
|
@ -1398,6 +1398,8 @@ object LogMarker {
|
|||
case None ⇒ None
|
||||
}
|
||||
|
||||
private[akka] final val Security = apply("SECURITY")
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -1678,3 +1680,49 @@ object NoLogging extends LoggingAdapter {
|
|||
final protected override def notifyInfo(message: String): Unit = ()
|
||||
final protected override def notifyDebug(message: String): Unit = ()
|
||||
}
|
||||
/**
|
||||
* NoLogging is a MarkerLoggingAdapter that does absolutely nothing – no logging at all.
|
||||
*/
|
||||
object NoMarkerLogging extends MarkerLoggingAdapter(null, "source", classOf[String], null) {
|
||||
|
||||
/**
|
||||
* Java API to return the reference to NoLogging
|
||||
* @return The NoLogging instance
|
||||
*/
|
||||
def getInstance = this
|
||||
|
||||
final override def isErrorEnabled = false
|
||||
final override def isWarningEnabled = false
|
||||
final override def isInfoEnabled = false
|
||||
final override def isDebugEnabled = false
|
||||
|
||||
final protected override def notifyError(message: String): Unit = ()
|
||||
final protected override def notifyError(cause: Throwable, message: String): Unit = ()
|
||||
final protected override def notifyWarning(message: String): Unit = ()
|
||||
final protected override def notifyInfo(message: String): Unit = ()
|
||||
final protected override def notifyDebug(message: String): Unit = ()
|
||||
final override def error(marker: LogMarker, cause: Throwable, message: String): Unit = ()
|
||||
final override def error(marker: LogMarker, cause: Throwable, template: String, arg1: Any): Unit = ()
|
||||
final override def error(marker: LogMarker, cause: Throwable, template: String, arg1: Any, arg2: Any): Unit = ()
|
||||
final override def error(marker: LogMarker, cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = ()
|
||||
final override def error(marker: LogMarker, cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = ()
|
||||
final override def error(marker: LogMarker, message: String): Unit = ()
|
||||
final override def error(marker: LogMarker, template: String, arg1: Any): Unit = ()
|
||||
final override def error(marker: LogMarker, template: String, arg1: Any, arg2: Any): Unit = ()
|
||||
final override def error(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = ()
|
||||
final override def error(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = ()
|
||||
final override def warning(marker: LogMarker, message: String): Unit = ()
|
||||
final override def warning(marker: LogMarker, template: String, arg1: Any): Unit = ()
|
||||
final override def warning(marker: LogMarker, template: String, arg1: Any, arg2: Any): Unit = ()
|
||||
final override def warning(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = ()
|
||||
final override def warning(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = ()
|
||||
final override def info(marker: LogMarker, message: String): Unit = ()
|
||||
final override def info(marker: LogMarker, template: String, arg1: Any, arg2: Any): Unit = ()
|
||||
final override def info(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = ()
|
||||
final override def info(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = ()
|
||||
final override def debug(marker: LogMarker, message: String): Unit = ()
|
||||
final override def debug(marker: LogMarker, template: String, arg1: Any): Unit = ()
|
||||
final override def debug(marker: LogMarker, template: String, arg1: Any, arg2: Any): Unit = ()
|
||||
final override def debug(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = ()
|
||||
final override def debug(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = ()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,16 +6,19 @@ package akka.serialization
|
|||
|
||||
import com.typesafe.config.Config
|
||||
import akka.actor._
|
||||
import akka.event.Logging
|
||||
import akka.event.{ LogMarker, Logging }
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
import java.io.NotSerializableException
|
||||
import scala.util.{ Try, DynamicVariable, Failure }
|
||||
|
||||
import scala.util.{ DynamicVariable, Failure, Try }
|
||||
import scala.collection.immutable
|
||||
import scala.util.control.NonFatal
|
||||
import scala.util.Success
|
||||
import java.nio.ByteBuffer
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import java.util.NoSuchElementException
|
||||
|
||||
|
|
@ -94,7 +97,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
|||
import Serialization._
|
||||
|
||||
val settings = new Settings(system.settings.config)
|
||||
val log = Logging(system, getClass.getName)
|
||||
val log = Logging.withMarker(system, getClass.getName)
|
||||
private val manifestCache = new AtomicReference[Map[String, Option[Class[_]]]](Map.empty[String, Option[Class[_]]])
|
||||
|
||||
/**
|
||||
|
|
@ -223,13 +226,13 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
|||
throw new NotSerializableException("No configured serialization-bindings for class [%s]" format clazz.getName)
|
||||
case possibilities ⇒
|
||||
if (!unique(possibilities))
|
||||
log.warning("Multiple serializers found for " + clazz + ", choosing first: " + 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)) {
|
||||
log.warning("Using the default Java serializer for class [{}] which is not recommended because of " +
|
||||
log.warning(LogMarker.Security, "Using the default Java serializer for class [{}] which is not recommended because of " +
|
||||
"performance implications. Use another serializer or disable this warning using the setting " +
|
||||
"'akka.actor.warn-about-java-serializer-usage'", clazz.getName)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -8,24 +8,26 @@ import akka.actor.SupervisorStrategy._
|
|||
import akka.actor.Terminated
|
||||
import akka.actor._
|
||||
import akka.dispatch.sysmsg.SystemMessage
|
||||
import akka.event.{ Logging, LoggingAdapter }
|
||||
import akka.event.{ LogMarker, Logging, LoggingAdapter, MarkerLoggingAdapter }
|
||||
import akka.pattern.pipe
|
||||
import akka.remote.EndpointManager.{ ResendState, Link, Send }
|
||||
import akka.remote.EndpointWriter.{ StoppedReading, FlushAndStop }
|
||||
import akka.remote.EndpointManager.{ Link, ResendState, Send }
|
||||
import akka.remote.EndpointWriter.{ FlushAndStop, StoppedReading }
|
||||
import akka.remote.WireFormats.SerializedMessage
|
||||
import akka.remote.transport.AkkaPduCodec.Message
|
||||
import akka.remote.transport.AssociationHandle.{ DisassociateInfo, ActorHandleEventListener, Disassociated, InboundPayload }
|
||||
import akka.remote.transport.AssociationHandle.{ ActorHandleEventListener, DisassociateInfo, Disassociated, InboundPayload }
|
||||
import akka.remote.transport.Transport.InvalidAssociationException
|
||||
import akka.remote.transport._
|
||||
import akka.serialization.Serialization
|
||||
import akka.util.ByteString
|
||||
import akka.{ OnlyCauseStackTrace, AkkaException }
|
||||
import akka.{ AkkaException, OnlyCauseStackTrace }
|
||||
import java.io.NotSerializableException
|
||||
import java.util.concurrent.{ TimeUnit, TimeoutException, ConcurrentHashMap }
|
||||
import java.util.concurrent.{ ConcurrentHashMap, TimeUnit, TimeoutException }
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.concurrent.duration.{ Deadline }
|
||||
import scala.concurrent.duration.Deadline
|
||||
import scala.util.control.NonFatal
|
||||
import java.util.concurrent.locks.LockSupport
|
||||
|
||||
import scala.concurrent.Future
|
||||
import akka.util.OptionVal
|
||||
import akka.util.OptionVal
|
||||
|
|
@ -47,7 +49,7 @@ private[remote] trait InboundMessageDispatcher {
|
|||
private[remote] class DefaultMessageDispatcher(
|
||||
private val system: ExtendedActorSystem,
|
||||
private val provider: RemoteActorRefProvider,
|
||||
private val log: LoggingAdapter) extends InboundMessageDispatcher {
|
||||
private val log: MarkerLoggingAdapter) extends InboundMessageDispatcher {
|
||||
|
||||
private val remoteDaemon = provider.remoteDaemon
|
||||
|
||||
|
|
@ -69,7 +71,7 @@ private[remote] class DefaultMessageDispatcher(
|
|||
recipient match {
|
||||
|
||||
case `remoteDaemon` ⇒
|
||||
if (UntrustedMode) log.debug("dropping daemon message in untrusted mode")
|
||||
if (UntrustedMode) log.debug(LogMarker.Security, "dropping daemon message in untrusted mode")
|
||||
else {
|
||||
if (LogReceive) log.debug("received daemon message {}", msgLog)
|
||||
remoteDaemon ! payload
|
||||
|
|
@ -82,6 +84,7 @@ private[remote] class DefaultMessageDispatcher(
|
|||
if (UntrustedMode && (!TrustedSelectionPaths.contains(sel.elements.mkString("/", "/", "")) ||
|
||||
sel.msg.isInstanceOf[PossiblyHarmful] || l != provider.rootGuardian))
|
||||
log.debug(
|
||||
LogMarker.Security,
|
||||
"operating in UntrustedMode, dropping inbound actor selection to [{}], " +
|
||||
"allow it by adding the path to 'akka.remote.trusted-selection-paths' configuration",
|
||||
sel.elements.mkString("/", "/", ""))
|
||||
|
|
@ -89,7 +92,7 @@ private[remote] class DefaultMessageDispatcher(
|
|||
// run the receive logic for ActorSelectionMessage here to make sure it is not stuck on busy user actor
|
||||
ActorSelection.deliverSelection(l, sender, sel)
|
||||
case msg: PossiblyHarmful if UntrustedMode ⇒
|
||||
log.debug("operating in UntrustedMode, dropping inbound PossiblyHarmful message of type [{}]", msg.getClass.getName)
|
||||
log.debug(LogMarker.Security, "operating in UntrustedMode, dropping inbound PossiblyHarmful message of type [{}]", msg.getClass.getName)
|
||||
case msg: SystemMessage ⇒ l.sendSystemMessage(msg)
|
||||
case msg ⇒ l.!(msg)(sender)
|
||||
}
|
||||
|
|
@ -532,6 +535,7 @@ private[remote] class EndpointWriter(
|
|||
import EndpointWriter._
|
||||
import context.dispatcher
|
||||
|
||||
private val markLog = Logging.withMarker(this)
|
||||
val extendedSystem: ExtendedActorSystem = context.system.asInstanceOf[ExtendedActorSystem]
|
||||
val remoteMetrics = RemoteMetricsExtension(extendedSystem)
|
||||
val backoffDispatcher = context.system.dispatchers.lookup("akka.remote.backoff-remote-dispatcher")
|
||||
|
|
@ -550,7 +554,7 @@ private[remote] class EndpointWriter(
|
|||
}
|
||||
|
||||
val provider = RARP(extendedSystem).provider
|
||||
val msgDispatch = new DefaultMessageDispatcher(extendedSystem, provider, log)
|
||||
val msgDispatch = new DefaultMessageDispatcher(extendedSystem, provider, markLog)
|
||||
|
||||
val inbound = handle.isDefined
|
||||
var stopReason: DisassociateInfo = AssociationHandle.Unknown
|
||||
|
|
|
|||
|
|
@ -195,7 +195,7 @@ private[akka] class RemoteActorRefProvider(
|
|||
local.rootPath / "remote",
|
||||
rootGuardian,
|
||||
remotingTerminator,
|
||||
log,
|
||||
_log,
|
||||
untrustedMode = remoteSettings.UntrustedMode)
|
||||
local.registerExtraNames(Map(("remote", d)))
|
||||
d
|
||||
|
|
@ -206,7 +206,7 @@ private[akka] class RemoteActorRefProvider(
|
|||
_internals = internals
|
||||
remotingTerminator ! internals
|
||||
|
||||
_log = Logging(eventStream, getClass.getName)
|
||||
_log = Logging.withMarker(eventStream, getClass.getName)
|
||||
|
||||
// this enables reception of remote requests
|
||||
transport.start()
|
||||
|
|
|
|||
|
|
@ -4,14 +4,11 @@
|
|||
|
||||
package akka.remote
|
||||
|
||||
import java.util
|
||||
import java.util.Collections
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import scala.annotation.tailrec
|
||||
import scala.util.control.NonFatal
|
||||
import akka.actor.{ Actor, ActorPath, ActorPathExtractor, ActorRef, ActorSystemImpl, AddressTerminated, Deploy, InternalActorRef, Nobody, Props, VirtualPathContainer }
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.event.{ AddressTerminatedTopic, LogMarker, LoggingAdapter, MarkerLoggingAdapter }
|
||||
import akka.dispatch.sysmsg.{ DeathWatchNotification, SystemMessage, Watch }
|
||||
import akka.actor.ActorRefWithCell
|
||||
import akka.actor.ActorRefScope
|
||||
|
|
@ -23,13 +20,11 @@ import akka.actor.SelectChildPattern
|
|||
import akka.actor.Identify
|
||||
import akka.actor.ActorIdentity
|
||||
import akka.actor.EmptyLocalActorRef
|
||||
import akka.event.AddressTerminatedTopic
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import scala.collection.immutable
|
||||
|
||||
import scala.collection.immutable
|
||||
import akka.dispatch.sysmsg.Unwatch
|
||||
import akka.NotUsed
|
||||
import com.typesafe.config.Config
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -54,7 +49,7 @@ private[akka] class RemoteSystemDaemon(
|
|||
_path: ActorPath,
|
||||
_parent: InternalActorRef,
|
||||
terminator: ActorRef,
|
||||
_log: LoggingAdapter,
|
||||
_log: MarkerLoggingAdapter,
|
||||
val untrustedMode: Boolean)
|
||||
extends VirtualPathContainer(system.provider, _path, _parent, _log) {
|
||||
|
||||
|
|
@ -163,8 +158,9 @@ private[akka] class RemoteSystemDaemon(
|
|||
doCreateActor(message, props, deploy, path, supervisor)
|
||||
else {
|
||||
val ex = new NotWhitelistedClassRemoteDeploymentAttemptException(props.actorClass, remoteDeploymentWhitelist)
|
||||
log.error(ex, "Received command to create remote Actor, but class [{}] is not white-listed! " +
|
||||
"Target path: [{}]", props.actorClass, path) // TODO add security marker?
|
||||
log.error(LogMarker.Security, ex,
|
||||
"Received command to create remote Actor, but class [{}] is not white-listed! " +
|
||||
"Target path: [{}]", props.actorClass, path)
|
||||
}
|
||||
case DaemonMsgCreate(props, deploy, path, supervisor) ⇒
|
||||
doCreateActor(message, props, deploy, path, supervisor)
|
||||
|
|
@ -211,7 +207,7 @@ private[akka] class RemoteSystemDaemon(
|
|||
case _ ⇒ // skip, this child doesn't belong to the terminated address
|
||||
}
|
||||
|
||||
case unknown ⇒ log.warning("Unknown message [{}] received by [{}]", unknown, this)
|
||||
case unknown ⇒ log.warning(LogMarker.Security, "Unknown message [{}] received by [{}]", unknown, this)
|
||||
|
||||
} catch {
|
||||
case NonFatal(e) ⇒ log.error(e, "exception while processing remote command [{}] from [{}]", msg, sender)
|
||||
|
|
|
|||
|
|
@ -13,11 +13,10 @@ import akka.actor.LocalRef
|
|||
import akka.actor.PossiblyHarmful
|
||||
import akka.actor.RepointableRef
|
||||
import akka.dispatch.sysmsg.SystemMessage
|
||||
import akka.event.Logging
|
||||
import akka.event.{ LogMarker, Logging, LoggingReceive }
|
||||
import akka.remote.RemoteActorRefProvider
|
||||
import akka.remote.RemoteRef
|
||||
import akka.util.OptionVal
|
||||
import akka.event.LoggingReceive
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -27,7 +26,7 @@ private[remote] class MessageDispatcher(
|
|||
provider: RemoteActorRefProvider) {
|
||||
|
||||
private val remoteDaemon = provider.remoteDaemon
|
||||
private val log = Logging(system, getClass.getName)
|
||||
private val log = Logging.withMarker(system, getClass.getName)
|
||||
private val debugLogEnabled = log.isDebugEnabled
|
||||
|
||||
def dispatch(
|
||||
|
|
@ -47,6 +46,7 @@ private[remote] class MessageDispatcher(
|
|||
case `remoteDaemon` ⇒
|
||||
if (UntrustedMode) {
|
||||
if (debugLogEnabled) log.debug(
|
||||
LogMarker.Security,
|
||||
"dropping daemon message [{}] in untrusted mode",
|
||||
messageClassName(message))
|
||||
} else {
|
||||
|
|
@ -65,6 +65,7 @@ private[remote] class MessageDispatcher(
|
|||
if (UntrustedMode && (!TrustedSelectionPaths.contains(sel.elements.mkString("/", "/", "")) ||
|
||||
sel.msg.isInstanceOf[PossiblyHarmful] || l != provider.rootGuardian)) {
|
||||
if (debugLogEnabled) log.debug(
|
||||
LogMarker.Security,
|
||||
"operating in UntrustedMode, dropping inbound actor selection to [{}], " +
|
||||
"allow it by adding the path to 'akka.remote.trusted-selection-paths' configuration",
|
||||
sel.elements.mkString("/", "/", ""))
|
||||
|
|
@ -73,6 +74,7 @@ private[remote] class MessageDispatcher(
|
|||
ActorSelection.deliverSelection(l, sender, sel)
|
||||
case msg: PossiblyHarmful if UntrustedMode ⇒
|
||||
if (debugLogEnabled) log.debug(
|
||||
LogMarker.Security,
|
||||
"operating in UntrustedMode, dropping inbound PossiblyHarmful message of type [{}] to [{}] from [{}]",
|
||||
messageClassName(msg), recipient, senderOption.getOrElse(originAddress.getOrElse("")))
|
||||
case msg: SystemMessage ⇒ l.sendSystemMessage(msg)
|
||||
|
|
|
|||
|
|
@ -4,6 +4,7 @@
|
|||
package akka.remote.transport
|
||||
|
||||
import java.util.concurrent.TimeoutException
|
||||
|
||||
import akka.actor.SupervisorStrategy.Stop
|
||||
import akka.actor._
|
||||
import akka.pattern.pipe
|
||||
|
|
@ -16,13 +17,15 @@ import akka.remote.transport.ProtocolStateActor._
|
|||
import akka.remote.transport.Transport._
|
||||
import akka.util.ByteString
|
||||
import akka.util.Helpers.Requiring
|
||||
import akka.{ OnlyCauseStackTrace, AkkaException }
|
||||
import akka.{ AkkaException, OnlyCauseStackTrace }
|
||||
import com.typesafe.config.Config
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.{ Future, Promise }
|
||||
import scala.util.control.NonFatal
|
||||
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
|
||||
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
|
||||
import akka.event.{ LogMarker, Logging }
|
||||
|
||||
@SerialVersionUID(1L)
|
||||
class AkkaProtocolException(msg: String, cause: Throwable) extends AkkaException(msg, cause) with OnlyCauseStackTrace {
|
||||
|
|
@ -290,6 +293,8 @@ private[transport] class ProtocolStateActor(
|
|||
extends Actor with FSM[AssociationState, ProtocolStateData]
|
||||
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
|
||||
|
||||
private val markerLog = Logging.withMarker(this)
|
||||
|
||||
import ProtocolStateActor._
|
||||
import context.dispatcher
|
||||
|
||||
|
|
@ -421,7 +426,7 @@ private[transport] class ProtocolStateActor(
|
|||
s"Association attempt with mismatching cookie from [{}]. Expected [{}] but received [{}].",
|
||||
info.origin, localHandshakeInfo.cookie.getOrElse(""), info.cookie.getOrElse(""))
|
||||
else
|
||||
log.warning(s"Association attempt with mismatching cookie from [{}].", info.origin)
|
||||
markerLog.warning(LogMarker.Security, s"Association attempt with mismatching cookie from [{}].", info.origin)
|
||||
stop()
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -5,15 +5,17 @@
|
|||
package akka.remote.transport.netty
|
||||
|
||||
import akka.ConfigurationException
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.event.{ LogMarker, LoggingAdapter, MarkerLoggingAdapter }
|
||||
import akka.japi.Util._
|
||||
import akka.remote.RemoteTransportException
|
||||
import akka.remote.security.provider.AkkaProvider
|
||||
import com.typesafe.config.Config
|
||||
import java.io.{ IOException, FileNotFoundException, FileInputStream }
|
||||
import java.io.{ FileInputStream, FileNotFoundException, IOException }
|
||||
import java.security._
|
||||
import javax.net.ssl.{ KeyManagerFactory, TrustManager, TrustManagerFactory, SSLContext }
|
||||
import javax.net.ssl.{ KeyManagerFactory, SSLContext, TrustManager, TrustManagerFactory }
|
||||
|
||||
import org.jboss.netty.handler.ssl.SslHandler
|
||||
|
||||
import scala.util.Try
|
||||
|
||||
/**
|
||||
|
|
@ -48,20 +50,20 @@ private[akka] object NettySSLSupport {
|
|||
/**
|
||||
* Construct a SSLHandler which can be inserted into a Netty server/client pipeline
|
||||
*/
|
||||
def apply(settings: SSLSettings, log: LoggingAdapter, isClient: Boolean): SslHandler =
|
||||
def apply(settings: SSLSettings, log: MarkerLoggingAdapter, isClient: Boolean): SslHandler =
|
||||
if (isClient) initializeClientSSL(settings, log) else initializeServerSSL(settings, log)
|
||||
|
||||
def initializeCustomSecureRandom(rngName: String, log: LoggingAdapter): SecureRandom = {
|
||||
def initializeCustomSecureRandom(rngName: String, log: MarkerLoggingAdapter): SecureRandom = {
|
||||
val rng = rngName match {
|
||||
case r @ ("AES128CounterSecureRNG" | "AES256CounterSecureRNG") ⇒
|
||||
log.debug("SSL random number generator set to: {}", r)
|
||||
SecureRandom.getInstance(r, AkkaProvider)
|
||||
case r @ ("AES128CounterInetRNG" | "AES256CounterInetRNG") ⇒
|
||||
log.warning("SSL random number generator {} is deprecated, " +
|
||||
log.warning(LogMarker.Security, "SSL random number generator {} is deprecated, " +
|
||||
"use AES128CounterSecureRNG or AES256CounterSecureRNG instead", r)
|
||||
SecureRandom.getInstance(r, AkkaProvider)
|
||||
case s @ ("SHA1PRNG" | "NativePRNG") ⇒
|
||||
log.debug("SSL random number generator set to: " + s)
|
||||
log.debug("SSL random number generator set to: {}", s)
|
||||
// SHA1PRNG needs /dev/urandom to be the source on Linux to prevent problems with /dev/random blocking
|
||||
// However, this also makes the seed source insecure as the seed is reused to avoid blocking (not a problem on FreeBSD).
|
||||
SecureRandom.getInstance(s)
|
||||
|
|
@ -71,17 +73,17 @@ private[akka] object NettySSLSupport {
|
|||
new SecureRandom
|
||||
|
||||
case unknown ⇒
|
||||
log.warning("Unknown SSLRandomNumberGenerator [{}] falling back to SecureRandom", unknown)
|
||||
log.warning(LogMarker.Security, "Unknown SSLRandomNumberGenerator [{}] falling back to SecureRandom", unknown)
|
||||
new SecureRandom
|
||||
}
|
||||
rng.nextInt() // prevent stall on first access
|
||||
rng
|
||||
}
|
||||
|
||||
def initializeClientSSL(settings: SSLSettings, log: LoggingAdapter): SslHandler = {
|
||||
def initializeClientSSL(settings: SSLSettings, log: MarkerLoggingAdapter): SslHandler = {
|
||||
log.debug("Client SSL is enabled, initialising ...")
|
||||
|
||||
def constructClientContext(settings: SSLSettings, log: LoggingAdapter, trustStorePath: String, trustStorePassword: String, protocol: String): Option[SSLContext] =
|
||||
def constructClientContext(settings: SSLSettings, log: MarkerLoggingAdapter, trustStorePath: String, trustStorePassword: String, protocol: String): Option[SSLContext] =
|
||||
try {
|
||||
val rng = initializeCustomSecureRandom(settings.SSLRandomNumberGenerator, log)
|
||||
val trustManagers: Array[TrustManager] = {
|
||||
|
|
@ -114,17 +116,16 @@ private[akka] object NettySSLSupport {
|
|||
case None ⇒
|
||||
throw new GeneralSecurityException(
|
||||
"""Failed to initialize client SSL because SSL context could not be found." +
|
||||
"Make sure your settings are correct: [trust-store: %s] [trust-store-password: %s] [protocol: %s]""".format(
|
||||
"Make sure your settings are correct: [trust-store: %s] [protocol: %s]""".format(
|
||||
settings.SSLTrustStore,
|
||||
settings.SSLTrustStorePassword,
|
||||
settings.SSLProtocol))
|
||||
}
|
||||
}
|
||||
|
||||
def initializeServerSSL(settings: SSLSettings, log: LoggingAdapter): SslHandler = {
|
||||
def initializeServerSSL(settings: SSLSettings, log: MarkerLoggingAdapter): SslHandler = {
|
||||
log.debug("Server SSL is enabled, initialising ...")
|
||||
|
||||
def constructServerContext(settings: SSLSettings, log: LoggingAdapter, keyStorePath: String, keyStorePassword: String, keyPassword: String, protocol: String): Option[SSLContext] =
|
||||
def constructServerContext(settings: SSLSettings, log: MarkerLoggingAdapter, keyStorePath: String, keyStorePassword: String, keyPassword: String, protocol: String): Option[SSLContext] =
|
||||
try {
|
||||
val rng = initializeCustomSecureRandom(settings.SSLRandomNumberGenerator, log)
|
||||
val factory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm)
|
||||
|
|
@ -162,7 +163,7 @@ private[akka] object NettySSLSupport {
|
|||
new SslHandler(sslEngine)
|
||||
case None ⇒ throw new GeneralSecurityException(
|
||||
"""Failed to initialize server SSL because SSL context could not be found.
|
||||
Make sure your settings are correct: [key-store: %s] [key-store-password: %s] [protocol: %s]""".format(
|
||||
Make sure your settings are correct: [key-store: %s] [key-store-password: %s] [protocol: %s]""".format(
|
||||
settings.SSLKeyStore,
|
||||
settings.SSLKeyStorePassword,
|
||||
settings.SSLProtocol))
|
||||
|
|
|
|||
|
|
@ -280,7 +280,7 @@ class NettyTransport(val settings: NettyTransportSettings, val system: ExtendedA
|
|||
@volatile private var boundTo: Address = _
|
||||
@volatile private var serverChannel: Channel = _
|
||||
|
||||
private val log = Logging(system, this.getClass)
|
||||
private val log = Logging.withMarker(system, this.getClass)
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
|
|||
|
|
@ -6,17 +6,21 @@ package akka.remote
|
|||
import akka.testkit._
|
||||
import akka.actor._
|
||||
import com.typesafe.config._
|
||||
|
||||
import scala.concurrent.Future
|
||||
import scala.reflect.classTag
|
||||
import akka.pattern.ask
|
||||
import java.security.{ NoSuchAlgorithmException }
|
||||
import java.security.NoSuchAlgorithmException
|
||||
|
||||
import akka.util.Timeout
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import akka.event.{ NoLogging }
|
||||
import akka.remote.transport.netty.{ SSLSettings, NettySSLSupport }
|
||||
import akka.event.{ NoLogging, NoMarkerLogging }
|
||||
import akka.remote.transport.netty.{ NettySSLSupport, SSLSettings }
|
||||
import Configuration.{ CipherConfig, getCipherConfig }
|
||||
import org.uncommons.maths.random.RandomDotOrgSeedGenerator
|
||||
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
object Configuration {
|
||||
|
|
@ -63,13 +67,13 @@ object Configuration {
|
|||
val fullConfig = config.withFallback(AkkaSpec.testConf).withFallback(ConfigFactory.load).getConfig("akka.remote.netty.ssl.security")
|
||||
val settings = new SSLSettings(fullConfig)
|
||||
|
||||
val rng = NettySSLSupport.initializeCustomSecureRandom(settings.SSLRandomNumberGenerator, NoLogging)
|
||||
val rng = NettySSLSupport.initializeCustomSecureRandom(settings.SSLRandomNumberGenerator, NoMarkerLogging)
|
||||
|
||||
rng.nextInt() // Has to work
|
||||
val sRng = settings.SSLRandomNumberGenerator
|
||||
rng.getAlgorithm == sRng || (throw new NoSuchAlgorithmException(sRng))
|
||||
|
||||
val engine = NettySSLSupport.initializeClientSSL(settings, NoLogging).getEngine
|
||||
val engine = NettySSLSupport.initializeClientSSL(settings, NoMarkerLogging).getEngine
|
||||
val gotAllSupported = enabled.toSet diff engine.getSupportedCipherSuites.toSet
|
||||
val gotAllEnabled = enabled.toSet diff engine.getEnabledCipherSuites.toSet
|
||||
gotAllSupported.isEmpty || (throw new IllegalArgumentException("Cipher Suite not supported: " + gotAllSupported))
|
||||
|
|
|
|||
|
|
@ -196,7 +196,7 @@ private[typed] object FutureRef {
|
|||
// On Scala 2.12, the field's name is exactly "_target" (and it's private), earlier Scala versions compile the val to a public field that's name mangled to "akka$typed$internal$FutureRef$$_target"
|
||||
val targetField = fields.find(_.getName.endsWith("_target"))
|
||||
assert(targetField.nonEmpty, s"Could not find _target field in FutureRef class among fields $fields.")
|
||||
|
||||
|
||||
unsafe.objectFieldOffset(targetField.get)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue