Refactoring of outbound compression, #21210

* outbound compression is now immutable, by simply using
  CompressionTable[ActorRef] and CompressionTable[String]
* immutable outbound compression will make it possible to use
  them from multiple Encoder instances, when we add several lanes
  for parallel serialization
* outbound compression tables not shared via AssociationState
* the advertised tables are sent to the Encoder stage via async
  callback, no need to reference the tables in other places than
  the Encoder stage, no more races via shared mutable state
* when outbound stream is started or restarted it can start out
  without compression, until next advertisement is received
* ensure outbound compression is cleared before handshake is signaled complete
This commit is contained in:
Patrik Nordwall 2016-08-24 19:52:07 +02:00
parent af5eb4c6bf
commit 0c0e3c5efd
16 changed files with 454 additions and 540 deletions

View file

@ -13,9 +13,28 @@ import akka.stream._
import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler }
import akka.util.{ ByteString, OptionVal, PrettyByteString }
import akka.actor.EmptyLocalActorRef
import akka.remote.artery.compress.{ InboundCompressions, OutboundCompressions, OutboundCompressionsImpl }
import akka.remote.artery.compress.InboundCompressions
import akka.stream.stage.TimerGraphStageLogic
import java.util.concurrent.TimeUnit
import scala.concurrent.Future
import akka.remote.artery.compress.CompressionTable
import akka.Done
import akka.stream.stage.GraphStageWithMaterializedValue
import scala.concurrent.Promise
/**
* INTERNAL API
*/
private[remote] object Encoder {
private[remote] trait ChangeOutboundCompression {
def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done]
def changeClassManifestCompression(table: CompressionTable[String]): Future[Done]
def clearCompression(): Future[Done]
}
private[remote] class ChangeOutboundCompressionFailed extends RuntimeException(
"Change of outbound compression table failed (will be retried), because materialization did not complete yet")
}
/**
* INTERNAL API
@ -23,42 +42,49 @@ import java.util.concurrent.TimeUnit
private[remote] class Encoder(
uniqueLocalAddress: UniqueAddress,
system: ActorSystem,
compression: OutboundCompressions,
outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope],
bufferPool: EnvelopeBufferPool)
extends GraphStage[FlowShape[OutboundEnvelope, EnvelopeBuffer]] {
extends GraphStageWithMaterializedValue[FlowShape[OutboundEnvelope, EnvelopeBuffer], Encoder.ChangeOutboundCompression] {
import Encoder._
val in: Inlet[OutboundEnvelope] = Inlet("Artery.Encoder.in")
val out: Outlet[EnvelopeBuffer] = Outlet("Artery.Encoder.out")
val shape: FlowShape[OutboundEnvelope, EnvelopeBuffer] = FlowShape(in, out)
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging {
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, ChangeOutboundCompression) = {
val logic = new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging with ChangeOutboundCompression {
private val headerBuilder = HeaderBuilder.out(compression)
private val headerBuilder = HeaderBuilder.out()
headerBuilder setVersion ArteryTransport.Version
headerBuilder setUid uniqueLocalAddress.uid
private val localAddress = uniqueLocalAddress.address
private val serialization = SerializationExtension(system)
private val serializationInfo = Serialization.Information(localAddress, system)
private val changeActorRefCompressionCb = getAsyncCallback[(CompressionTable[ActorRef], Promise[Done])] {
case (table, done)
headerBuilder.setOutboundActorRefCompression(table)
done.success(Done)
}
private val changeClassManifsetCompressionCb = getAsyncCallback[(CompressionTable[String], Promise[Done])] {
case (table, done)
headerBuilder.setOutboundClassManifestCompression(table)
done.success(Done)
}
private val clearCompressionCb = getAsyncCallback[Promise[Done]] { done
headerBuilder.setOutboundActorRefCompression(CompressionTable.empty[ActorRef])
headerBuilder.setOutboundClassManifestCompression(CompressionTable.empty[String])
done.success(Done)
}
override protected def logSource = classOf[Encoder]
override def onPush(): Unit = {
val outboundEnvelope = grab(in)
val envelope = bufferPool.acquire()
// FIXME: OMG race between setting the version, and using the table!!!!
// incoming messages are concurrent to outgoing ones
// incoming message may be table advertisement
// which swaps the table in Outgoing*Compression for the new one (n+1)
// by itself it does so atomically,
// race: however here we store the compression table version separately from actually using it (storing the refs / manifests etc).
// so there is a slight race IF the table is swapped right between us setting the version n here [then the table being swapped to n+1] and then we use the n+1 version to compressions the compressions (which the receiving end will fail to read, since the encoding could be completely different, and it picks the table based on the version Int).
// A solution would be to getTable => use it to set and compress things
headerBuilder setActorRefCompressionTableVersion compression.actorRefCompressionTableVersion
headerBuilder setClassManifestCompressionTableVersion compression.classManifestCompressionTableVersion
// internally compression is applied by the builder:
outboundEnvelope.recipient match {
case OptionVal.Some(r) headerBuilder setRecipientActorRef r
@ -109,8 +135,49 @@ private[remote] class Encoder(
override def onPull(): Unit = pull(in)
/**
* External call from ChangeOutboundCompression materialized value
*/
override def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done] = {
val done = Promise[Done]()
try changeActorRefCompressionCb.invoke((table, done)) catch {
// This is a harmless failure, it will be retried on next advertisement or handshake attempt.
// It will only occur when callback is invoked before preStart. That is highly unlikely to
// happen since advertisement is not done immediately and handshake involves network roundtrip.
case NonFatal(_) done.tryFailure(new ChangeOutboundCompressionFailed)
}
done.future
}
/**
* External call from ChangeOutboundCompression materialized value
*/
override def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] = {
val done = Promise[Done]()
try changeClassManifsetCompressionCb.invoke((table, done)) catch {
// in case materialization not completed yet
case NonFatal(_) done.tryFailure(new ChangeOutboundCompressionFailed)
}
done.future
}
/**
* External call from ChangeOutboundCompression materialized value
*/
override def clearCompression(): Future[Done] = {
val done = Promise[Done]()
try clearCompressionCb.invoke(done) catch {
// in case materialization not completed yet
case NonFatal(_) done.tryFailure(new ChangeOutboundCompressionFailed)
}
done.future
}
setHandlers(in, out, this)
}
(logic, logic)
}
}
/**
@ -198,20 +265,17 @@ private[remote] class Decoder(
val remoteAddress = assoc.remoteAddress
sender match {
case OptionVal.Some(snd)
compression.hitActorRef(originUid, headerBuilder.actorRefCompressionTableVersion,
remoteAddress, snd, 1)
compression.hitActorRef(originUid, remoteAddress, snd, 1)
case OptionVal.None
}
recipient match {
case OptionVal.Some(rcp)
compression.hitActorRef(originUid, headerBuilder.actorRefCompressionTableVersion,
remoteAddress, rcp, 1)
compression.hitActorRef(originUid, remoteAddress, rcp, 1)
case OptionVal.None
}
compression.hitClassManifest(originUid, headerBuilder.classManifestCompressionTableVersion,
remoteAddress, classManifest, 1)
compression.hitClassManifest(originUid, remoteAddress, classManifest, 1)
case _
// we don't want to record hits for compression while handshake is still in progress.