replace unicode arrows

* ⇒, →, ←
* because we don't want to show them in documentation snippets and
  then it's complicated to avoid that when snippets are
  located in src/test/scala in individual modules
* dont replace object `→` in FSM.scala and PersistentFSM.scala
This commit is contained in:
Patrik Nordwall 2019-02-09 15:25:39 +01:00
parent e4d38f92a4
commit 5c96a5f556
1521 changed files with 18846 additions and 18786 deletions

View file

@ -41,30 +41,30 @@ import akka.util.OptionVal
}
val DefaultPhases: Map[IslandTag, Phase[Any]] = Map[IslandTag, Phase[Any]](
SinkModuleIslandTag new Phase[Any] {
SinkModuleIslandTag -> new Phase[Any] {
override def apply(settings: ActorMaterializerSettings, effectiveAttributes: Attributes,
materializer: PhasedFusingActorMaterializer,
islandName: String): PhaseIsland[Any] =
new SinkModulePhase(materializer, islandName).asInstanceOf[PhaseIsland[Any]]
},
SourceModuleIslandTag new Phase[Any] {
SourceModuleIslandTag -> new Phase[Any] {
override def apply(settings: ActorMaterializerSettings, effectiveAttributes: Attributes,
materializer: PhasedFusingActorMaterializer,
islandName: String): PhaseIsland[Any] =
new SourceModulePhase(materializer, islandName).asInstanceOf[PhaseIsland[Any]]
},
ProcessorModuleIslandTag new Phase[Any] {
ProcessorModuleIslandTag -> new Phase[Any] {
override def apply(settings: ActorMaterializerSettings, effectiveAttributes: Attributes,
materializer: PhasedFusingActorMaterializer,
islandName: String): PhaseIsland[Any] =
new ProcessorModulePhase().asInstanceOf[PhaseIsland[Any]]
},
TlsModuleIslandTag new Phase[Any] {
TlsModuleIslandTag -> new Phase[Any] {
def apply(settings: ActorMaterializerSettings, effectiveAttributes: Attributes,
materializer: PhasedFusingActorMaterializer, islandName: String): PhaseIsland[Any] =
new TlsModulePhase(materializer, islandName).asInstanceOf[PhaseIsland[Any]]
},
GraphStageTag DefaultPhase)
GraphStageTag -> DefaultPhase)
@InternalApi private[akka] def apply()(implicit context: ActorRefFactory): ActorMaterializer = {
val haveShutDown = new AtomicBoolean(false)
@ -86,10 +86,10 @@ import akka.util.OptionVal
private def actorSystemOf(context: ActorRefFactory): ActorSystem = {
val system = context match {
case s: ExtendedActorSystem s
case c: ActorContext c.system
case null throw new IllegalArgumentException("ActorRefFactory context must be defined")
case _
case s: ExtendedActorSystem => s
case c: ActorContext => c.system
case null => throw new IllegalArgumentException("ActorRefFactory context must be defined")
case _ =>
throw new IllegalArgumentException(s"ActorRefFactory context must be an ActorSystem or ActorContext, got [${context.getClass.getName}]")
}
system
@ -393,8 +393,8 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
}
override lazy val executionContext: ExecutionContextExecutor = dispatchers.lookup(settings.dispatcher match {
case Deploy.NoDispatcherGiven Dispatchers.DefaultDispatcherId
case other other
case Deploy.NoDispatcherGiven => Dispatchers.DefaultDispatcherId
case other => other
})
override def schedulePeriodically(initialDelay: FiniteDuration, interval: FiniteDuration, task: Runnable): Cancellable =
@ -446,7 +446,7 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
while (current ne EmptyTraversal) {
var nextStep: Traversal = EmptyTraversal
current match {
case MaterializeAtomic(mod, outToSlot)
case MaterializeAtomic(mod, outToSlot) =>
if (Debug) println(s"materializing module: $mod")
val matAndStage = islandTracking.getCurrentPhase.materializeAtomic(mod, attributesStack.getLast)
val logic = matAndStage._1
@ -461,37 +461,37 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
if (Debug) println(s"PUSH: $matValue => $matValueStack")
case Concat(first, next)
case Concat(first, next) =>
if (next ne EmptyTraversal) traversalStack.add(next)
nextStep = first
case Pop
case Pop =>
val popped = matValueStack.removeLast()
if (Debug) println(s"POP: $popped => $matValueStack")
case PushNotUsed
case PushNotUsed =>
matValueStack.addLast(NotUsed)
if (Debug) println(s"PUSH: NotUsed => $matValueStack")
case transform: Transform
case transform: Transform =>
val prev = matValueStack.removeLast()
val result = transform(prev)
matValueStack.addLast(result)
if (Debug) println(s"TRFM: $matValueStack")
case compose: Compose
case compose: Compose =>
val second = matValueStack.removeLast()
val first = matValueStack.removeLast()
val result = compose(first, second)
matValueStack.addLast(result)
if (Debug) println(s"COMP: $matValueStack")
case PushAttributes(attr)
case PushAttributes(attr) =>
attributesStack.addLast(attributesStack.getLast and attr)
if (Debug) println(s"ATTR PUSH: $attr")
case PopAttributes
case PopAttributes =>
attributesStack.removeLast()
if (Debug) println(s"ATTR POP")
case EnterIsland(tag)
case EnterIsland(tag) =>
islandTracking.enterIsland(tag, attributesStack.getLast)
case ExitIsland
case ExitIsland =>
islandTracking.exitIsland()
case _
case _ =>
}
current = nextStep
}
@ -605,7 +605,7 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
effectiveAttributes: Attributes,
materializer: PhasedFusingActorMaterializer,
islandName: String,
subflowFuser: OptionVal[GraphInterpreterShell ActorRef]) extends PhaseIsland[GraphStageLogic] {
subflowFuser: OptionVal[GraphInterpreterShell => ActorRef]) extends PhaseIsland[GraphStageLogic] {
// TODO: remove these
private val logicArrayType = Array.empty[GraphStageLogic]
private[this] val logics = new util.ArrayList[GraphStageLogic](16)
@ -635,8 +635,8 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
logics.add(logic)
logic.stageId = logics.size() - 1
fullIslandName match {
case OptionVal.Some(_) // already set
case OptionVal.None fullIslandName = OptionVal.Some(islandName + "-" + logic.attributes.nameOrDefault())
case OptionVal.Some(_) => // already set
case OptionVal.None => fullIslandName = OptionVal.Some(islandName + "-" + logic.attributes.nameOrDefault())
}
matAndLogic
}
@ -733,16 +733,16 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
shell.logics = logics.toArray(logicArrayType)
subflowFuser match {
case OptionVal.Some(fuseIntoExistingInterpreter)
case OptionVal.Some(fuseIntoExistingInterpreter) =>
fuseIntoExistingInterpreter(shell)
case _
case _ =>
val props = ActorGraphInterpreter.props(shell).withDispatcher(ActorAttributes.Dispatcher.resolve(effectiveAttributes, settings))
val actorName = fullIslandName match {
case OptionVal.Some(n) n
case OptionVal.None islandName
case OptionVal.Some(n) => n
case OptionVal.None => islandName
}
val ref = materializer.actorOf(props, actorName)
@ -756,10 +756,10 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
val missingHandlerIdx = logic.handlers.indexWhere(_.asInstanceOf[AnyRef] eq null)
val isIn = missingHandlerIdx < logic.inCount
val portLabel = logic.originalStage match {
case OptionVal.Some(stage)
case OptionVal.Some(stage) =>
if (isIn) s"in port [${stage.shape.inlets(missingHandlerIdx)}]"
else s"out port [${stage.shape.outlets(missingHandlerIdx - logic.inCount)}"
case OptionVal.None
case OptionVal.None =>
if (isIn) s"in port id [$missingHandlerIdx]"
else s"out port id [$missingHandlerIdx]"
}
@ -832,8 +832,8 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
override def takePublisher(slot: Int, publisher: Publisher[Any]): Unit = {
subscriberOrVirtualPublisher match {
case v: VirtualPublisher[_] v.registerPublisher(publisher)
case s: Subscriber[Any] @unchecked publisher.subscribe(s)
case v: VirtualPublisher[_] => v.registerPublisher(publisher)
case s: Subscriber[Any] @unchecked => publisher.subscribe(s)
}
}