=rem Streamline Artery encoding of metadata to avoid allocations
* Change RemoteInstrument API to work with ByteBuffers * Remove the MetadataMap since serialization is done in buffer
This commit is contained in:
parent
913c166f88
commit
8d05592e2e
14 changed files with 609 additions and 589 deletions
|
|
@ -5,7 +5,11 @@
|
|||
package akka.remote.artery
|
||||
|
||||
import akka.actor.{ ActorRef, ExtendedActorSystem }
|
||||
import akka.util.{ ByteString, OptionVal }
|
||||
import akka.event.{ Logging, LoggingAdapter }
|
||||
import akka.util.OptionVal
|
||||
import java.nio.ByteBuffer
|
||||
import scala.annotation.tailrec
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -16,48 +20,274 @@ import akka.util.{ ByteString, OptionVal }
|
|||
* Multiple instruments are automatically handled, however they MUST NOT overlap in their idenfitiers.
|
||||
*
|
||||
* Instances of `RemoteInstrument` are created from configuration. A new instance of RemoteInstrument
|
||||
* will be created for each encoder and decoder. It's only called from the stage, so if it dosn't
|
||||
* will be created for each encoder and decoder. It's only called from the stage, so if it doesn't
|
||||
* delegate to any shared instance it doesn't have to be thread-safe.
|
||||
*/
|
||||
abstract class RemoteInstrument {
|
||||
/**
|
||||
* Instrument identifier.
|
||||
*
|
||||
* MUST be >=0 and <32.
|
||||
* MUST be >=1 and <32.
|
||||
*
|
||||
* Values between 0 and 7 are reserved for Akka internal use.
|
||||
* Values between 1 and 7 are reserved for Akka internal use.
|
||||
*/
|
||||
def identifier: Byte
|
||||
|
||||
/**
|
||||
* Called right before putting the message onto the wire.
|
||||
* Parameters MAY be `null` (except `message`)!
|
||||
*
|
||||
* @return `metadata` rendered to be serialized into the remove envelope, or `null` if no metadata should be attached
|
||||
* Should the serialization be timed? Otherwise times are always 0.
|
||||
*/
|
||||
def remoteMessageSent(recipient: ActorRef, message: Object, sender: ActorRef): ByteString
|
||||
def serializationTimingEnabled: Boolean = false
|
||||
|
||||
/**
|
||||
* Called once a message (containing a metadata field designated for this instrument) has been deserialized from the wire.
|
||||
* Called while serializing the message.
|
||||
* Parameters MAY be `null` (except `message` and `buffer`)!
|
||||
*/
|
||||
def remoteMessageReceived(recipient: ActorRef, message: Object, sender: ActorRef, metadata: ByteString): Unit
|
||||
def remoteWriteMetadata(recipient: ActorRef, message: Object, sender: ActorRef, buffer: ByteBuffer): Unit
|
||||
|
||||
/**
|
||||
* Called right before putting the message onto the wire.
|
||||
* Parameters MAY be `null` (except `message` and `buffer`)!
|
||||
*
|
||||
* The `size` is the total serialized size in bytes of the complete message including akka specific headers and any
|
||||
* `RemoteInstrument` metadata.
|
||||
* If `serializationTimingEnabled` returns true, then `time` will be the total time it took to serialize all data
|
||||
* in the message in nanoseconds, otherwise it is 0.
|
||||
*/
|
||||
def remoteMessageSent(recipient: ActorRef, message: Object, sender: ActorRef, size: Int, time: Long): Unit
|
||||
|
||||
/**
|
||||
* Called while deserializing the message once a message (containing a metadata field designated for this instrument) is found.
|
||||
*/
|
||||
def remoteReadMetadata(recipient: ActorRef, message: Object, sender: ActorRef, buffer: ByteBuffer): Unit
|
||||
|
||||
/**
|
||||
* Called when the message has been deserialized.
|
||||
*
|
||||
* The `size` is the total serialized size in bytes of the complete message including akka specific headers and any
|
||||
* `RemoteInstrument` metadata.
|
||||
* If `serializationTimingEnabled` returns true, then `time` will be the total time it took to deserialize all data
|
||||
* in the message in nanoseconds, otherwise it is 0.
|
||||
*/
|
||||
def remoteMessageReceived(recipient: ActorRef, message: Object, sender: ActorRef, size: Int, time: Long): Unit
|
||||
}
|
||||
|
||||
object NoopRemoteInstrument extends RemoteInstrument {
|
||||
override def identifier: Byte =
|
||||
-1
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* The metadata section is stored as raw bytes (prefixed with an Int length field,
|
||||
* the same way as any other literal), however the internal structure of it is as follows:
|
||||
*
|
||||
* {{{
|
||||
* Metadata entry:
|
||||
*
|
||||
* 0 1 2 3
|
||||
* 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2
|
||||
* +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
|
||||
* | Key | Metadata entry length |
|
||||
* +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
|
||||
* | ... metadata entry ... |
|
||||
* +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
|
||||
* }}}
|
||||
*
|
||||
*/
|
||||
private[remote] final class RemoteInstruments(
|
||||
private val system: ExtendedActorSystem,
|
||||
private val log: LoggingAdapter,
|
||||
_instruments: Vector[RemoteInstrument]) {
|
||||
import RemoteInstruments._
|
||||
|
||||
override def remoteMessageSent(recipient: ActorRef, message: Object, sender: ActorRef): ByteString =
|
||||
null
|
||||
def this(system: ExtendedActorSystem, log: LoggingAdapter) = this(system, log, RemoteInstruments.create(system, log))
|
||||
def this(system: ExtendedActorSystem) = this(system, Logging.getLogger(system, classOf[RemoteInstruments]))
|
||||
|
||||
override def remoteMessageReceived(recipient: ActorRef, message: Object, sender: ActorRef, metadata: ByteString): Unit =
|
||||
()
|
||||
// keep the remote instruments sorted by identifier to speed up deserialization
|
||||
private val instruments: Vector[RemoteInstrument] = _instruments.sortBy(_.identifier)
|
||||
// does any of the instruments want serialization timing?
|
||||
private val serializationTimingEnabled = instruments.exists(_.serializationTimingEnabled)
|
||||
|
||||
def serialize(outboundEnvelope: OptionVal[OutboundEnvelope], buffer: ByteBuffer): Unit = {
|
||||
if (instruments.nonEmpty && outboundEnvelope.isDefined) {
|
||||
val startPos = buffer.position()
|
||||
val oe = outboundEnvelope.get
|
||||
try {
|
||||
buffer.putInt(0)
|
||||
val dataPos = buffer.position()
|
||||
var i = 0
|
||||
while (i < instruments.length) {
|
||||
val rewindPos = buffer.position()
|
||||
val instrument = instruments(i)
|
||||
try {
|
||||
serializeInstrument(instrument, oe, buffer)
|
||||
} catch {
|
||||
case NonFatal(t) ⇒
|
||||
log.debug(
|
||||
"Skipping serialization of RemoteInstrument {} since it failed with {}",
|
||||
instrument.identifier, t.getMessage)
|
||||
buffer.position(rewindPos)
|
||||
}
|
||||
i += 1
|
||||
}
|
||||
val endPos = buffer.position()
|
||||
if (endPos == dataPos) {
|
||||
// no instruments wrote anything so we need to rewind to start
|
||||
buffer.position(startPos)
|
||||
} else {
|
||||
// some instruments wrote data, so write the total length
|
||||
buffer.putInt(startPos, endPos - dataPos)
|
||||
}
|
||||
} catch {
|
||||
case NonFatal(t) ⇒
|
||||
log.debug("Skipping serialization of all RemoteInstruments due to unhandled failure {}", t)
|
||||
buffer.position(startPos)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def serializeInstrument(instrument: RemoteInstrument, outboundEnvelope: OutboundEnvelope, buffer: ByteBuffer): Unit = {
|
||||
val startPos = buffer.position()
|
||||
buffer.putInt(0)
|
||||
val dataPos = buffer.position()
|
||||
instrument.remoteWriteMetadata(outboundEnvelope.recipient.orNull, outboundEnvelope.message, outboundEnvelope.sender.orNull, buffer)
|
||||
val endPos = buffer.position()
|
||||
if (endPos == dataPos) {
|
||||
// if the instrument didn't write anything, then rewind to the start
|
||||
buffer.position(startPos)
|
||||
} else {
|
||||
// the instrument wrote something so we need to write the identifier and length
|
||||
buffer.putInt(startPos, combineKeyLength(instrument.identifier, endPos - dataPos))
|
||||
}
|
||||
}
|
||||
|
||||
def deserialize(inboundEnvelope: InboundEnvelope): Unit = {
|
||||
if (inboundEnvelope.flag(EnvelopeBuffer.MetadataPresentFlag)) {
|
||||
inboundEnvelope.envelopeBuffer.byteBuffer.position(EnvelopeBuffer.MetadataContainerAndLiteralSectionOffset)
|
||||
deserializeRaw(inboundEnvelope)
|
||||
}
|
||||
}
|
||||
|
||||
def deserializeRaw(inboundEnvelope: InboundEnvelope): Unit = {
|
||||
val buffer = inboundEnvelope.envelopeBuffer.byteBuffer
|
||||
val length = buffer.getInt
|
||||
val endPos = buffer.position() + length
|
||||
try {
|
||||
if (instruments.nonEmpty) {
|
||||
var i = 0
|
||||
while (i < instruments.length && buffer.position() < endPos) {
|
||||
val instrument = instruments(i)
|
||||
val startPos = buffer.position()
|
||||
val keyAndLength = buffer.getInt
|
||||
val dataPos = buffer.position()
|
||||
val key = getKey(keyAndLength)
|
||||
val length = getLength(keyAndLength)
|
||||
var nextPos = dataPos + length
|
||||
val identifier = instrument.identifier
|
||||
if (key == identifier) {
|
||||
try {
|
||||
deserializeInstrument(instrument, inboundEnvelope, buffer)
|
||||
} catch {
|
||||
case NonFatal(t) ⇒
|
||||
log.debug(
|
||||
"Skipping deserialization of RemoteInstrument {} since it failed with {}",
|
||||
instrument.identifier, t.getMessage)
|
||||
}
|
||||
i += 1
|
||||
} else if (key > identifier) {
|
||||
// since instruments are sorted on both sides skip this local one and retry the serialized one
|
||||
log.debug("Skipping local RemoteInstrument {} that has no matching data in the message", identifier)
|
||||
nextPos = startPos
|
||||
i += 1
|
||||
} else {
|
||||
// since instruments are sorted on both sides skip the serialized one and retry the local one
|
||||
log.debug("Skipping serialized data in message for RemoteInstrument {} that has no local match", key)
|
||||
}
|
||||
buffer.position(nextPos)
|
||||
}
|
||||
} else {
|
||||
if (log.isDebugEnabled) log.debug(
|
||||
"Skipping serialized data in message for RemoteInstrument(s) {} that has no local match",
|
||||
remoteInstrumentIdIteratorRaw(buffer, endPos).mkString("[", ", ", "]"))
|
||||
}
|
||||
} catch {
|
||||
case NonFatal(t) ⇒
|
||||
log.debug("Skipping further deserialization of remaining RemoteInstruments due to unhandled failure {}", t)
|
||||
} finally {
|
||||
buffer.position(endPos)
|
||||
}
|
||||
}
|
||||
|
||||
private def deserializeInstrument(instrument: RemoteInstrument, inboundEnvelope: InboundEnvelope, buffer: ByteBuffer): Unit = {
|
||||
instrument.remoteReadMetadata(inboundEnvelope.recipient.orNull, inboundEnvelope.message, inboundEnvelope.sender.orNull, buffer)
|
||||
}
|
||||
|
||||
def messageSent(outboundEnvelope: OutboundEnvelope, size: Int, time: Long): Unit = {
|
||||
@tailrec def messageSent(pos: Int): Unit = {
|
||||
if (pos < instruments.length) {
|
||||
val instrument = instruments(pos)
|
||||
try {
|
||||
messageSentInstrument(instrument, outboundEnvelope, size, time)
|
||||
} catch {
|
||||
case NonFatal(t) ⇒
|
||||
log.debug("Message sent in RemoteInstrument {} failed with {}", instrument.identifier, t.getMessage)
|
||||
}
|
||||
messageSent(pos + 1)
|
||||
}
|
||||
}
|
||||
messageSent(0)
|
||||
}
|
||||
|
||||
private def messageSentInstrument(instrument: RemoteInstrument, outboundEnvelope: OutboundEnvelope, size: Int, time: Long): Unit = {
|
||||
instrument.remoteMessageSent(outboundEnvelope.recipient.orNull, outboundEnvelope.message, outboundEnvelope.sender.orNull, size, time)
|
||||
}
|
||||
|
||||
def messageReceived(inboundEnvelope: InboundEnvelope, size: Int, time: Long): Unit = {
|
||||
@tailrec def messageRecieved(pos: Int): Unit = {
|
||||
if (pos < instruments.length) {
|
||||
val instrument = instruments(pos)
|
||||
try {
|
||||
messageReceivedInstrument(instrument, inboundEnvelope, size, time)
|
||||
} catch {
|
||||
case NonFatal(t) ⇒
|
||||
log.debug("Message received in RemoteInstrument {} failed with {}", instrument.identifier, t.getMessage)
|
||||
}
|
||||
messageRecieved(pos + 1)
|
||||
}
|
||||
}
|
||||
messageRecieved(0)
|
||||
}
|
||||
|
||||
private def messageReceivedInstrument(instrument: RemoteInstrument, inboundEnvelope: InboundEnvelope, size: Int, time: Long): Unit = {
|
||||
instrument.remoteMessageReceived(inboundEnvelope.recipient.orNull, inboundEnvelope.message, inboundEnvelope.sender.orNull, size, time)
|
||||
}
|
||||
|
||||
private def remoteInstrumentIdIteratorRaw(buffer: ByteBuffer, endPos: Int): Iterator[Int] = {
|
||||
new Iterator[Int] {
|
||||
override def hasNext: Boolean = buffer.position() < endPos
|
||||
override def next(): Int = {
|
||||
val keyAndLength = buffer.getInt
|
||||
buffer.position(buffer.position() + getLength(keyAndLength))
|
||||
getKey(keyAndLength)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def isEmpty: Boolean = instruments.isEmpty
|
||||
def nonEmpty: Boolean = instruments.nonEmpty
|
||||
def timeSerialization = serializationTimingEnabled
|
||||
}
|
||||
|
||||
/** INTERNAL API */
|
||||
private[remote] object RemoteInstruments {
|
||||
def create(system: ExtendedActorSystem): Vector[RemoteInstrument] = {
|
||||
|
||||
def apply(system: ExtendedActorSystem): RemoteInstruments =
|
||||
new RemoteInstruments(system)
|
||||
|
||||
// key/length of a metadata element are encoded within a single integer:
|
||||
// supports keys in the range of <0-31>
|
||||
private final val lengthMask: Int = ~(31 << 26)
|
||||
def combineKeyLength(k: Byte, l: Int): Int = (k.toInt << 26) | (l & lengthMask)
|
||||
def getKey(kl: Int): Byte = (kl >>> 26).toByte
|
||||
def getLength(kl: Int): Int = kl & lengthMask
|
||||
|
||||
def create(system: ExtendedActorSystem, log: LoggingAdapter): Vector[RemoteInstrument] = {
|
||||
val c = system.settings.config
|
||||
val path = "akka.remote.artery.advanced.instruments"
|
||||
import scala.collection.JavaConverters._
|
||||
|
|
@ -69,124 +299,3 @@ private[remote] object RemoteInstruments {
|
|||
}(collection.breakOut)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* This datastructure is specialized for addressing directly into a known IDs slot.
|
||||
* It is used when deserializing/serializing metadata and we know the ID we want to reach into.
|
||||
*
|
||||
* Mutable & NOT thread-safe.
|
||||
*
|
||||
* Fixed-size: 32-slots array-backed Map-like structure.
|
||||
* Lazy: The backing array is allocated lazily only when needed, thus we pay no cost for the metadata array if
|
||||
* the system is not using metadata.
|
||||
* Life-cycle: Owned and linked to the lifecycle of an [[OutboundEnvelope]].
|
||||
* Re-cycled: Aimed to be re-cycled to produce the least possible GC-churn, by calling `clear()` when done with it.
|
||||
*
|
||||
* Reserved keys: The keys 0–7 are reserved for Akka internal purposes and future extensions.
|
||||
*/
|
||||
private[remote] final class MetadataMap[T >: Null] {
|
||||
val capacity = 32
|
||||
|
||||
protected var backing: Array[T] = null // TODO re-think if a plain LinkedList wouldn't be fine here?
|
||||
|
||||
private var _usedSlots = 0
|
||||
|
||||
def usedSlots = _usedSlots
|
||||
|
||||
def apply(i: Int): OptionVal[T] =
|
||||
if (backing == null) OptionVal.None
|
||||
else OptionVal[T](backing(i))
|
||||
|
||||
def isEmpty = usedSlots == 0
|
||||
|
||||
def nonEmpty = !isEmpty
|
||||
|
||||
def hasValueFor(i: Int) = nonEmpty && backing(i) != null
|
||||
|
||||
// FIXME too specialized...
|
||||
def foldLeftValues[A](zero: A)(f: (A, T) ⇒ A): A = {
|
||||
var acc: A = zero
|
||||
var hit = 0
|
||||
var i = 0
|
||||
while (i < capacity && hit < _usedSlots) {
|
||||
val it = backing(i)
|
||||
if (it != null) {
|
||||
acc = f(acc, it)
|
||||
hit += 1
|
||||
}
|
||||
i += 1
|
||||
}
|
||||
acc
|
||||
}
|
||||
|
||||
/** Heavy operation, only used for error logging */
|
||||
def keysWithValues: List[Int] = backing.zipWithIndex.filter({ case (t, id) ⇒ t != null }).map(_._2).toList
|
||||
|
||||
def foreach(f: (Byte, T) ⇒ Unit) = {
|
||||
var i = 0
|
||||
var hit = 0
|
||||
while (i < capacity && hit < _usedSlots) {
|
||||
val t = backing(i)
|
||||
if (t != null) {
|
||||
f(i.toByte, t)
|
||||
hit += 1
|
||||
}
|
||||
i += 1
|
||||
}
|
||||
}
|
||||
|
||||
private def allocateIfNeeded(): Unit =
|
||||
if (backing == null) backing = Array.ofDim[Object](capacity).asInstanceOf[Array[T]]
|
||||
|
||||
/**
|
||||
* Set a value at given index.
|
||||
* Setting a null value removes the entry (the slot is marked as not used).
|
||||
*/
|
||||
def set(i: Int, t: T): Unit =
|
||||
if (t == null) {
|
||||
if (backing == null) ()
|
||||
else {
|
||||
if (backing(i) != null) _usedSlots -= 1 // we're clearing a spot
|
||||
backing(i) = null.asInstanceOf[T]
|
||||
}
|
||||
} else {
|
||||
allocateIfNeeded()
|
||||
|
||||
if (backing(i) == null) {
|
||||
// was empty previously
|
||||
_usedSlots += 1
|
||||
} else {
|
||||
// replacing previous value, usedSlots remains unchanged
|
||||
}
|
||||
|
||||
backing(i) = t
|
||||
}
|
||||
|
||||
/**
|
||||
* If the backing array was already allocated clears it, if it wasn't does nothing (no-op).
|
||||
* This is so in order to pay no-cost when not using metadata - clearing then is instant.
|
||||
*/
|
||||
def clear() =
|
||||
if (isEmpty) ()
|
||||
else {
|
||||
var i = 0
|
||||
while (i < capacity) {
|
||||
backing(i) = null.asInstanceOf[T]
|
||||
i += 1
|
||||
}
|
||||
_usedSlots = 0
|
||||
}
|
||||
|
||||
override def toString() =
|
||||
if (backing == null) s"MetadataMap(<empty>)"
|
||||
else s"MetadataMap(${backing.toList.mkString("[", ",", "]")})"
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[remote] object MetadataMap {
|
||||
def apply[T >: Null]() = new MetadataMap[T]
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue