Remove OptionVal workaround (#30789)
* Remove OptionVal workaround * Revert "Avoid pattern-matching on OptionVal since Scala 2.13 allocates when checking the pattern" This reverts commit f0194bbc1ad43ac2c79bf156bfe91adf7fd5e538. * Revert "Optimizes retrieval of mandatoryAttributes by removing potential allocation of OptionVal" This reverts commit 165b0e0d5c057965e37418299061bdf48c33fc44.
This commit is contained in:
parent
0bc96eaa93
commit
62139a2220
4 changed files with 290 additions and 268 deletions
|
|
@ -103,13 +103,13 @@ import scala.util.Success
|
||||||
private var _currentActorThread: OptionVal[Thread] = OptionVal.None
|
private var _currentActorThread: OptionVal[Thread] = OptionVal.None
|
||||||
|
|
||||||
// context-shared timer needed to allow for nested timer usage
|
// context-shared timer needed to allow for nested timer usage
|
||||||
def timer: TimerSchedulerCrossDslSupport[T] = _timer.orNull match {
|
def timer: TimerSchedulerCrossDslSupport[T] = _timer match {
|
||||||
case null =>
|
case OptionVal.Some(timer) => timer
|
||||||
|
case _ =>
|
||||||
checkCurrentActorThread()
|
checkCurrentActorThread()
|
||||||
val timer = mkTimer()
|
val timer = mkTimer()
|
||||||
_timer = OptionVal.Some(timer)
|
_timer = OptionVal.Some(timer)
|
||||||
timer
|
timer
|
||||||
case timer => timer
|
|
||||||
}
|
}
|
||||||
|
|
||||||
protected[this] def mkTimer(): TimerSchedulerCrossDslSupport[T] = new TimerSchedulerImpl[T](this)
|
protected[this] def mkTimer(): TimerSchedulerCrossDslSupport[T] = new TimerSchedulerImpl[T](this)
|
||||||
|
|
@ -150,14 +150,14 @@ import scala.util.Success
|
||||||
|
|
||||||
private def loggingContext(): LoggingContext = {
|
private def loggingContext(): LoggingContext = {
|
||||||
// lazy init of logging setup
|
// lazy init of logging setup
|
||||||
_logging.orNull match {
|
_logging match {
|
||||||
case null =>
|
case OptionVal.Some(l) => l
|
||||||
|
case _ =>
|
||||||
val logClass = LoggerClass.detectLoggerClassFromStack(classOf[Behavior[_]])
|
val logClass = LoggerClass.detectLoggerClassFromStack(classOf[Behavior[_]])
|
||||||
val logger = LoggerFactory.getLogger(logClass.getName)
|
val logger = LoggerFactory.getLogger(logClass.getName)
|
||||||
val l = LoggingContext(logger, classicActorContext.props.deploy.tags, this)
|
val l = LoggingContext(logger, classicActorContext.props.deploy.tags, this)
|
||||||
_logging = OptionVal.Some(l)
|
_logging = OptionVal.Some(l)
|
||||||
l
|
l
|
||||||
case l => l
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -183,10 +183,11 @@ import scala.util.Success
|
||||||
// MDC is cleared (if used) from aroundReceive in ActorAdapter after processing each message
|
// MDC is cleared (if used) from aroundReceive in ActorAdapter after processing each message
|
||||||
override private[akka] def clearMdc(): Unit = {
|
override private[akka] def clearMdc(): Unit = {
|
||||||
// avoid access to MDC ThreadLocal if not needed, see details in LoggingContext
|
// avoid access to MDC ThreadLocal if not needed, see details in LoggingContext
|
||||||
val ctx = _logging.orNull
|
_logging match {
|
||||||
if ((ctx ne null) && ctx.mdcUsed) {
|
case OptionVal.Some(ctx) if ctx.mdcUsed =>
|
||||||
ActorMdc.clearMdc()
|
ActorMdc.clearMdc()
|
||||||
ctx.mdcUsed = false
|
ctx.mdcUsed = false
|
||||||
|
case _ =>
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -296,14 +297,14 @@ import scala.util.Success
|
||||||
val boxedMessageClass = BoxedType(messageClass).asInstanceOf[Class[U]]
|
val boxedMessageClass = BoxedType(messageClass).asInstanceOf[Class[U]]
|
||||||
_messageAdapters = (boxedMessageClass, f.asInstanceOf[Any => T]) ::
|
_messageAdapters = (boxedMessageClass, f.asInstanceOf[Any => T]) ::
|
||||||
_messageAdapters.filterNot { case (cls, _) => cls == boxedMessageClass }
|
_messageAdapters.filterNot { case (cls, _) => cls == boxedMessageClass }
|
||||||
val ref = messageAdapterRef.orNull match {
|
val ref = messageAdapterRef match {
|
||||||
case null =>
|
case OptionVal.Some(ref) => ref.asInstanceOf[ActorRef[U]]
|
||||||
|
case _ =>
|
||||||
// AdaptMessage is not really a T, but that is erased
|
// AdaptMessage is not really a T, but that is erased
|
||||||
val ref =
|
val ref =
|
||||||
internalSpawnMessageAdapter[Any](msg => AdaptWithRegisteredMessageAdapter(msg).asInstanceOf[T], "adapter")
|
internalSpawnMessageAdapter[Any](msg => AdaptWithRegisteredMessageAdapter(msg).asInstanceOf[T], "adapter")
|
||||||
messageAdapterRef = OptionVal.Some(ref)
|
messageAdapterRef = OptionVal.Some(ref)
|
||||||
ref
|
ref
|
||||||
case ref => ref.asInstanceOf[ActorRef[U]]
|
|
||||||
}
|
}
|
||||||
ref.asInstanceOf[ActorRef[U]]
|
ref.asInstanceOf[ActorRef[U]]
|
||||||
}
|
}
|
||||||
|
|
@ -317,14 +318,13 @@ import scala.util.Success
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
@InternalApi private[akka] def setCurrentActorThread(): Unit = {
|
@InternalApi private[akka] def setCurrentActorThread(): Unit = {
|
||||||
val callerThread = Thread.currentThread()
|
_currentActorThread match {
|
||||||
_currentActorThread.orNull match {
|
case OptionVal.Some(t) =>
|
||||||
case null =>
|
|
||||||
_currentActorThread = OptionVal.Some(callerThread)
|
|
||||||
case t =>
|
|
||||||
throw new IllegalStateException(
|
throw new IllegalStateException(
|
||||||
s"Invalid access by thread from the outside of $self. " +
|
s"Invalid access by thread from the outside of $self. " +
|
||||||
s"Current message is processed by $t, but also accessed from $callerThread.")
|
s"Current message is processed by $t, but also accessed from ${Thread.currentThread()}.")
|
||||||
|
case _ =>
|
||||||
|
_currentActorThread = OptionVal.Some(Thread.currentThread())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -340,17 +340,17 @@ import scala.util.Success
|
||||||
*/
|
*/
|
||||||
@InternalApi private[akka] def checkCurrentActorThread(): Unit = {
|
@InternalApi private[akka] def checkCurrentActorThread(): Unit = {
|
||||||
val callerThread = Thread.currentThread()
|
val callerThread = Thread.currentThread()
|
||||||
_currentActorThread.orNull match {
|
_currentActorThread match {
|
||||||
case null =>
|
case OptionVal.Some(t) =>
|
||||||
throw new UnsupportedOperationException(
|
|
||||||
s"Unsupported access to ActorContext from the outside of $self. " +
|
|
||||||
s"No message is currently processed by the actor, but ActorContext was called from $callerThread.")
|
|
||||||
case t =>
|
|
||||||
if (callerThread ne t) {
|
if (callerThread ne t) {
|
||||||
throw new UnsupportedOperationException(
|
throw new UnsupportedOperationException(
|
||||||
s"Unsupported access to ActorContext operation from the outside of $self. " +
|
s"Unsupported access to ActorContext operation from the outside of $self. " +
|
||||||
s"Current message is processed by $t, but ActorContext was called from $callerThread.")
|
s"Current message is processed by $t, but ActorContext was called from $callerThread.")
|
||||||
}
|
}
|
||||||
|
case _ =>
|
||||||
|
throw new UnsupportedOperationException(
|
||||||
|
s"Unsupported access to ActorContext from the outside of $self. " +
|
||||||
|
s"No message is currently processed by the actor, but ActorContext was called from $callerThread.")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -82,12 +82,12 @@ private[remote] class Encoder(
|
||||||
|
|
||||||
// lazy init of SerializationExtension to avoid loading serializers before ActorRefProvider has been initialized
|
// lazy init of SerializationExtension to avoid loading serializers before ActorRefProvider has been initialized
|
||||||
private var _serialization: OptionVal[Serialization] = OptionVal.None
|
private var _serialization: OptionVal[Serialization] = OptionVal.None
|
||||||
private def serialization: Serialization = _serialization.orNull match {
|
private def serialization: Serialization = _serialization match {
|
||||||
case null =>
|
case OptionVal.Some(s) => s
|
||||||
|
case _ =>
|
||||||
val s = SerializationExtension(system)
|
val s = SerializationExtension(system)
|
||||||
_serialization = OptionVal.Some(s)
|
_serialization = OptionVal.Some(s)
|
||||||
s
|
s
|
||||||
case s => s
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private val instruments: RemoteInstruments = RemoteInstruments(system)
|
private val instruments: RemoteInstruments = RemoteInstruments(system)
|
||||||
|
|
@ -130,14 +130,14 @@ private[remote] class Encoder(
|
||||||
Serialization.currentTransportInformation.value = serialization.serializationInformation
|
Serialization.currentTransportInformation.value = serialization.serializationInformation
|
||||||
|
|
||||||
// internally compression is applied by the builder:
|
// internally compression is applied by the builder:
|
||||||
outboundEnvelope.recipient.orNull match {
|
outboundEnvelope.recipient match {
|
||||||
case null => headerBuilder.setNoRecipient()
|
case OptionVal.Some(r) => headerBuilder.setRecipientActorRef(r)
|
||||||
case r => headerBuilder.setRecipientActorRef(r)
|
case _ => headerBuilder.setNoRecipient()
|
||||||
}
|
}
|
||||||
|
|
||||||
outboundEnvelope.sender.orNull match {
|
outboundEnvelope.sender match {
|
||||||
case null => headerBuilder.setNoSender()
|
case OptionVal.Some(s) => headerBuilder.setSenderActorRef(s)
|
||||||
case s => headerBuilder.setSenderActorRef(s)
|
case _ => headerBuilder.setNoSender()
|
||||||
}
|
}
|
||||||
|
|
||||||
val startTime: Long = if (instruments.timeSerialization) System.nanoTime else 0
|
val startTime: Long = if (instruments.timeSerialization) System.nanoTime else 0
|
||||||
|
|
@ -180,18 +180,18 @@ private[remote] class Encoder(
|
||||||
new OversizedPayloadException(reasonText),
|
new OversizedPayloadException(reasonText),
|
||||||
"Failed to serialize oversized message [{}].",
|
"Failed to serialize oversized message [{}].",
|
||||||
Logging.messageClassName(outboundEnvelope.message))
|
Logging.messageClassName(outboundEnvelope.message))
|
||||||
system.eventStream.publish(outboundEnvelope.sender.orNull match {
|
system.eventStream.publish(outboundEnvelope.sender match {
|
||||||
case null =>
|
case OptionVal.Some(msgSender) =>
|
||||||
Dropped(
|
|
||||||
outboundEnvelope.message,
|
|
||||||
reasonText,
|
|
||||||
outboundEnvelope.recipient.getOrElse(ActorRef.noSender))
|
|
||||||
case msgSender =>
|
|
||||||
Dropped(
|
Dropped(
|
||||||
outboundEnvelope.message,
|
outboundEnvelope.message,
|
||||||
reasonText,
|
reasonText,
|
||||||
msgSender,
|
msgSender,
|
||||||
outboundEnvelope.recipient.getOrElse(ActorRef.noSender))
|
outboundEnvelope.recipient.getOrElse(ActorRef.noSender))
|
||||||
|
case _ =>
|
||||||
|
Dropped(
|
||||||
|
outboundEnvelope.message,
|
||||||
|
reasonText,
|
||||||
|
outboundEnvelope.recipient.getOrElse(ActorRef.noSender))
|
||||||
})
|
})
|
||||||
pull(in)
|
pull(in)
|
||||||
case _ =>
|
case _ =>
|
||||||
|
|
@ -415,14 +415,13 @@ private[remote] class Decoder(
|
||||||
val originUid = headerBuilder.uid
|
val originUid = headerBuilder.uid
|
||||||
val association = inboundContext.association(originUid)
|
val association = inboundContext.association(originUid)
|
||||||
|
|
||||||
val recipient: OptionVal[InternalActorRef] = try headerBuilder.recipientActorRef(originUid).orNull match {
|
val recipient: OptionVal[InternalActorRef] = try headerBuilder.recipientActorRef(originUid) match {
|
||||||
case null =>
|
case OptionVal.Some(ref) =>
|
||||||
headerBuilder.recipientActorRefPath.orNull match {
|
OptionVal(ref.asInstanceOf[InternalActorRef])
|
||||||
case null => OptionVal.None
|
case OptionVal.None if headerBuilder.recipientActorRefPath.isDefined =>
|
||||||
case path => resolveRecipient(path)
|
resolveRecipient(headerBuilder.recipientActorRefPath.get)
|
||||||
}
|
case _ =>
|
||||||
case ref =>
|
OptionVal.None
|
||||||
OptionVal.Some(ref.asInstanceOf[InternalActorRef])
|
|
||||||
} catch {
|
} catch {
|
||||||
case NonFatal(e) =>
|
case NonFatal(e) =>
|
||||||
// probably version mismatch due to restarted system
|
// probably version mismatch due to restarted system
|
||||||
|
|
@ -430,14 +429,13 @@ private[remote] class Decoder(
|
||||||
OptionVal.None
|
OptionVal.None
|
||||||
}
|
}
|
||||||
|
|
||||||
val sender: OptionVal[InternalActorRef] = try headerBuilder.senderActorRef(originUid).orNull match {
|
val sender: OptionVal[InternalActorRef] = try headerBuilder.senderActorRef(originUid) match {
|
||||||
case null =>
|
case OptionVal.Some(ref) =>
|
||||||
headerBuilder.senderActorRefPath.orNull match {
|
OptionVal(ref.asInstanceOf[InternalActorRef])
|
||||||
case null => OptionVal.None
|
case OptionVal.None if headerBuilder.senderActorRefPath.isDefined =>
|
||||||
case path => OptionVal(actorRefResolver.resolve(path))
|
OptionVal(actorRefResolver.resolve(headerBuilder.senderActorRefPath.get))
|
||||||
}
|
case _ =>
|
||||||
case ref =>
|
OptionVal.None
|
||||||
OptionVal.Some(ref.asInstanceOf[InternalActorRef])
|
|
||||||
} catch {
|
} catch {
|
||||||
case NonFatal(e) =>
|
case NonFatal(e) =>
|
||||||
// probably version mismatch due to restarted system
|
// probably version mismatch due to restarted system
|
||||||
|
|
@ -475,12 +473,8 @@ private[remote] class Decoder(
|
||||||
|
|
||||||
if ((messageCount & heavyHitterMask) == 0) {
|
if ((messageCount & heavyHitterMask) == 0) {
|
||||||
// --- hit refs and manifests for heavy-hitter counting
|
// --- hit refs and manifests for heavy-hitter counting
|
||||||
association.orNull match {
|
association match {
|
||||||
case null =>
|
case OptionVal.Some(assoc) =>
|
||||||
// we don't want to record hits for compression while handshake is still in progress.
|
|
||||||
log.debug(
|
|
||||||
"Decoded message but unable to record hits for compression as no remoteAddress known. No association yet?")
|
|
||||||
case assoc =>
|
|
||||||
val remoteAddress = assoc.remoteAddress
|
val remoteAddress = assoc.remoteAddress
|
||||||
if (sender.isDefined)
|
if (sender.isDefined)
|
||||||
compressions.hitActorRef(originUid, remoteAddress, sender.get, 1)
|
compressions.hitActorRef(originUid, remoteAddress, sender.get, 1)
|
||||||
|
|
@ -489,6 +483,10 @@ private[remote] class Decoder(
|
||||||
compressions.hitActorRef(originUid, remoteAddress, recipient.get, 1)
|
compressions.hitActorRef(originUid, remoteAddress, recipient.get, 1)
|
||||||
|
|
||||||
compressions.hitClassManifest(originUid, remoteAddress, classManifest, 1)
|
compressions.hitClassManifest(originUid, remoteAddress, classManifest, 1)
|
||||||
|
case _ =>
|
||||||
|
// we don't want to record hits for compression while handshake is still in progress.
|
||||||
|
log.debug(
|
||||||
|
"Decoded message but unable to record hits for compression as no remoteAddress known. No association yet?")
|
||||||
}
|
}
|
||||||
// --- end of hit refs and manifests for heavy-hitter counting
|
// --- end of hit refs and manifests for heavy-hitter counting
|
||||||
}
|
}
|
||||||
|
|
@ -516,19 +514,19 @@ private[remote] class Decoder(
|
||||||
val recipientActorRefPath = headerBuilder.recipientActorRefPath.get
|
val recipientActorRefPath = headerBuilder.recipientActorRefPath.get
|
||||||
if (bannedRemoteDeployedActorRefs.contains(recipientActorRefPath)) {
|
if (bannedRemoteDeployedActorRefs.contains(recipientActorRefPath)) {
|
||||||
|
|
||||||
headerBuilder.recipientActorRefPath.orNull match {
|
headerBuilder.recipientActorRefPath match {
|
||||||
case null =>
|
case OptionVal.Some(path) =>
|
||||||
log.warning(
|
|
||||||
"Dropping message for banned (terminated, unresolved) remote deployed recipient [{}].",
|
|
||||||
recipientActorRefPath)
|
|
||||||
pull(in)
|
|
||||||
case path =>
|
|
||||||
val ref = actorRefResolver.getOrCompute(path)
|
val ref = actorRefResolver.getOrCompute(path)
|
||||||
if (ref.isInstanceOf[EmptyLocalActorRef])
|
if (ref.isInstanceOf[EmptyLocalActorRef])
|
||||||
log.warning(
|
log.warning(
|
||||||
"Message for banned (terminated, unresolved) remote deployed recipient [{}].",
|
"Message for banned (terminated, unresolved) remote deployed recipient [{}].",
|
||||||
recipientActorRefPath)
|
recipientActorRefPath)
|
||||||
push(out, decoded.withRecipient(ref))
|
push(out, decoded.withRecipient(ref))
|
||||||
|
case _ =>
|
||||||
|
log.warning(
|
||||||
|
"Dropping message for banned (terminated, unresolved) remote deployed recipient [{}].",
|
||||||
|
recipientActorRefPath)
|
||||||
|
pull(in)
|
||||||
}
|
}
|
||||||
|
|
||||||
} else
|
} else
|
||||||
|
|
@ -588,8 +586,10 @@ private[remote] class Decoder(
|
||||||
.runNextClassManifestAdvertisement() // TODO: optimise these operations, otherwise they stall the hotpath
|
.runNextClassManifestAdvertisement() // TODO: optimise these operations, otherwise they stall the hotpath
|
||||||
|
|
||||||
case RetryResolveRemoteDeployedRecipient(attemptsLeft, recipientPath, inboundEnvelope) =>
|
case RetryResolveRemoteDeployedRecipient(attemptsLeft, recipientPath, inboundEnvelope) =>
|
||||||
resolveRecipient(recipientPath).orNull match {
|
resolveRecipient(recipientPath) match {
|
||||||
case null =>
|
case OptionVal.Some(recipient) =>
|
||||||
|
push(out, inboundEnvelope.withRecipient(recipient))
|
||||||
|
case _ =>
|
||||||
if (attemptsLeft > 0)
|
if (attemptsLeft > 0)
|
||||||
scheduleOnce(
|
scheduleOnce(
|
||||||
RetryResolveRemoteDeployedRecipient(attemptsLeft - 1, recipientPath, inboundEnvelope),
|
RetryResolveRemoteDeployedRecipient(attemptsLeft - 1, recipientPath, inboundEnvelope),
|
||||||
|
|
@ -608,8 +608,6 @@ private[remote] class Decoder(
|
||||||
val recipient = actorRefResolver.getOrCompute(recipientPath)
|
val recipient = actorRefResolver.getOrCompute(recipientPath)
|
||||||
push(out, inboundEnvelope.withRecipient(recipient))
|
push(out, inboundEnvelope.withRecipient(recipient))
|
||||||
}
|
}
|
||||||
case recipient =>
|
|
||||||
push(out, inboundEnvelope.withRecipient(recipient))
|
|
||||||
}
|
}
|
||||||
|
|
||||||
case unknown => throw new IllegalArgumentException(s"Unknown timer key: $unknown")
|
case unknown => throw new IllegalArgumentException(s"Unknown timer key: $unknown")
|
||||||
|
|
@ -643,12 +641,12 @@ private[remote] class Deserializer(
|
||||||
|
|
||||||
// lazy init of SerializationExtension to avoid loading serializers before ActorRefProvider has been initialized
|
// lazy init of SerializationExtension to avoid loading serializers before ActorRefProvider has been initialized
|
||||||
private var _serialization: OptionVal[Serialization] = OptionVal.None
|
private var _serialization: OptionVal[Serialization] = OptionVal.None
|
||||||
private def serialization: Serialization = _serialization.orNull match {
|
private def serialization: Serialization = _serialization match {
|
||||||
case null =>
|
case OptionVal.Some(s) => s
|
||||||
|
case _ =>
|
||||||
val s = SerializationExtension(system)
|
val s = SerializationExtension(system)
|
||||||
_serialization = OptionVal.Some(s)
|
_serialization = OptionVal.Some(s)
|
||||||
s
|
s
|
||||||
case s => s
|
|
||||||
}
|
}
|
||||||
|
|
||||||
override protected def logSource = classOf[Deserializer]
|
override protected def logSource = classOf[Deserializer]
|
||||||
|
|
@ -677,9 +675,9 @@ private[remote] class Deserializer(
|
||||||
push(out, envelopeWithMessage)
|
push(out, envelopeWithMessage)
|
||||||
} catch {
|
} catch {
|
||||||
case NonFatal(e) =>
|
case NonFatal(e) =>
|
||||||
val from = envelope.association.orNull match {
|
val from = envelope.association match {
|
||||||
case null => "unknown"
|
case OptionVal.Some(a) => a.remoteAddress
|
||||||
case assoc => assoc.remoteAddress
|
case _ => "unknown"
|
||||||
}
|
}
|
||||||
log.warning(
|
log.warning(
|
||||||
"Failed to deserialize message from [{}] with serializer id [{}] and manifest [{}]. {}",
|
"Failed to deserialize message from [{}] with serializer id [{}] and manifest [{}]. {}",
|
||||||
|
|
|
||||||
|
|
@ -17,7 +17,7 @@ import akka.annotation.InternalApi
|
||||||
import akka.event.Logging
|
import akka.event.Logging
|
||||||
import akka.japi.function
|
import akka.japi.function
|
||||||
import akka.stream.impl.TraversalBuilder
|
import akka.stream.impl.TraversalBuilder
|
||||||
import akka.util.ByteString
|
import akka.util.{ ByteString, OptionVal }
|
||||||
import akka.util.JavaDurationConverters._
|
import akka.util.JavaDurationConverters._
|
||||||
import akka.util.LineNumbers
|
import akka.util.LineNumbers
|
||||||
|
|
||||||
|
|
@ -122,14 +122,17 @@ final case class Attributes(attributeList: List[Attributes.Attribute] = Nil) {
|
||||||
*/
|
*/
|
||||||
def getMandatoryAttribute[T <: MandatoryAttribute](c: Class[T]): T = {
|
def getMandatoryAttribute[T <: MandatoryAttribute](c: Class[T]): T = {
|
||||||
@tailrec
|
@tailrec
|
||||||
def find(list: List[Attribute]): T = list match {
|
def find(list: List[Attribute]): OptionVal[Attribute] = list match {
|
||||||
case Nil => throw new IllegalStateException(s"Mandatory attribute [$c] not found")
|
case Nil => OptionVal.None
|
||||||
case head :: tail =>
|
case head :: tail =>
|
||||||
if (c.isInstance(head)) c.cast(head)
|
if (c.isInstance(head)) OptionVal.Some(head)
|
||||||
else find(tail)
|
else find(tail)
|
||||||
}
|
}
|
||||||
|
|
||||||
find(attributeList)
|
find(attributeList) match {
|
||||||
|
case OptionVal.Some(t) => t.asInstanceOf[T]
|
||||||
|
case _ => throw new IllegalStateException(s"Mandatory attribute [$c] not found")
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -350,8 +350,8 @@ import akka.util.unused
|
||||||
case _ =>
|
case _ =>
|
||||||
graph.traversalBuilder match {
|
graph.traversalBuilder match {
|
||||||
case l: LinearTraversalBuilder =>
|
case l: LinearTraversalBuilder =>
|
||||||
l.pendingBuilder.orNull match {
|
l.pendingBuilder match {
|
||||||
case a: AtomicTraversalBuilder =>
|
case OptionVal.Some(a: AtomicTraversalBuilder) =>
|
||||||
a.module match {
|
a.module match {
|
||||||
case m: GraphStageModule[_, _] =>
|
case m: GraphStageModule[_, _] =>
|
||||||
m.stage match {
|
m.stage match {
|
||||||
|
|
@ -359,8 +359,7 @@ import akka.util.unused
|
||||||
// It would be != EmptyTraversal if mapMaterializedValue was used and then we can't optimize.
|
// It would be != EmptyTraversal if mapMaterializedValue was used and then we can't optimize.
|
||||||
if ((l.traversalSoFar eq EmptyTraversal) && !l.attributes.isAsync)
|
if ((l.traversalSoFar eq EmptyTraversal) && !l.attributes.isAsync)
|
||||||
OptionVal.Some(single)
|
OptionVal.Some(single)
|
||||||
else
|
else OptionVal.None
|
||||||
OptionVal.None
|
|
||||||
case _ => OptionVal.None
|
case _ => OptionVal.None
|
||||||
}
|
}
|
||||||
case _ => OptionVal.None
|
case _ => OptionVal.None
|
||||||
|
|
@ -506,9 +505,10 @@ import akka.util.unused
|
||||||
}
|
}
|
||||||
|
|
||||||
override def traversal: Traversal = {
|
override def traversal: Traversal = {
|
||||||
val withIsland =
|
val withIsland = islandTag match {
|
||||||
if (islandTag.isDefined) EnterIsland(islandTag.get).concat(traversalSoFar).concat(ExitIsland)
|
case OptionVal.Some(tag) => EnterIsland(tag).concat(traversalSoFar).concat(ExitIsland)
|
||||||
else traversalSoFar
|
case _ => traversalSoFar
|
||||||
|
}
|
||||||
|
|
||||||
if (attributes eq Attributes.none) withIsland
|
if (attributes eq Attributes.none) withIsland
|
||||||
else PushAttributes(attributes).concat(withIsland).concat(PopAttributes)
|
else PushAttributes(attributes).concat(withIsland).concat(PopAttributes)
|
||||||
|
|
@ -531,8 +531,10 @@ import akka.util.unused
|
||||||
override def unwiredOuts: Int = 0
|
override def unwiredOuts: Int = 0
|
||||||
|
|
||||||
override def makeIsland(islandTag: IslandTag): TraversalBuilder =
|
override def makeIsland(islandTag: IslandTag): TraversalBuilder =
|
||||||
if (this.islandTag.isDefined) this
|
this.islandTag match {
|
||||||
else copy(islandTag = OptionVal(islandTag))
|
case OptionVal.None => copy(islandTag = OptionVal(islandTag))
|
||||||
|
case _ => this
|
||||||
|
}
|
||||||
|
|
||||||
override def assign(out: OutPort, relativeSlot: Int): TraversalBuilder =
|
override def assign(out: OutPort, relativeSlot: Int): TraversalBuilder =
|
||||||
throw new UnsupportedOperationException("Cannot assign ports to slots in a completed builder.")
|
throw new UnsupportedOperationException("Cannot assign ports to slots in a completed builder.")
|
||||||
|
|
@ -683,10 +685,13 @@ import akka.util.unused
|
||||||
|
|
||||||
case completed: CompletedTraversalBuilder =>
|
case completed: CompletedTraversalBuilder =>
|
||||||
val inOpt = OptionVal(shape.inlets.headOption.orNull)
|
val inOpt = OptionVal(shape.inlets.headOption.orNull)
|
||||||
val inOffs = if (inOpt.isDefined) completed.offsetOf(inOpt.get) else 0
|
val inOffs = inOpt match {
|
||||||
|
case OptionVal.Some(in) => completed.offsetOf(in)
|
||||||
|
case _ => 0
|
||||||
|
}
|
||||||
|
|
||||||
LinearTraversalBuilder(
|
LinearTraversalBuilder(
|
||||||
inPort = inOpt,
|
inPort = OptionVal(inOpt.orNull),
|
||||||
outPort = OptionVal.None,
|
outPort = OptionVal.None,
|
||||||
inOffset = inOffs,
|
inOffset = inOffs,
|
||||||
inSlots = completed.inSlots,
|
inSlots = completed.inSlots,
|
||||||
|
|
@ -697,10 +702,13 @@ import akka.util.unused
|
||||||
case composite =>
|
case composite =>
|
||||||
val inOpt = OptionVal(shape.inlets.headOption.orNull)
|
val inOpt = OptionVal(shape.inlets.headOption.orNull)
|
||||||
val out = shape.outlets.head // Cannot be empty, otherwise it would be a CompletedTraversalBuilder
|
val out = shape.outlets.head // Cannot be empty, otherwise it would be a CompletedTraversalBuilder
|
||||||
val inOffs = if (inOpt.isDefined) composite.offsetOf(inOpt.get) else 0
|
val inOffs = inOpt match {
|
||||||
|
case OptionVal.Some(in) => composite.offsetOf(in)
|
||||||
|
case _ => 0
|
||||||
|
}
|
||||||
|
|
||||||
LinearTraversalBuilder(
|
LinearTraversalBuilder(
|
||||||
inPort = inOpt,
|
inPort = OptionVal(inOpt.orNull),
|
||||||
outPort = OptionVal.Some(out),
|
outPort = OptionVal.Some(out),
|
||||||
inOffset = inOffs,
|
inOffset = inOffs,
|
||||||
inSlots = composite.inSlots,
|
inSlots = composite.inSlots,
|
||||||
|
|
@ -762,7 +770,10 @@ import akka.util.unused
|
||||||
}
|
}
|
||||||
|
|
||||||
private def applyIslandAndAttributes(t: Traversal): Traversal = {
|
private def applyIslandAndAttributes(t: Traversal): Traversal = {
|
||||||
val withIslandTag = if (islandTag.isDefined) EnterIsland(islandTag.get).concat(t).concat(ExitIsland) else t
|
val withIslandTag = islandTag match {
|
||||||
|
case OptionVal.Some(tag) => EnterIsland(tag).concat(t).concat(ExitIsland)
|
||||||
|
case _ => t
|
||||||
|
}
|
||||||
|
|
||||||
if (attributes eq Attributes.none) withIslandTag
|
if (attributes eq Attributes.none) withIslandTag
|
||||||
else PushAttributes(attributes).concat(withIslandTag).concat(PopAttributes)
|
else PushAttributes(attributes).concat(withIslandTag).concat(PopAttributes)
|
||||||
|
|
@ -786,22 +797,22 @@ import akka.util.unused
|
||||||
*/
|
*/
|
||||||
override def wire(out: OutPort, in: InPort): TraversalBuilder = {
|
override def wire(out: OutPort, in: InPort): TraversalBuilder = {
|
||||||
if (outPort.contains(out) && inPort.contains(in)) {
|
if (outPort.contains(out) && inPort.contains(in)) {
|
||||||
if (pendingBuilder.isDefined) {
|
pendingBuilder match {
|
||||||
val composite = pendingBuilder.get
|
case OptionVal.Some(composite) =>
|
||||||
copy(
|
copy(
|
||||||
inPort = OptionVal.None,
|
inPort = OptionVal.None,
|
||||||
outPort = OptionVal.None,
|
outPort = OptionVal.None,
|
||||||
traversalSoFar = applyIslandAndAttributes(
|
traversalSoFar = applyIslandAndAttributes(
|
||||||
beforeBuilder
|
beforeBuilder
|
||||||
.concat(composite.assign(out, inOffset - composite.offsetOfModule(out)).traversal)
|
.concat(composite.assign(out, inOffset - composite.offsetOfModule(out)).traversal)
|
||||||
.concat(traversalSoFar)),
|
.concat(traversalSoFar)),
|
||||||
pendingBuilder = OptionVal.None,
|
pendingBuilder = OptionVal.None,
|
||||||
beforeBuilder = EmptyTraversal)
|
beforeBuilder = EmptyTraversal)
|
||||||
} else {
|
case _ =>
|
||||||
copy(
|
copy(
|
||||||
inPort = OptionVal.None,
|
inPort = OptionVal.None,
|
||||||
outPort = OptionVal.None,
|
outPort = OptionVal.None,
|
||||||
traversalSoFar = rewireLastOutTo(traversalSoFar, inOffset))
|
traversalSoFar = rewireLastOutTo(traversalSoFar, inOffset))
|
||||||
}
|
}
|
||||||
} else
|
} else
|
||||||
throw new IllegalArgumentException(s"The ports $in and $out cannot be accessed in this builder.")
|
throw new IllegalArgumentException(s"The ports $in and $out cannot be accessed in this builder.")
|
||||||
|
|
@ -809,10 +820,10 @@ import akka.util.unused
|
||||||
|
|
||||||
override def offsetOfModule(out: OutPort): Int = {
|
override def offsetOfModule(out: OutPort): Int = {
|
||||||
if (outPort.contains(out)) {
|
if (outPort.contains(out)) {
|
||||||
if (pendingBuilder.isDefined)
|
pendingBuilder match {
|
||||||
pendingBuilder.get.offsetOfModule(out)
|
case OptionVal.Some(composite) => composite.offsetOfModule(out)
|
||||||
else
|
case _ => 0 // Output belongs to the last module, which will be materialized *first*
|
||||||
0 // Output belongs to the last module, which will be materialized *first*
|
}
|
||||||
} else
|
} else
|
||||||
throw new IllegalArgumentException(s"Port $out cannot be accessed in this builder")
|
throw new IllegalArgumentException(s"Port $out cannot be accessed in this builder")
|
||||||
}
|
}
|
||||||
|
|
@ -828,16 +839,16 @@ import akka.util.unused
|
||||||
|
|
||||||
override def assign(out: OutPort, relativeSlot: Int): TraversalBuilder = {
|
override def assign(out: OutPort, relativeSlot: Int): TraversalBuilder = {
|
||||||
if (outPort.contains(out)) {
|
if (outPort.contains(out)) {
|
||||||
if (pendingBuilder.isDefined) {
|
pendingBuilder match {
|
||||||
val composite = pendingBuilder.get
|
case OptionVal.Some(composite) =>
|
||||||
copy(
|
copy(
|
||||||
outPort = OptionVal.None,
|
outPort = OptionVal.None,
|
||||||
traversalSoFar = applyIslandAndAttributes(
|
traversalSoFar = applyIslandAndAttributes(
|
||||||
beforeBuilder.concat(composite.assign(out, relativeSlot).traversal.concat(traversalSoFar))),
|
beforeBuilder.concat(composite.assign(out, relativeSlot).traversal.concat(traversalSoFar))),
|
||||||
pendingBuilder = OptionVal.None,
|
pendingBuilder = OptionVal.None,
|
||||||
beforeBuilder = EmptyTraversal)
|
beforeBuilder = EmptyTraversal)
|
||||||
} else {
|
case _ =>
|
||||||
copy(outPort = OptionVal.None, traversalSoFar = rewireLastOutTo(traversalSoFar, relativeSlot))
|
copy(outPort = OptionVal.None, traversalSoFar = rewireLastOutTo(traversalSoFar, relativeSlot))
|
||||||
}
|
}
|
||||||
} else
|
} else
|
||||||
throw new IllegalArgumentException(s"Port $out cannot be assigned in this builder")
|
throw new IllegalArgumentException(s"Port $out cannot be assigned in this builder")
|
||||||
|
|
@ -907,82 +918,83 @@ import akka.util.unused
|
||||||
* Depending whether we have a composite builder as our last step or not, composition will be somewhat
|
* Depending whether we have a composite builder as our last step or not, composition will be somewhat
|
||||||
* different.
|
* different.
|
||||||
*/
|
*/
|
||||||
val assembledTraversalForThis = if (this.pendingBuilder.isDefined) {
|
val assembledTraversalForThis = this.pendingBuilder match {
|
||||||
val composite = this.pendingBuilder.get
|
case OptionVal.Some(composite) =>
|
||||||
/*
|
|
||||||
* This is the case where our last module is a composite, and since it does not have its output port
|
|
||||||
* wired yet, the traversal is split into the parts, traversalSoFar, pendingBuilder and beforeBuilder.
|
|
||||||
*
|
|
||||||
* Since we will wire now the output port, we can assemble everything together:
|
|
||||||
*/
|
|
||||||
val out = outPort.get
|
|
||||||
/*
|
|
||||||
* Since we will be visited second (and the appended toAppend first), we need to
|
|
||||||
*
|
|
||||||
* 1. go back to the start of the composite module, i.e. composite.offsetOfModule(out) steps. This
|
|
||||||
* is necessary because the composite might not have the internal module as the first visited
|
|
||||||
* module in the Traversal and hence not have a base offset of 0 in the composite
|
|
||||||
* 2. go backward toAppend.inSlots slots to reach the beginning offset of toAppend
|
|
||||||
* 3. now go forward toAppend.inOffset to reach the correct location
|
|
||||||
*
|
|
||||||
* <------- (-composite.offsetOfModule(out))
|
|
||||||
* <-------------- (-toAppend.inSlots)
|
|
||||||
* -------> (+toAppend.inOffset)
|
|
||||||
*
|
|
||||||
* --------in----|-------[out module]----------
|
|
||||||
* toAppend this
|
|
||||||
*
|
|
||||||
*/
|
|
||||||
val compositeTraversal = composite
|
|
||||||
.assign(out, -composite.offsetOfModule(out) - toAppend.inSlots + toAppend.inOffset)
|
|
||||||
.traversal // All output ports are finished, so we can finally call this now
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Now we can assemble the pieces for the final Traversal of _this_ builder.
|
|
||||||
*
|
|
||||||
* beforeBuilder ~ compositeTraversal ~ traversalSoFar
|
|
||||||
*
|
|
||||||
* (remember that this is the _reverse_ of the Flow DSL order)
|
|
||||||
*/
|
|
||||||
beforeBuilder.concat(compositeTraversal).concat(traversalSoFar)
|
|
||||||
} else {
|
|
||||||
/*
|
|
||||||
* This is the case where we are a pure linear builder (all composites have been already completed),
|
|
||||||
* which means that traversalSoFar contains everything already, except the final attributes and islands
|
|
||||||
* applied.
|
|
||||||
*
|
|
||||||
* Since the exposed output port has been wired optimistically to -1, we need to check if this is correct,
|
|
||||||
* and correct if necessary. This is the step below:
|
|
||||||
*/
|
|
||||||
if (toAppend.inOffset == (toAppend.inSlots - 1)) {
|
|
||||||
/*
|
/*
|
||||||
* if the builder we want to append (remember that is _prepend_ from the Traversal's perspective)
|
* This is the case where our last module is a composite, and since it does not have its output port
|
||||||
* has its exposed input port at the last location (which is toAppend.inSlots - 1 because input
|
* wired yet, the traversal is split into the parts, traversalSoFar, pendingBuilder and beforeBuilder.
|
||||||
* port offsets start with 0), then -1 is the correct wiring. I.e.
|
|
||||||
*
|
*
|
||||||
* 1. Visit the appended module first in the traversal, its input port is the last
|
* Since we will wire now the output port, we can assemble everything together:
|
||||||
* 2. Visit this module second in the traversal, wire the output port back to the previous input port (-1)
|
|
||||||
*/
|
*/
|
||||||
traversalSoFar
|
val out = outPort.get
|
||||||
} else {
|
|
||||||
/*
|
/*
|
||||||
* The optimistic mapping to -1 is not correct, we need to unfold the Traversal to find our last module
|
|
||||||
* (which is the _first_ module in the Traversal) and rewire the output assignment to the correct offset.
|
|
||||||
*
|
|
||||||
* Since we will be visited second (and the appended toAppend first), we need to
|
* Since we will be visited second (and the appended toAppend first), we need to
|
||||||
*
|
*
|
||||||
* 1. go backward toAppend.inSlots slots to reach the beginning offset of toAppend
|
* 1. go back to the start of the composite module, i.e. composite.offsetOfModule(out) steps. This
|
||||||
* 2. now go forward toAppend.inOffset to reach the correct location
|
* is necessary because the composite might not have the internal module as the first visited
|
||||||
|
* module in the Traversal and hence not have a base offset of 0 in the composite
|
||||||
|
* 2. go backward toAppend.inSlots slots to reach the beginning offset of toAppend
|
||||||
|
* 3. now go forward toAppend.inOffset to reach the correct location
|
||||||
*
|
*
|
||||||
* <-------------- (-toAppend.inSlots)
|
* <------- (-composite.offsetOfModule(out))
|
||||||
* -------> (+toAppend.inOffset)
|
* <-------------- (-toAppend.inSlots)
|
||||||
|
* -------> (+toAppend.inOffset)
|
||||||
*
|
*
|
||||||
* --------in----|[out module]----------
|
* --------in----|-------[out module]----------
|
||||||
* toAppend this
|
* toAppend this
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
rewireLastOutTo(traversalSoFar, toAppend.inOffset - toAppend.inSlots)
|
val compositeTraversal = composite
|
||||||
}
|
.assign(out, -composite.offsetOfModule(out) - toAppend.inSlots + toAppend.inOffset)
|
||||||
|
.traversal // All output ports are finished, so we can finally call this now
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Now we can assemble the pieces for the final Traversal of _this_ builder.
|
||||||
|
*
|
||||||
|
* beforeBuilder ~ compositeTraversal ~ traversalSoFar
|
||||||
|
*
|
||||||
|
* (remember that this is the _reverse_ of the Flow DSL order)
|
||||||
|
*/
|
||||||
|
beforeBuilder.concat(compositeTraversal).concat(traversalSoFar)
|
||||||
|
|
||||||
|
case _ =>
|
||||||
|
/*
|
||||||
|
* This is the case where we are a pure linear builder (all composites have been already completed),
|
||||||
|
* which means that traversalSoFar contains everything already, except the final attributes and islands
|
||||||
|
* applied.
|
||||||
|
*
|
||||||
|
* Since the exposed output port has been wired optimistically to -1, we need to check if this is correct,
|
||||||
|
* and correct if necessary. This is the step below:
|
||||||
|
*/
|
||||||
|
if (toAppend.inOffset == (toAppend.inSlots - 1)) {
|
||||||
|
/*
|
||||||
|
* if the builder we want to append (remember that is _prepend_ from the Traversal's perspective)
|
||||||
|
* has its exposed input port at the last location (which is toAppend.inSlots - 1 because input
|
||||||
|
* port offsets start with 0), then -1 is the correct wiring. I.e.
|
||||||
|
*
|
||||||
|
* 1. Visit the appended module first in the traversal, its input port is the last
|
||||||
|
* 2. Visit this module second in the traversal, wire the output port back to the previous input port (-1)
|
||||||
|
*/
|
||||||
|
traversalSoFar
|
||||||
|
} else {
|
||||||
|
/*
|
||||||
|
* The optimistic mapping to -1 is not correct, we need to unfold the Traversal to find our last module
|
||||||
|
* (which is the _first_ module in the Traversal) and rewire the output assignment to the correct offset.
|
||||||
|
*
|
||||||
|
* Since we will be visited second (and the appended toAppend first), we need to
|
||||||
|
*
|
||||||
|
* 1. go backward toAppend.inSlots slots to reach the beginning offset of toAppend
|
||||||
|
* 2. now go forward toAppend.inOffset to reach the correct location
|
||||||
|
*
|
||||||
|
* <-------------- (-toAppend.inSlots)
|
||||||
|
* -------> (+toAppend.inOffset)
|
||||||
|
*
|
||||||
|
* --------in----|[out module]----------
|
||||||
|
* toAppend this
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
rewireLastOutTo(traversalSoFar, toAppend.inOffset - toAppend.inSlots)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
|
@ -997,75 +1009,79 @@ import akka.util.unused
|
||||||
* We have finished "this" builder, now we need to construct the new builder as the result of appending.
|
* We have finished "this" builder, now we need to construct the new builder as the result of appending.
|
||||||
* There are two variants, depending whether toAppend is purely linear or if it has a composite at the end.
|
* There are two variants, depending whether toAppend is purely linear or if it has a composite at the end.
|
||||||
*/
|
*/
|
||||||
if (toAppend.pendingBuilder.isEmpty) {
|
toAppend.pendingBuilder match {
|
||||||
/*
|
case OptionVal.None =>
|
||||||
* This is the simple case, when the other is purely linear. We just concatenate the traversals
|
/*
|
||||||
* and do some bookkeeping.
|
* This is the simple case, when the other is purely linear. We just concatenate the traversals
|
||||||
*/
|
* and do some bookkeeping.
|
||||||
LinearTraversalBuilder(
|
*/
|
||||||
inPort = inPort,
|
LinearTraversalBuilder(
|
||||||
outPort = toAppend.outPort,
|
inPort = inPort,
|
||||||
inSlots = inSlots + toAppend.inSlots, // we have now more input ports than before
|
outPort = toAppend.outPort,
|
||||||
// the inOffset of _this_ gets shifted by toAppend.inSlots, because the traversal of toAppend is _prepended_
|
inSlots = inSlots + toAppend.inSlots, // we have now more input ports than before
|
||||||
inOffset = inOffset + toAppend.inSlots,
|
// the inOffset of _this_ gets shifted by toAppend.inSlots, because the traversal of toAppend is _prepended_
|
||||||
// Build in reverse so it yields a more efficient layout for left-to-right building
|
inOffset = inOffset + toAppend.inSlots,
|
||||||
traversalSoFar = toAppend.applyIslandAndAttributes(toAppend.traversalSoFar).concat(finalTraversalForThis),
|
// Build in reverse so it yields a more efficient layout for left-to-right building
|
||||||
pendingBuilder = OptionVal.None,
|
traversalSoFar = toAppend.applyIslandAndAttributes(toAppend.traversalSoFar).concat(finalTraversalForThis),
|
||||||
attributes = Attributes.none, // attributes are none for the new enclosing builder
|
pendingBuilder = OptionVal.None,
|
||||||
beforeBuilder = EmptyTraversal, // no need for beforeBuilder as there are no composites
|
attributes = Attributes.none, // attributes are none for the new enclosing builder
|
||||||
islandTag = OptionVal.None // islandTag is reset for the new enclosing builder
|
beforeBuilder = EmptyTraversal, // no need for beforeBuilder as there are no composites
|
||||||
)
|
islandTag = OptionVal.None // islandTag is reset for the new enclosing builder
|
||||||
} else {
|
)
|
||||||
/*
|
|
||||||
* In this case we need to assemble as much as we can, and create a new "sandwich" of
|
|
||||||
* beforeBuilder ~ pendingBuilder ~ traversalSoFar
|
|
||||||
*
|
|
||||||
* We need to apply the attributes and islandTags of the appended builder, but we cannot do it in one
|
|
||||||
* step, instead we need to append half of the steps to traversalSoFar, and the other half to
|
|
||||||
* beforeBuilder.
|
|
||||||
*/
|
|
||||||
var newTraversalSoFar = finalTraversalForThis
|
|
||||||
var newBeforeTraversal = toAppend.beforeBuilder
|
|
||||||
|
|
||||||
// First prepare island enter and exit if tags are present
|
case _ => // Some(pendingBuilder)
|
||||||
if (toAppend.islandTag.isDefined) {
|
/*
|
||||||
// Enter the island just before the appended builder (keeping the toAppend.beforeBuilder steps)
|
* In this case we need to assemble as much as we can, and create a new "sandwich" of
|
||||||
newBeforeTraversal = EnterIsland(toAppend.islandTag.get).concat(newBeforeTraversal)
|
* beforeBuilder ~ pendingBuilder ~ traversalSoFar
|
||||||
// Exit the island just after the appended builder (they should not applied to _this_ builder)
|
*
|
||||||
newTraversalSoFar = ExitIsland.concat(newTraversalSoFar)
|
* We need to apply the attributes and islandTags of the appended builder, but we cannot do it in one
|
||||||
}
|
* step, instead we need to append half of the steps to traversalSoFar, and the other half to
|
||||||
|
* beforeBuilder.
|
||||||
|
*/
|
||||||
|
var newTraversalSoFar = finalTraversalForThis
|
||||||
|
var newBeforeTraversal = toAppend.beforeBuilder
|
||||||
|
|
||||||
// Secondly, prepare attribute push and pop if Attributes are present
|
// First prepare island enter and exit if tags are present
|
||||||
if (toAppend.attributes ne Attributes.none) {
|
toAppend.islandTag match {
|
||||||
// Push the attributes just before the appended builder.
|
case OptionVal.Some(tag) =>
|
||||||
newBeforeTraversal = PushAttributes(toAppend.attributes).concat(newBeforeTraversal)
|
// Enter the island just before the appended builder (keeping the toAppend.beforeBuilder steps)
|
||||||
// Pop the attributes immediately after the appended builder (they should not applied to _this_ builder)
|
newBeforeTraversal = EnterIsland(tag).concat(newBeforeTraversal)
|
||||||
newTraversalSoFar = PopAttributes.concat(newTraversalSoFar)
|
// Exit the island just after the appended builder (they should not applied to _this_ builder)
|
||||||
}
|
newTraversalSoFar = ExitIsland.concat(newTraversalSoFar)
|
||||||
|
case _ => // Nothing changes
|
||||||
|
}
|
||||||
|
|
||||||
// This is roughly how things will look like in the end:
|
// Secondly, prepare attribute push and pop if Attributes are present
|
||||||
//
|
if (toAppend.attributes ne Attributes.none) {
|
||||||
// newBeforeTraversal newTraversalSoFar
|
// Push the attributes just before the appended builder.
|
||||||
// [PushAttributes ~ EnterIsland] ~ composite ~ [toAppend.traversalSoFar ~ ExitIsland ~ PopAttributes ~ finalTraversalForThis]
|
newBeforeTraversal = PushAttributes(toAppend.attributes).concat(newBeforeTraversal)
|
||||||
|
// Pop the attributes immediately after the appended builder (they should not applied to _this_ builder)
|
||||||
|
newTraversalSoFar = PopAttributes.concat(newTraversalSoFar)
|
||||||
|
}
|
||||||
|
|
||||||
// Finally add the already completed part of toAppend to newTraversalSoFar
|
// This is roughly how things will look like in the end:
|
||||||
newTraversalSoFar = toAppend.traversalSoFar.concat(newTraversalSoFar)
|
//
|
||||||
|
// newBeforeTraversal newTraversalSoFar
|
||||||
|
// [PushAttributes ~ EnterIsland] ~ composite ~ [toAppend.traversalSoFar ~ ExitIsland ~ PopAttributes ~ finalTraversalForThis]
|
||||||
|
|
||||||
LinearTraversalBuilder(
|
// Finally add the already completed part of toAppend to newTraversalSoFar
|
||||||
inPort = inPort,
|
newTraversalSoFar = toAppend.traversalSoFar.concat(newTraversalSoFar)
|
||||||
outPort = toAppend.outPort,
|
|
||||||
inSlots = inSlots + toAppend.inSlots, // we have now more input ports than before
|
LinearTraversalBuilder(
|
||||||
// the inOffset of _this_ gets shifted by toAppend.inSlots, because the traversal of toAppend is _prepended_
|
inPort = inPort,
|
||||||
inOffset = inOffset + toAppend.inSlots,
|
outPort = toAppend.outPort,
|
||||||
// Build in reverse so it yields a more efficient layout for left-to-right building. We cannot
|
inSlots = inSlots + toAppend.inSlots, // we have now more input ports than before
|
||||||
// apply the full traversal, only the completed part of it
|
// the inOffset of _this_ gets shifted by toAppend.inSlots, because the traversal of toAppend is _prepended_
|
||||||
traversalSoFar = newTraversalSoFar,
|
inOffset = inOffset + toAppend.inSlots,
|
||||||
// Last composite of toAppend is still pending
|
// Build in reverse so it yields a more efficient layout for left-to-right building. We cannot
|
||||||
pendingBuilder = toAppend.pendingBuilder,
|
// apply the full traversal, only the completed part of it
|
||||||
attributes = Attributes.none, // attributes are none for the new enclosing builder
|
traversalSoFar = newTraversalSoFar,
|
||||||
beforeBuilder = newBeforeTraversal, // no need for beforeBuilder as there are no composites
|
// Last composite of toAppend is still pending
|
||||||
islandTag = OptionVal.None // islandTag is reset for the new enclosing builder
|
pendingBuilder = toAppend.pendingBuilder,
|
||||||
)
|
attributes = Attributes.none, // attributes are none for the new enclosing builder
|
||||||
|
beforeBuilder = newBeforeTraversal, // no need for beforeBuilder as there are no composites
|
||||||
|
islandTag = OptionVal.None // islandTag is reset for the new enclosing builder
|
||||||
|
)
|
||||||
}
|
}
|
||||||
} else throw new Exception("should this happen?")
|
} else throw new Exception("should this happen?")
|
||||||
|
|
||||||
|
|
@ -1082,9 +1098,11 @@ import akka.util.unused
|
||||||
* between islands.
|
* between islands.
|
||||||
*/
|
*/
|
||||||
override def makeIsland(islandTag: IslandTag): LinearTraversalBuilder =
|
override def makeIsland(islandTag: IslandTag): LinearTraversalBuilder =
|
||||||
if (this.islandTag.isDefined)
|
this.islandTag match {
|
||||||
this // Wrapping with an island, then immediately re-wrapping makes the second island empty, so can be omitted
|
case OptionVal.Some(_) =>
|
||||||
else copy(islandTag = OptionVal.Some(islandTag))
|
this // Wrapping with an island, then immediately re-wrapping makes the second island empty, so can be omitted
|
||||||
|
case _ => copy(islandTag = OptionVal.Some(islandTag))
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -1188,9 +1206,10 @@ import akka.util.unused
|
||||||
remaining = remaining.tail
|
remaining = remaining.tail
|
||||||
}
|
}
|
||||||
|
|
||||||
val finalTraversal =
|
val finalTraversal = islandTag match {
|
||||||
if (islandTag.isDefined) EnterIsland(islandTag.get).concat(traversal).concat(ExitIsland)
|
case OptionVal.Some(tag) => EnterIsland(tag).concat(traversal).concat(ExitIsland)
|
||||||
else traversal
|
case _ => traversal
|
||||||
|
}
|
||||||
|
|
||||||
// The CompleteTraversalBuilder only keeps the minimum amount of necessary information that is needed for it
|
// The CompleteTraversalBuilder only keeps the minimum amount of necessary information that is needed for it
|
||||||
// to be embedded in a larger graph, making partial graph reuse much more efficient.
|
// to be embedded in a larger graph, making partial graph reuse much more efficient.
|
||||||
|
|
@ -1325,8 +1344,10 @@ import akka.util.unused
|
||||||
}
|
}
|
||||||
|
|
||||||
override def makeIsland(islandTag: IslandTag): TraversalBuilder = {
|
override def makeIsland(islandTag: IslandTag): TraversalBuilder = {
|
||||||
if (this.islandTag.isDefined)
|
this.islandTag match {
|
||||||
this // Wrapping with an island, then immediately re-wrapping makes the second island empty, so can be omitted
|
case OptionVal.None => copy(islandTag = OptionVal(islandTag))
|
||||||
else copy(islandTag = OptionVal(islandTag))
|
case _ =>
|
||||||
|
this // Wrapping with an island, then immediately re-wrapping makes the second island empty, so can be omitted
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue