akka-stream compiling on Scala 3 (#30324)

Changes:
* Private constructor for case class means private apply in Scala 3
* Logger class with Any instead of wildcard where needed
* Explicit import of internal implicit conversion import SinkToCompletionStage
* Hopefully source and binary compatible Scala 3 signature for GraphApply.create methods
* ZipLatestWith leaking private type
* Auto apply-to-lambda made explicit
* Internal async callback event class covariant
* Hub leaking private type
* Remove cycles in stream Buffer classes
* Avoid cyclic import in ResizableMultiReaderRingBuffer
* Safe cast of Subscribers
* Explicit import of internal implicit conversion SourceToCompletionStage
* Ambigous field name and method
* recover delegates using PF from javadsl not inferred, made explicit
* TcpStage completing promise with wrong type
* Collect aggregator empty value cannot be underscore with type ascription
* Some type alias shenanigans in QueueSink removed
* Explicit type on effectiveMaximumBurst in Throttle
* Override method return type not inferred in GraphInterpreter
* MutableCollectorState.accumulated getter got wrong type somehow
* TLS actor fallback case only applicable to null
* Some internal unchecked pattern matching removed
* Ok that createGraph is not present in Java API
* Separate impl per language for implicit actor system apply
This commit is contained in:
Johan Andrén 2021-07-06 18:01:41 +02:00 committed by GitHub
parent 7f3bfb1a30
commit 3a3e643e07
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
39 changed files with 259 additions and 103 deletions

View file

@ -38,7 +38,7 @@ jobs:
- stage: scala3 - stage: scala3
name: scala3 name: scala3
# separate job since only a few modules compile with Scala 3 yet # separate job since only a few modules compile with Scala 3 yet
script: jabba install adopt@1.11-0 && jabba use adopt@1.11-0 && sbt -Dakka.build.scalaVersion=3.0 "akka-actor-tests/test:compile" akka-actor-typed/compile script: jabba install adopt@1.11-0 && jabba use adopt@1.11-0 && sbt -Dakka.build.scalaVersion=3.0 "akka-actor-tests/test:compile" akka-actor-typed/compile akka-stream/compile
stages: stages:
- name: whitesource - name: whitesource

View file

@ -29,7 +29,8 @@ class DslFactoriesConsistencySpec extends AnyWordSpec with Matchers {
"lazyFutureFlow", // lazyCompletionStageFlow "lazyFutureFlow", // lazyCompletionStageFlow
"futureFlow", // completionStageFlow "futureFlow", // completionStageFlow
"futureSink", // completionStageSink "futureSink", // completionStageSink
"lazyFutureSink" // lazyCompletionStageSink "lazyFutureSink", // lazyCompletionStageSink
"createGraph" // renamed/overload of create for getting type inference working in Scala 3
) )
val javaIgnore = val javaIgnore =

View file

@ -22,7 +22,7 @@ private[stream] abstract class GraphCreate {
*/ */
def create[S1 <: Shape, S <: Shape, M](g1: Graph[S1, M], def create[S1 <: Shape, S <: Shape, M](g1: Graph[S1, M],
block: function.Function2[GraphDSL.Builder[M], S1, S]): Graph[S, M] = block: function.Function2[GraphDSL.Builder[M], S1, S]): Graph[S, M] =
scaladsl.GraphDSL.create(g1) { b => s => block.apply(b.asJava, s) } scaladsl.GraphDSL.createGraph(g1) { b => s => block.apply(b.asJava, s) }
/** /**
* Creates a new [[Graph]] by importing the given graphs and passing their [[Shape]]s * Creates a new [[Graph]] by importing the given graphs and passing their [[Shape]]s
@ -30,7 +30,7 @@ private[stream] abstract class GraphCreate {
*/ */
def create[S1 <: Shape, S2 <: Shape, S <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M], def create[S1 <: Shape, S2 <: Shape, S <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M],
block: function.Function3[GraphDSL.Builder[M], S1, S2, S]): Graph[S, M] = block: function.Function3[GraphDSL.Builder[M], S1, S2, S]): Graph[S, M] =
scaladsl.GraphDSL.create(g1, g2)(combineMat.apply) { b => (s1, s2) => block.apply(b.asJava, s1, s2) } scaladsl.GraphDSL.createGraph(g1, g2)(combineMat.apply) { b => (s1, s2) => block.apply(b.asJava, s1, s2) }
[3..21#/** [3..21#/**
* Creates a new [[Graph]] by importing the given graphs and passing their [[Shape]]s * Creates a new [[Graph]] by importing the given graphs and passing their [[Shape]]s
@ -38,7 +38,7 @@ private[stream] abstract class GraphCreate {
*/ */
def create1[[#S1 <: Shape#], S <: Shape, [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M], def create1[[#S1 <: Shape#], S <: Shape, [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M],
block: function.Function2[GraphDSL.Builder[M], [#S1#], S]): Graph[S, M] = block: function.Function2[GraphDSL.Builder[M], [#S1#], S]): Graph[S, M] =
scaladsl.GraphDSL.create([#g1#])(combineMat.apply) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) }# scaladsl.GraphDSL.createGraph([#g1#])(combineMat.apply) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) }#
] ]
} }

View file

@ -21,6 +21,8 @@ trait GraphApply {
/** /**
* Creates a new [[Graph]] by importing the given graph `g1` and passing its [[Shape]] * Creates a new [[Graph]] by importing the given graph `g1` and passing its [[Shape]]
* along with the [[GraphDSL.Builder]] to the given create function. * along with the [[GraphDSL.Builder]] to the given create function.
*
* Deprecated: this method signature does not work with Scala 3 type inference, kept for binary compatiblity. Use createGraph instead.
*/ */
def create[S <: Shape, Mat](g1: Graph[Shape, Mat])(buildBlock: GraphDSL.Builder[Mat] => (g1.Shape) => S): Graph[S, Mat] = { def create[S <: Shape, Mat](g1: Graph[Shape, Mat])(buildBlock: GraphDSL.Builder[Mat] => (g1.Shape) => S): Graph[S, Mat] = {
val builder = new GraphDSL.Builder val builder = new GraphDSL.Builder
@ -30,12 +32,26 @@ trait GraphApply {
createGraph(s, builder) createGraph(s, builder)
} }
/**
* Creates a new [[Graph]] by importing the given graph `g1` and passing its [[Shape]]
* along with the [[GraphDSL.Builder]] to the given create function.
*/
def createGraph[S <: Shape, S1 <: Shape, Mat](g1: Graph[S1, Mat])(buildBlock: GraphDSL.Builder[Mat] => S1 => S): Graph[S, Mat] = {
val builder = new GraphDSL.Builder
val s1 = builder.add(g1, Keep.right)
val s = buildBlock(builder)(s1)
createGraph(s, builder)
}
[2..# [2..#
/** /**
* Creates a new [[Graph]] by importing the given graphs and passing their [[Shape]]s * Creates a new [[Graph]] by importing the given graphs and passing their [[Shape]]s
* along with the [[GraphDSL.Builder]] to the given create function. * along with the [[GraphDSL.Builder]] to the given create function.
*
* Deprecated: this method signature does not work with Scala 3 type inference, kept for binary compatiblity. Use createGraph instead.
*/ */
def create[S <: Shape, Mat, [#M1#]]([#g1: Graph[Shape, M1]#])(combineMat: ([#M1#]) => Mat)(buildBlock: GraphDSL.Builder[Mat] => ([#g1.Shape#]) => S): Graph[S, Mat] = { def create[S <: Shape, Mat, [#M1#]]([#g1: Graph[Shape, M1]#])(combineMat: ([#M1#]) => Mat)(buildBlock: GraphDSL.Builder[Mat] => ([#g1.Shape#]) => S): Graph[S, Mat] = {
val builder = new GraphDSL.Builder val builder = new GraphDSL.Builder
@ -46,8 +62,22 @@ trait GraphApply {
val s = buildBlock(builder)([#s1#]) val s = buildBlock(builder)([#s1#])
createGraph(s, builder) createGraph(s, builder)
}# }
/**
* Creates a new [[Graph]] by importing the given graphs and passing their [[Shape]]s
* along with the [[GraphDSL.Builder]] to the given create function.
*/
def createGraph[S <: Shape, Mat, [#M1#], [#S1 <: Shape#]]([#g1: Graph[S1, M1]#])(combineMat: ([#M1#]) => Mat)(buildBlock: GraphDSL.Builder[Mat] => ([#S1#]) => S): Graph[S, Mat] = {
val builder = new GraphDSL.Builder
val curried = combineMat.curried
val s##1 = builder.add(g##1, (m##1: M##1) => curried(m##1))
[2..#val s1 = builder.add(g1, (f: M1 => Any, m1: M1) => f(m1))#
]
val s = buildBlock(builder)([#s1#])
createGraph(s, builder)
}#
] ]
private def createGraph[S <: Shape, Mat](shape: S, graphBuilder: GraphDSL.Builder[Mat]): Graph[S, Mat] = private def createGraph[S <: Shape, Mat](shape: S, graphBuilder: GraphDSL.Builder[Mat]): Graph[S, Mat] =

View file

@ -36,7 +36,7 @@ class ZipLatestWith1[[#A1#], O] (val zipper: ([#A1#]) => O) extends GraphStage[F
private var willShutDown = false private var willShutDown = false
private val contextPropagation = ContextPropagation() private val contextPropagation = ContextPropagation()
[#val inlet0 = new ZipLatestInlet(in0)# [#private val inlet0 = new ZipLatestInlet(in0)#
] ]
private var waitingForTuple = false private var waitingForTuple = false
private var staleTupleValues = true private var staleTupleValues = true

View file

@ -0,0 +1,8 @@
# internal
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.scaladsl.MergeHub#MergedSourceLogic.enqueue")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.BoundedBuffer#FixedQueue.this")
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.BoundedBuffer#DynamicQueue.this")
ProblemFilters.exclude[IncompatibleSignatureProblem]("akka.stream.impl.PhasedFusingActorMaterializer.makeLogger")
# Effectively internal
ProblemFilters.exclude[IncompatibleSignatureProblem]("akka.stream.MaterializerLoggingProvider.makeLogger")

View file

@ -0,0 +1,35 @@
/*
* Copyright (C) 2009-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.stream.impl
import akka.actor.ActorSystem
import akka.actor.ExtensionId
import akka.annotation.DoNotInherit
import akka.stream.scaladsl.Tcp
import com.typesafe.sslconfig.akka.AkkaSSLConfig
import scala.annotation.nowarn
/*
* Some extensions here provide an apply that takes an implicit actor system which needs different slightly syntax to define
* on Scala 2 and Scala 3
*/
/**
* Not for user extension
*/
@DoNotInherit
trait TcpImplicitExtensionIdApply extends ExtensionId[Tcp] {
def apply()(implicit system: ActorSystem): Tcp = super.apply(system)
}
/**
* Not for user extension
*/
@DoNotInherit
@nowarn("msg=deprecated")
trait AkkaSSLConfigExtensionIdApply extends ExtensionId[AkkaSSLConfig] {
def apply()(implicit system: ActorSystem): AkkaSSLConfig = super.apply(system)
}

View file

@ -0,0 +1,35 @@
/*
* Copyright (C) 2009-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.stream.impl
import akka.actor.ActorSystem
import akka.actor.ExtensionId
import akka.annotation.DoNotInherit
import akka.stream.scaladsl.Tcp
import com.typesafe.sslconfig.akka.AkkaSSLConfig
import scala.annotation.nowarn
/*
* Some extensions here provide an apply that takes an implicit actor system which needs different slightly syntax to define
* on Scala 2 and Scala 3
*/
/**
* Not for user extension
*/
@DoNotInherit
trait TcpImplicitExtensionIdApply extends ExtensionId[Tcp] {
def apply()(implicit system: ActorSystem): Tcp = super.apply(system)
}
/**
* Not for user extension
*/
@DoNotInherit
@nowarn("msg=deprecated")
trait AkkaSSLConfigExtensionIdApply extends ExtensionId[AkkaSSLConfig] {
def apply()(implicit system: ActorSystem): AkkaSSLConfig = super.apply(system)
}

View file

@ -0,0 +1,35 @@
/*
* Copyright (C) 2009-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.stream.impl
import akka.actor.ActorSystem
import akka.actor.ExtensionId
import akka.annotation.DoNotInherit
import akka.stream.scaladsl.Tcp
import com.typesafe.sslconfig.akka.AkkaSSLConfig
import scala.annotation.nowarn
/*
* Some extensions here provide an apply that takes an implicit actor system which needs different slightly syntax to define
* on Scala 2 and Scala 3
*/
/**
* Not for user extension
*/
@DoNotInherit
trait TcpImplicitExtensionIdApply extends ExtensionId[Tcp] {
override def apply(implicit system: ActorSystem): Tcp = super.apply(system)
}
/**
* Not for user extension
*/
@DoNotInherit
@nowarn("msg=deprecated")
trait AkkaSSLConfigExtensionIdApply extends ExtensionId[AkkaSSLConfig] {
override def apply(implicit system: ActorSystem): AkkaSSLConfig = super.apply(system)
}

View file

@ -4,14 +4,15 @@
package akka.stream package akka.stream
import akka.annotation.DoNotInherit
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
/** /**
* SPI intended only to be extended by custom [[Materializer]] implementations, * Not for user extension
* that also want to provide operators they materialize with specialized [[akka.event.LoggingAdapter]] instances.
*/ */
@DoNotInherit
trait MaterializerLoggingProvider { this: Materializer => trait MaterializerLoggingProvider { this: Materializer =>
def makeLogger(logSource: Class[_]): LoggingAdapter def makeLogger(logSource: Class[Any]): LoggingAdapter
} }

View file

@ -233,7 +233,7 @@ import akka.util.unused
protected def downstreamRunning: Actor.Receive = { protected def downstreamRunning: Actor.Receive = {
case SubscribePending => case SubscribePending =>
subscribePending(exposedPublisher.takePendingSubscribers()) subscribePending(exposedPublisher.takePendingSubscribers().asInstanceOf[Seq[Subscriber[Any]]])
case RequestMore(_, elements) => case RequestMore(_, elements) =>
if (elements < 1) { if (elements < 1) {
error(ReactiveStreamsCompliance.numberOfElementsInRequestMustBePositiveException) error(ReactiveStreamsCompliance.numberOfElementsInRequestMustBePositiveException)

View file

@ -143,27 +143,42 @@ private[akka] object Buffer {
*/ */
@InternalApi private[akka] final class BoundedBuffer[T](val capacity: Int) extends Buffer[T] { @InternalApi private[akka] final class BoundedBuffer[T](val capacity: Int) extends Buffer[T] {
import BoundedBuffer._
def used: Int = q.used def used: Int = q.used
def isFull: Boolean = q.isFull def isFull: Boolean = q.isFull
def isEmpty: Boolean = q.isEmpty def isEmpty: Boolean = q.isEmpty
def nonEmpty: Boolean = q.nonEmpty def nonEmpty: Boolean = q.nonEmpty
def enqueue(elem: T): Unit = q.enqueue(elem) def enqueue(elem: T): Unit = q.enqueue(elem)
def dequeue(): T = q.dequeue() def dequeue(): T = q.dequeue()
def peek(): T = q.peek() def peek(): T = q.peek()
def clear(): Unit = q.clear() def clear(): Unit = q.clear()
def dropHead(): Unit = q.dropHead() def dropHead(): Unit = q.dropHead()
def dropTail(): Unit = q.dropTail() def dropTail(): Unit = q.dropTail()
private final class FixedQueue extends Buffer[T] { private var q: Buffer[T] = new FixedQueue[T](capacity, newBuffer => q = newBuffer)
}
/**
* INTERNAL API
*/
@InternalApi private[akka] object BoundedBuffer {
private final class FixedQueue[T](override val capacity: Int, switchBuffer: Buffer[T] => Unit) extends Buffer[T] {
import Buffer._ import Buffer._
private val queue = new Array[AnyRef](FixedQueueSize) private val queue = new Array[AnyRef](FixedQueueSize)
private var head = 0 private var head = 0
private var tail = 0 private var tail = 0
override def capacity = BoundedBuffer.this.capacity
override def used = tail - head override def used = tail - head
override def isFull = used == capacity override def isFull = used == capacity
override def isEmpty = tail == head override def isEmpty = tail == head
@ -171,11 +186,11 @@ private[akka] object Buffer {
override def enqueue(elem: T): Unit = override def enqueue(elem: T): Unit =
if (tail - head == FixedQueueSize) { if (tail - head == FixedQueueSize) {
val queue = new DynamicQueue() val queue = new DynamicQueue[T](capacity)
while (nonEmpty) { while (nonEmpty) {
queue.enqueue(dequeue()) queue.enqueue(dequeue())
} }
q = queue switchBuffer(queue)
queue.enqueue(elem) queue.enqueue(elem)
} else { } else {
queue(tail & FixedQueueMask) = elem.asInstanceOf[AnyRef] queue(tail & FixedQueueMask) = elem.asInstanceOf[AnyRef]
@ -203,8 +218,7 @@ private[akka] object Buffer {
} }
} }
private final class DynamicQueue() extends ju.LinkedList[T] with Buffer[T] { private final class DynamicQueue[T](override val capacity: Int) extends ju.LinkedList[T] with Buffer[T] {
override def capacity = BoundedBuffer.this.capacity
override def used = size override def used = size
override def isFull = size == capacity override def isFull = size == capacity
override def nonEmpty = !isEmpty() override def nonEmpty = !isEmpty()
@ -215,6 +229,4 @@ private[akka] object Buffer {
override def dropHead(): Unit = remove() override def dropHead(): Unit = remove()
override def dropTail(): Unit = removeLast() override def dropTail(): Unit = removeLast()
} }
private var q: Buffer[T] = new FixedQueue
} }

View file

@ -610,7 +610,7 @@ private final case class SavedIslandData(
} }
} }
override def makeLogger(logSource: Class[_]): LoggingAdapter = override def makeLogger(logSource: Class[Any]): LoggingAdapter =
Logging(system, logSource) Logging(system, logSource)
/** /**

View file

@ -7,7 +7,7 @@ package akka.stream.impl
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import ResizableMultiReaderRingBuffer._ import ResizableMultiReaderRingBuffer.{ Cursor, Cursors, NothingToReadException }
import akka.annotation.InternalApi import akka.annotation.InternalApi

View file

@ -299,8 +299,6 @@ import akka.util.ccompat._
require(maxConcurrentPulls > 0, "Max concurrent pulls must be greater than 0") require(maxConcurrentPulls > 0, "Max concurrent pulls must be greater than 0")
type Requested[E] = Promise[Option[E]]
val in = Inlet[T]("queueSink.in") val in = Inlet[T]("queueSink.in")
override def initialAttributes = DefaultAttributes.queueSink override def initialAttributes = DefaultAttributes.queueSink
override val shape: SinkShape[T] = SinkShape.of(in) override val shape: SinkShape[T] = SinkShape.of(in)
@ -309,14 +307,13 @@ import akka.util.ccompat._
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = { override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
val stageLogic = new GraphStageLogic(shape) with InHandler with SinkQueueWithCancel[T] { val stageLogic = new GraphStageLogic(shape) with InHandler with SinkQueueWithCancel[T] {
type Received[E] = Try[Option[E]]
val maxBuffer = inheritedAttributes.get[InputBuffer](InputBuffer(16, 16)).max val maxBuffer = inheritedAttributes.get[InputBuffer](InputBuffer(16, 16)).max
require(maxBuffer > 0, "Buffer size must be greater than 0") require(maxBuffer > 0, "Buffer size must be greater than 0")
// Allocates one additional element to hold stream closed/failure indicators // Allocates one additional element to hold stream closed/failure indicators
val buffer: Buffer[Received[T]] = Buffer(maxBuffer + 1, inheritedAttributes) val buffer: Buffer[Try[Option[T]]] = Buffer(maxBuffer + 1, inheritedAttributes)
val currentRequests: Buffer[Requested[T]] = Buffer(maxConcurrentPulls, inheritedAttributes) val currentRequests: Buffer[Promise[Option[T]]] = Buffer(maxConcurrentPulls, inheritedAttributes)
override def preStart(): Unit = { override def preStart(): Unit = {
setKeepGoing(true) setKeepGoing(true)
@ -324,7 +321,7 @@ import akka.util.ccompat._
} }
private val callback = getAsyncCallback[Output[T]] { private val callback = getAsyncCallback[Output[T]] {
case QueueSink.Pull(pullPromise) => case QueueSink.Pull(pullPromise: Promise[Option[T]] @unchecked) =>
if (currentRequests.isFull) if (currentRequests.isFull)
pullPromise.failure( pullPromise.failure(
new IllegalStateException(s"Too many concurrent pulls. Specified maximum is $maxConcurrentPulls. " + new IllegalStateException(s"Too many concurrent pulls. Specified maximum is $maxConcurrentPulls. " +
@ -337,7 +334,7 @@ import akka.util.ccompat._
case QueueSink.Cancel => completeStage() case QueueSink.Cancel => completeStage()
} }
def sendDownstream(promise: Requested[T]): Unit = { def sendDownstream(promise: Promise[Option[T]]): Unit = {
val e = buffer.dequeue() val e = buffer.dequeue()
promise.complete(e) promise.complete(e)
e match { e match {
@ -445,17 +442,19 @@ import akka.util.ccompat._
@InternalApi private[akka] final class MutableCollectorState[T, R]( @InternalApi private[akka] final class MutableCollectorState[T, R](
collector: java.util.stream.Collector[T, Any, R], collector: java.util.stream.Collector[T, Any, R],
accumulator: java.util.function.BiConsumer[Any, T], accumulator: java.util.function.BiConsumer[Any, T],
val accumulated: Any) _accumulated: Any)
extends CollectorState[T, R] { extends CollectorState[T, R] {
override def accumulated(): Any = _accumulated
override def update(elem: T): CollectorState[T, R] = { override def update(elem: T): CollectorState[T, R] = {
accumulator.accept(accumulated, elem) accumulator.accept(_accumulated, elem)
this this
} }
override def finish(): R = { override def finish(): R = {
// only called if completed without elements // only called if completed without elements
collector.finisher().apply(accumulated) collector.finisher().apply(_accumulated)
} }
} }

View file

@ -40,7 +40,7 @@ import akka.util.NanoTimeTokenBucket
private val nanosBetweenTokens = per.toNanos / cost private val nanosBetweenTokens = per.toNanos / cost
// 100 ms is a realistic minimum between tokens, otherwise the maximumBurst is adjusted // 100 ms is a realistic minimum between tokens, otherwise the maximumBurst is adjusted
// to be able to support higher rates // to be able to support higher rates
val effectiveMaximumBurst = val effectiveMaximumBurst: Long =
if (maximumBurst == Throttle.AutomaticMaximumBurst) math.max(1, ((100 * 1000 * 1000) / nanosBetweenTokens)) if (maximumBurst == Throttle.AutomaticMaximumBurst) math.max(1, ((100 * 1000 * 1000) / nanosBetweenTokens))
else maximumBurst else maximumBurst
require(!(mode == ThrottleMode.Enforcing && effectiveMaximumBurst < 0), "maximumBurst must be > 0 in Enforcing mode") require(!(mode == ThrottleMode.Enforcing && effectiveMaximumBurst < 0), "maximumBurst must be > 0 in Enforcing mode")

View file

@ -109,7 +109,7 @@ import akka.stream.stage._
* when this accidentally leaks onto threads that are not stopped when this * when this accidentally leaks onto threads that are not stopped when this
* class should be unloaded. * class should be unloaded.
*/ */
override def initialValue = new Array(1) override def initialValue: Array[AnyRef] = new Array(1)
} }
/** /**

View file

@ -2085,6 +2085,7 @@ private[stream] object Collect {
override def toString = s"Reduce.Logic(aggregator=$aggregator)" override def toString = s"Reduce.Logic(aggregator=$aggregator)"
private var aggregator: T = _ private var aggregator: T = _
private val empty: T = aggregator
private def decider = private def decider =
inheritedAttributes.mandatoryAttribute[SupervisionStrategy].decider inheritedAttributes.mandatoryAttribute[SupervisionStrategy].decider
@ -2113,7 +2114,7 @@ private[stream] object Collect {
decider(ex) match { decider(ex) match {
case Supervision.Stop => failStage(ex) case Supervision.Stop => failStage(ex)
case Supervision.Restart => case Supervision.Restart =>
aggregator = _: T aggregator = empty
setInitialInHandler() setInitialInHandler()
case _ => () case _ => ()

View file

@ -425,7 +425,7 @@ import akka.util.ByteString
case BUFFER_OVERFLOW => case BUFFER_OVERFLOW =>
flushToUser() flushToUser()
transportInChoppingBlock.putBack(transportInBuffer) transportInChoppingBlock.putBack(transportInBuffer)
case s => fail(new IllegalStateException(s"unexpected status $s in doUnwrap()")) case null => fail(new IllegalStateException(s"unexpected status 'null' in doUnwrap()"))
} }
} }

View file

@ -177,7 +177,7 @@ import akka.util.ByteString
private def unbindCompleted(): Unit = { private def unbindCompleted(): Unit = {
stageActor.unwatch(listener) stageActor.unwatch(listener)
unbindPromise.trySuccess(Done) unbindPromise.trySuccess(())
if (connectionFlowsAwaitingInitialization.get() == 0) completeStage() if (connectionFlowsAwaitingInitialization.get() == 0) completeStage()
else scheduleOnce(BindShutdownTimer, bindShutdownTimeout) else scheduleOnce(BindShutdownTimer, bindShutdownTimeout)
} }
@ -192,7 +192,7 @@ import akka.util.ByteString
override def postStop(): Unit = { override def postStop(): Unit = {
// a bit unexpected to succeed here rather than fail with abrupt stage termination // a bit unexpected to succeed here rather than fail with abrupt stage termination
// but there was an existing test case covering this behavior // but there was an existing test case covering this behavior
unbindPromise.trySuccess(Done) unbindPromise.trySuccess(())
bindingPromise.tryFailure(new NoSuchElementException("Binding was unbound before it was completely finished")) bindingPromise.tryFailure(new NoSuchElementException("Binding was unbound before it was completely finished"))
} }
} }

View file

@ -12,7 +12,7 @@ import akka.stream.StreamRefSettings
/** INTERNAL API */ /** INTERNAL API */
@InternalApi @InternalApi
private[akka] final case class StreamRefSettingsImpl private ( private[akka] final case class StreamRefSettingsImpl(
override val bufferCapacity: Int, override val bufferCapacity: Int,
override val demandRedeliveryInterval: FiniteDuration, override val demandRedeliveryInterval: FiniteDuration,
override val subscriptionTimeout: FiniteDuration, override val subscriptionTimeout: FiniteDuration,

View file

@ -25,8 +25,8 @@ object BidiFlow {
*/ */
def fromGraph[I1, O1, I2, O2, M](g: Graph[BidiShape[I1, O1, I2, O2], M]): BidiFlow[I1, O1, I2, O2, M] = def fromGraph[I1, O1, I2, O2, M](g: Graph[BidiShape[I1, O1, I2, O2], M]): BidiFlow[I1, O1, I2, O2, M] =
g match { g match {
case bidi: BidiFlow[I1, O1, I2, O2, M] => bidi case bidi: BidiFlow[I1, O1, I2, O2, M] @unchecked => bidi
case other => new BidiFlow(scaladsl.BidiFlow.fromGraph(other)) case other => new BidiFlow(scaladsl.BidiFlow.fromGraph(other))
} }
/** /**

View file

@ -12,6 +12,8 @@ import java.util.concurrent.CompletionStage
import akka.stream.{ javadsl, scaladsl, IOResult } import akka.stream.{ javadsl, scaladsl, IOResult }
import akka.util.ByteString import akka.util.ByteString
import akka.util.ccompat.JavaConverters._ import akka.util.ccompat.JavaConverters._
import akka.stream.scaladsl.SinkToCompletionStage
import akka.stream.scaladsl.SourceToCompletionStage
/** /**
* Java API: Factories to create sinks and sources from files * Java API: Factories to create sinks and sources from files

View file

@ -64,8 +64,8 @@ object Flow {
*/ */
def fromGraph[I, O, M](g: Graph[FlowShape[I, O], M]): Flow[I, O, M] = def fromGraph[I, O, M](g: Graph[FlowShape[I, O], M]): Flow[I, O, M] =
g match { g match {
case f: Flow[I, O, M] => f case f: Flow[I, O, M] @unchecked => f
case other => new Flow(scaladsl.Flow.fromGraph(other)) case other => new Flow(scaladsl.Flow.fromGraph(other))
} }
/** /**
@ -1679,9 +1679,9 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
def recoverWith( def recoverWith(
clazz: Class[_ <: Throwable], clazz: Class[_ <: Throwable],
supplier: Supplier[Graph[SourceShape[Out], NotUsed]]): javadsl.Flow[In, Out, Mat] = supplier: Supplier[Graph[SourceShape[Out], NotUsed]]): javadsl.Flow[In, Out, Mat] =
recoverWith { recoverWith({
case elem if clazz.isInstance(elem) => supplier.get() case elem if clazz.isInstance(elem) => supplier.get()
} }: PartialFunction[Throwable, Graph[SourceShape[Out], NotUsed]])
/** /**
* RecoverWithRetries allows to switch to alternative Source on flow failure. It will stay in effect after * RecoverWithRetries allows to switch to alternative Source on flow failure. It will stay in effect after

View file

@ -668,15 +668,15 @@ object GraphDSL extends GraphCreate {
def to[I, O](j: UniformFanInShape[I, O]): ReverseOps[I] = new ReverseOps(findIn(delegate, j, 0)) def to[I, O](j: UniformFanInShape[I, O]): ReverseOps[I] = new ReverseOps(findIn(delegate, j, 0))
def to[I, O](j: UniformFanOutShape[I, O]): ReverseOps[I] = new ReverseOps(j.in) def to[I, O](j: UniformFanOutShape[I, O]): ReverseOps[I] = new ReverseOps(j.in)
final class ForwardOps[T](out: Outlet[T]) { final class ForwardOps[T](_out: Outlet[T]) {
def toInlet(in: Inlet[_ >: T]): Builder[Mat] = { out ~> in; self } def toInlet(in: Inlet[_ >: T]): Builder[Mat] = { _out ~> in; self }
def to(dst: SinkShape[_ >: T]): Builder[Mat] = { out ~> dst; self } def to(dst: SinkShape[_ >: T]): Builder[Mat] = { _out ~> dst; self }
def toFanIn[U](j: UniformFanInShape[_ >: T, U]): Builder[Mat] = { out ~> j; self } def toFanIn[U](j: UniformFanInShape[_ >: T, U]): Builder[Mat] = { _out ~> j; self }
def toFanOut[U](j: UniformFanOutShape[_ >: T, U]): Builder[Mat] = { out ~> j; self } def toFanOut[U](j: UniformFanOutShape[_ >: T, U]): Builder[Mat] = { _out ~> j; self }
def via[U](f: FlowShape[_ >: T, U]): ForwardOps[U] = from((out ~> f).outlet) def via[U](f: FlowShape[_ >: T, U]): ForwardOps[U] = from((_out ~> f).outlet)
def viaFanIn[U](j: UniformFanInShape[_ >: T, U]): ForwardOps[U] = from((out ~> j).outlet) def viaFanIn[U](j: UniformFanInShape[_ >: T, U]): ForwardOps[U] = from((_out ~> j).outlet)
def viaFanOut[U](j: UniformFanOutShape[_ >: T, U]): ForwardOps[U] = from((out ~> j).outlet) def viaFanOut[U](j: UniformFanOutShape[_ >: T, U]): ForwardOps[U] = from((_out ~> j).outlet)
def out(): Outlet[T] = out def out(): Outlet[T] = _out
} }
final class ReverseOps[T](out: Inlet[T]) { final class ReverseOps[T](out: Inlet[T]) {

View file

@ -25,6 +25,7 @@ import akka.japi.function
import akka.japi.function.Creator import akka.japi.function.Creator
import akka.stream.{ javadsl, scaladsl, _ } import akka.stream.{ javadsl, scaladsl, _ }
import akka.stream.impl.LinearTraversalBuilder import akka.stream.impl.LinearTraversalBuilder
import akka.stream.scaladsl.SinkToCompletionStage
/** Java API */ /** Java API */
object Sink { object Sink {
@ -313,8 +314,8 @@ object Sink {
*/ */
def fromGraph[T, M](g: Graph[SinkShape[T], M]): Sink[T, M] = def fromGraph[T, M](g: Graph[SinkShape[T], M]): Sink[T, M] =
g match { g match {
case s: Sink[T, M] => s case s: Sink[T, M] @unchecked => s
case other => new Sink(scaladsl.Sink.fromGraph(other)) case other => new Sink(scaladsl.Sink.fromGraph(other))
} }
/** /**

View file

@ -622,7 +622,7 @@ object Source {
*/ */
def fromGraph[T, M](g: Graph[SourceShape[T], M]): Source[T, M] = def fromGraph[T, M](g: Graph[SourceShape[T], M]): Source[T, M] =
g match { g match {
case s: Source[T, M] => s case s: Source[T, M] @unchecked => s
case s if s eq scaladsl.Source.empty => empty().asInstanceOf[Source[T, M]] case s if s eq scaladsl.Source.empty => empty().asInstanceOf[Source[T, M]]
case other => new Source(scaladsl.Source.fromGraph(other)) case other => new Source(scaladsl.Source.fromGraph(other))
} }
@ -2131,9 +2131,9 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
def recoverWith( def recoverWith(
clazz: Class[_ <: Throwable], clazz: Class[_ <: Throwable],
supplier: Supplier[Graph[SourceShape[Out], NotUsed]]): Source[Out, Mat] = supplier: Supplier[Graph[SourceShape[Out], NotUsed]]): Source[Out, Mat] =
recoverWith { recoverWith({
case elem if clazz.isInstance(elem) => supplier.get() case elem if clazz.isInstance(elem) => supplier.get()
} }: PartialFunction[Throwable, Graph[SourceShape[Out], NotUsed]])
/** /**
* RecoverWithRetries allows to switch to alternative Source on flow failure. It will stay in effect after * RecoverWithRetries allows to switch to alternative Source on flow failure. It will stay in effect after
@ -2195,7 +2195,7 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
supplier: Supplier[Graph[SourceShape[Out], NotUsed]]): Source[Out, Mat] = supplier: Supplier[Graph[SourceShape[Out], NotUsed]]): Source[Out, Mat] =
recoverWithRetries(attempts, { recoverWithRetries(attempts, {
case elem if clazz.isInstance(elem) => supplier.get() case elem if clazz.isInstance(elem) => supplier.get()
}) }: PartialFunction[Throwable, Graph[SourceShape[Out], NotUsed]])
/** /**
* Transform each input element into an `Iterable` of output elements that is * Transform each input element into an `Iterable` of output elements that is

View file

@ -17,6 +17,8 @@ import akka.japi.function
import akka.stream.{ javadsl, scaladsl } import akka.stream.{ javadsl, scaladsl }
import akka.stream.IOResult import akka.stream.IOResult
import akka.util.ByteString import akka.util.ByteString
import akka.stream.scaladsl.SinkToCompletionStage
import akka.stream.scaladsl.SourceToCompletionStage
/** /**
* Converters for interacting with the blocking `java.io` streams APIs and Java 8 Streams * Converters for interacting with the blocking `java.io` streams APIs and Java 8 Streams

View file

@ -232,8 +232,8 @@ object BidiFlow {
*/ */
def fromGraph[I1, O1, I2, O2, Mat](graph: Graph[BidiShape[I1, O1, I2, O2], Mat]): BidiFlow[I1, O1, I2, O2, Mat] = def fromGraph[I1, O1, I2, O2, Mat](graph: Graph[BidiShape[I1, O1, I2, O2], Mat]): BidiFlow[I1, O1, I2, O2, Mat] =
graph match { graph match {
case bidi: BidiFlow[I1, O1, I2, O2, Mat] => bidi case bidi: BidiFlow[I1, O1, I2, O2, Mat] => bidi
case bidi: javadsl.BidiFlow[I1, O1, I2, O2, Mat] => bidi.asScala case bidi: javadsl.BidiFlow[I1, O1, I2, O2, Mat] @unchecked => bidi.asScala
case other => case other =>
new BidiFlow(other.traversalBuilder, other.shape) new BidiFlow(other.traversalBuilder, other.shape)
} }

View file

@ -379,7 +379,7 @@ object Flow {
def fromGraph[I, O, M](g: Graph[FlowShape[I, O], M]): Flow[I, O, M] = def fromGraph[I, O, M](g: Graph[FlowShape[I, O], M]): Flow[I, O, M] =
g match { g match {
case f: Flow[I, O, M] => f case f: Flow[I, O, M] => f
case f: javadsl.Flow[I, O, M] => f.asScala case f: javadsl.Flow[I, O, M] @unchecked => f.asScala
case g: GraphStageWithMaterializedValue[FlowShape[I, O], M] => case g: GraphStageWithMaterializedValue[FlowShape[I, O], M] =>
// move these from the operator itself to make the returned source // move these from the operator itself to make the returned source
// behave as it is the operator with regards to attributes // behave as it is the operator with regards to attributes
@ -464,7 +464,7 @@ object Flow {
*/ */
def fromSinkAndSourceMat[I, O, M1, M2, M](sink: Graph[SinkShape[I], M1], source: Graph[SourceShape[O], M2])( def fromSinkAndSourceMat[I, O, M1, M2, M](sink: Graph[SinkShape[I], M1], source: Graph[SourceShape[O], M2])(
combine: (M1, M2) => M): Flow[I, O, M] = combine: (M1, M2) => M): Flow[I, O, M] =
fromGraph(GraphDSL.create(sink, source)(combine) { _ => (in, out) => fromGraph(GraphDSL.createGraph(sink, source)(combine) { _ => (in, out) =>
FlowShape(in.in, out.out) FlowShape(in.in, out.out)
}) })
@ -560,7 +560,7 @@ object Flow {
def fromSinkAndSourceCoupledMat[I, O, M1, M2, M](sink: Graph[SinkShape[I], M1], source: Graph[SourceShape[O], M2])( def fromSinkAndSourceCoupledMat[I, O, M1, M2, M](sink: Graph[SinkShape[I], M1], source: Graph[SourceShape[O], M2])(
combine: (M1, M2) => M): Flow[I, O, M] = combine: (M1, M2) => M): Flow[I, O, M] =
// format: OFF // format: OFF
Flow.fromGraph(GraphDSL.create(sink, source)(combine) { implicit b => (i, o) => Flow.fromGraph(GraphDSL.createGraph(sink, source)(combine) { implicit b => (i, o) =>
import GraphDSL.Implicits._ import GraphDSL.Implicits._
val bidi = b.add(new CoupledTerminationBidi[I, O]) val bidi = b.add(new CoupledTerminationBidi[I, O])
/* bidi.in1 ~> */ bidi.out1 ~> i; o ~> bidi.in2 /* ~> bidi.out2 */ /* bidi.in1 ~> */ bidi.out1 ~> i; o ~> bidi.in2 /* ~> bidi.out2 */
@ -2699,7 +2699,7 @@ trait FlowOps[+Out, +Mat] {
} }
protected def zipGraph[U, M](that: Graph[SourceShape[U], M]): Graph[FlowShape[Out @uncheckedVariance, (Out, U)], M] = protected def zipGraph[U, M](that: Graph[SourceShape[U], M]): Graph[FlowShape[Out @uncheckedVariance, (Out, U)], M] =
GraphDSL.create(that) { implicit b => r => GraphDSL.createGraph(that) { implicit b => r =>
val zip = b.add(Zip[Out, U]()) val zip = b.add(Zip[Out, U]())
r ~> zip.in1 r ~> zip.in1
FlowShape(zip.in0, zip.out) FlowShape(zip.in0, zip.out)
@ -2725,7 +2725,7 @@ trait FlowOps[+Out, +Mat] {
protected def zipLatestGraph[U, M]( protected def zipLatestGraph[U, M](
that: Graph[SourceShape[U], M]): Graph[FlowShape[Out @uncheckedVariance, (Out, U)], M] = that: Graph[SourceShape[U], M]): Graph[FlowShape[Out @uncheckedVariance, (Out, U)], M] =
GraphDSL.create(that) { implicit b => r => GraphDSL.createGraph(that) { implicit b => r =>
val zip = b.add(ZipLatest[Out, U]()) val zip = b.add(ZipLatest[Out, U]())
r ~> zip.in1 r ~> zip.in1
FlowShape(zip.in0, zip.out) FlowShape(zip.in0, zip.out)
@ -2748,7 +2748,7 @@ trait FlowOps[+Out, +Mat] {
protected def zipWithGraph[Out2, Out3, M](that: Graph[SourceShape[Out2], M])( protected def zipWithGraph[Out2, Out3, M](that: Graph[SourceShape[Out2], M])(
combine: (Out, Out2) => Out3): Graph[FlowShape[Out @uncheckedVariance, Out3], M] = combine: (Out, Out2) => Out3): Graph[FlowShape[Out @uncheckedVariance, Out3], M] =
GraphDSL.create(that) { implicit b => r => GraphDSL.createGraph(that) { implicit b => r =>
val zip = b.add(ZipWith[Out, Out2, Out3](combine)) val zip = b.add(ZipWith[Out, Out2, Out3](combine))
r ~> zip.in1 r ~> zip.in1
FlowShape(zip.in0, zip.out) FlowShape(zip.in0, zip.out)
@ -2776,7 +2776,7 @@ trait FlowOps[+Out, +Mat] {
protected def zipLatestWithGraph[Out2, Out3, M](that: Graph[SourceShape[Out2], M])( protected def zipLatestWithGraph[Out2, Out3, M](that: Graph[SourceShape[Out2], M])(
combine: (Out, Out2) => Out3): Graph[FlowShape[Out @uncheckedVariance, Out3], M] = combine: (Out, Out2) => Out3): Graph[FlowShape[Out @uncheckedVariance, Out3], M] =
GraphDSL.create(that) { implicit b => r => GraphDSL.createGraph(that) { implicit b => r =>
val zip = b.add(ZipLatestWith[Out, Out2, Out3](combine)) val zip = b.add(ZipLatestWith[Out, Out2, Out3](combine))
r ~> zip.in1 r ~> zip.in1
FlowShape(zip.in0, zip.out) FlowShape(zip.in0, zip.out)
@ -2858,7 +2858,7 @@ trait FlowOps[+Out, +Mat] {
that: Graph[SourceShape[U], M], that: Graph[SourceShape[U], M],
segmentSize: Int, segmentSize: Int,
eagerClose: Boolean = false): Graph[FlowShape[Out @uncheckedVariance, U], M] = eagerClose: Boolean = false): Graph[FlowShape[Out @uncheckedVariance, U], M] =
GraphDSL.create(that) { implicit b => r => GraphDSL.createGraph(that) { implicit b => r =>
val interleave = b.add(Interleave[U](2, segmentSize, eagerClose)) val interleave = b.add(Interleave[U](2, segmentSize, eagerClose))
r ~> interleave.in(1) r ~> interleave.in(1)
FlowShape(interleave.in(0), interleave.out) FlowShape(interleave.in(0), interleave.out)
@ -2882,7 +2882,7 @@ trait FlowOps[+Out, +Mat] {
protected def mergeGraph[U >: Out, M]( protected def mergeGraph[U >: Out, M](
that: Graph[SourceShape[U], M], that: Graph[SourceShape[U], M],
eagerComplete: Boolean): Graph[FlowShape[Out @uncheckedVariance, U], M] = eagerComplete: Boolean): Graph[FlowShape[Out @uncheckedVariance, U], M] =
GraphDSL.create(that) { implicit b => r => GraphDSL.createGraph(that) { implicit b => r =>
val merge = b.add(Merge[U](2, eagerComplete)) val merge = b.add(Merge[U](2, eagerComplete))
r ~> merge.in(1) r ~> merge.in(1)
FlowShape(merge.in(0), merge.out) FlowShape(merge.in(0), merge.out)
@ -2904,7 +2904,7 @@ trait FlowOps[+Out, +Mat] {
protected def mergeLatestGraph[U >: Out, M]( protected def mergeLatestGraph[U >: Out, M](
that: Graph[SourceShape[U], M], that: Graph[SourceShape[U], M],
eagerComplete: Boolean): Graph[FlowShape[Out @uncheckedVariance, immutable.Seq[U]], M] = eagerComplete: Boolean): Graph[FlowShape[Out @uncheckedVariance, immutable.Seq[U]], M] =
GraphDSL.create(that) { implicit b => r => GraphDSL.createGraph(that) { implicit b => r =>
val merge = b.add(MergeLatest[U](2, eagerComplete)) val merge = b.add(MergeLatest[U](2, eagerComplete))
r ~> merge.in(1) r ~> merge.in(1)
FlowShape(merge.in(0), merge.out) FlowShape(merge.in(0), merge.out)
@ -2929,7 +2929,7 @@ trait FlowOps[+Out, +Mat] {
that: Graph[SourceShape[U], M], that: Graph[SourceShape[U], M],
priority: Boolean, priority: Boolean,
eagerComplete: Boolean): Graph[FlowShape[Out @uncheckedVariance, U], M] = eagerComplete: Boolean): Graph[FlowShape[Out @uncheckedVariance, U], M] =
GraphDSL.create(that) { implicit b => r => GraphDSL.createGraph(that) { implicit b => r =>
val merge = b.add(MergePreferred[U](1, eagerComplete)) val merge = b.add(MergePreferred[U](1, eagerComplete))
r ~> merge.in(if (priority) 0 else 1) r ~> merge.in(if (priority) 0 else 1)
FlowShape(merge.in(if (priority) 1 else 0), merge.out) FlowShape(merge.in(if (priority) 1 else 0), merge.out)
@ -2956,7 +2956,7 @@ trait FlowOps[+Out, +Mat] {
leftPriority: Int, leftPriority: Int,
rightPriority: Int, rightPriority: Int,
eagerComplete: Boolean): Graph[FlowShape[Out @uncheckedVariance, U], M] = eagerComplete: Boolean): Graph[FlowShape[Out @uncheckedVariance, U], M] =
GraphDSL.create(that) { implicit b => r => GraphDSL.createGraph(that) { implicit b => r =>
val merge = b.add(MergePrioritized[U](Seq(leftPriority, rightPriority), eagerComplete)) val merge = b.add(MergePrioritized[U](Seq(leftPriority, rightPriority), eagerComplete))
r ~> merge.in(1) r ~> merge.in(1)
FlowShape(merge.in(0), merge.out) FlowShape(merge.in(0), merge.out)
@ -2982,7 +2982,7 @@ trait FlowOps[+Out, +Mat] {
protected def mergeSortedGraph[U >: Out, M](that: Graph[SourceShape[U], M])( protected def mergeSortedGraph[U >: Out, M](that: Graph[SourceShape[U], M])(
implicit ord: Ordering[U]): Graph[FlowShape[Out @uncheckedVariance, U], M] = implicit ord: Ordering[U]): Graph[FlowShape[Out @uncheckedVariance, U], M] =
GraphDSL.create(that) { implicit b => r => GraphDSL.createGraph(that) { implicit b => r =>
val merge = b.add(new MergeSorted[U]) val merge = b.add(new MergeSorted[U])
r ~> merge.in1 r ~> merge.in1
FlowShape(merge.in0, merge.out) FlowShape(merge.in0, merge.out)
@ -3017,7 +3017,7 @@ trait FlowOps[+Out, +Mat] {
protected def concatGraph[U >: Out, Mat2]( protected def concatGraph[U >: Out, Mat2](
that: Graph[SourceShape[U], Mat2], that: Graph[SourceShape[U], Mat2],
detached: Boolean): Graph[FlowShape[Out @uncheckedVariance, U], Mat2] = detached: Boolean): Graph[FlowShape[Out @uncheckedVariance, U], Mat2] =
GraphDSL.create(that) { implicit b => r => GraphDSL.createGraph(that) { implicit b => r =>
val merge = b.add(Concat[U](2, detached)) val merge = b.add(Concat[U](2, detached))
r ~> merge.in(1) r ~> merge.in(1)
FlowShape(merge.in(0), merge.out) FlowShape(merge.in(0), merge.out)
@ -3088,7 +3088,7 @@ trait FlowOps[+Out, +Mat] {
protected def prependGraph[U >: Out, Mat2]( protected def prependGraph[U >: Out, Mat2](
that: Graph[SourceShape[U], Mat2], that: Graph[SourceShape[U], Mat2],
detached: Boolean): Graph[FlowShape[Out @uncheckedVariance, U], Mat2] = detached: Boolean): Graph[FlowShape[Out @uncheckedVariance, U], Mat2] =
GraphDSL.create(that) { implicit b => r => GraphDSL.createGraph(that) { implicit b => r =>
val merge = b.add(Concat[U](2, detached)) val merge = b.add(Concat[U](2, detached))
r ~> merge.in(0) r ~> merge.in(0)
FlowShape(merge.in(1), merge.out) FlowShape(merge.in(1), merge.out)
@ -3144,7 +3144,7 @@ trait FlowOps[+Out, +Mat] {
protected def orElseGraph[U >: Out, Mat2]( protected def orElseGraph[U >: Out, Mat2](
secondary: Graph[SourceShape[U], Mat2]): Graph[FlowShape[Out @uncheckedVariance, U], Mat2] = secondary: Graph[SourceShape[U], Mat2]): Graph[FlowShape[Out @uncheckedVariance, U], Mat2] =
GraphDSL.create(secondary) { implicit b => secondary => GraphDSL.createGraph(secondary) { implicit b => secondary =>
val orElse = b.add(OrElse[U]()) val orElse = b.add(OrElse[U]())
secondary ~> orElse.in(1) secondary ~> orElse.in(1)
@ -3200,7 +3200,7 @@ trait FlowOps[+Out, +Mat] {
def alsoTo(that: Graph[SinkShape[Out], _]): Repr[Out] = via(alsoToGraph(that)) def alsoTo(that: Graph[SinkShape[Out], _]): Repr[Out] = via(alsoToGraph(that))
protected def alsoToGraph[M](that: Graph[SinkShape[Out], M]): Graph[FlowShape[Out @uncheckedVariance, Out], M] = protected def alsoToGraph[M](that: Graph[SinkShape[Out], M]): Graph[FlowShape[Out @uncheckedVariance, Out], M] =
GraphDSL.create(that) { implicit b => r => GraphDSL.createGraph(that) { implicit b => r =>
import GraphDSL.Implicits._ import GraphDSL.Implicits._
val bcast = b.add(Broadcast[Out](2, eagerCancel = true)) val bcast = b.add(Broadcast[Out](2, eagerCancel = true))
bcast.out(1) ~> r bcast.out(1) ~> r
@ -3224,7 +3224,7 @@ trait FlowOps[+Out, +Mat] {
protected def divertToGraph[M]( protected def divertToGraph[M](
that: Graph[SinkShape[Out], M], that: Graph[SinkShape[Out], M],
when: Out => Boolean): Graph[FlowShape[Out @uncheckedVariance, Out], M] = when: Out => Boolean): Graph[FlowShape[Out @uncheckedVariance, Out], M] =
GraphDSL.create(that) { implicit b => r => GraphDSL.createGraph(that) { implicit b => r =>
import GraphDSL.Implicits._ import GraphDSL.Implicits._
val partition = b.add(new Partition[Out](2, out => if (when(out)) 1 else 0, true)) val partition = b.add(new Partition[Out](2, out => if (when(out)) 1 else 0, true))
partition.out(1) ~> r partition.out(1) ~> r
@ -3250,7 +3250,7 @@ trait FlowOps[+Out, +Mat] {
def wireTap(that: Graph[SinkShape[Out], _]): Repr[Out] = via(wireTapGraph(that)) def wireTap(that: Graph[SinkShape[Out], _]): Repr[Out] = via(wireTapGraph(that))
protected def wireTapGraph[M](that: Graph[SinkShape[Out], M]): Graph[FlowShape[Out @uncheckedVariance, Out], M] = protected def wireTapGraph[M](that: Graph[SinkShape[Out], M]): Graph[FlowShape[Out @uncheckedVariance, Out], M] =
GraphDSL.create(that) { implicit b => r => GraphDSL.createGraph(that) { implicit b => r =>
import GraphDSL.Implicits._ import GraphDSL.Implicits._
val bcast = b.add(WireTap[Out]()) val bcast = b.add(WireTap[Out]())
bcast.out1 ~> r bcast.out1 ~> r

View file

@ -239,7 +239,7 @@ private[akka] class MergeHub[T](perProducerBufferSize: Int, drainingEnabled: Boo
def isDraining: Boolean = drainingEnabled && draining def isDraining: Boolean = drainingEnabled && draining
// External API // External API
def enqueue(ev: Event): Unit = { private[MergeHub] def enqueue(ev: Event): Unit = {
queue.add(ev) queue.add(ev)
/* /*
* Simple volatile var is enough, there is no need for a CAS here. The first important thing to note * Simple volatile var is enough, there is no need for a CAS here. The first important thing to note

View file

@ -139,7 +139,7 @@ object Sink {
def fromGraph[T, M](g: Graph[SinkShape[T], M]): Sink[T, M] = def fromGraph[T, M](g: Graph[SinkShape[T], M]): Sink[T, M] =
g match { g match {
case s: Sink[T, M] => s case s: Sink[T, M] => s
case s: javadsl.Sink[T, M] => s.asScala case s: javadsl.Sink[T, M] @unchecked => s.asScala
case g: GraphStageWithMaterializedValue[SinkShape[T], M] => case g: GraphStageWithMaterializedValue[SinkShape[T], M] =>
// move these from the stage itself to make the returned source // move these from the stage itself to make the returned source
// behave as it is the stage with regards to attributes // behave as it is the stage with regards to attributes
@ -161,7 +161,7 @@ object Sink {
def fromMaterializer[T, M](factory: (Materializer, Attributes) => Sink[T, M]): Sink[T, Future[M]] = def fromMaterializer[T, M](factory: (Materializer, Attributes) => Sink[T, M]): Sink[T, Future[M]] =
Flow Flow
.fromMaterializer({ (mat, attr) => .fromMaterializer({ (mat, attr) =>
Flow.fromGraph(GraphDSL.create(factory(mat, attr)) { b => sink => Flow.fromGraph(GraphDSL.createGraph(factory(mat, attr)) { b => sink =>
FlowShape(sink.in, b.materializedValue.outlet) FlowShape(sink.in, b.materializedValue.outlet)
}) })
}) })
@ -579,7 +579,7 @@ object Sink {
onInitMessage: Any, onInitMessage: Any,
ackMessage: Any, ackMessage: Any,
onCompleteMessage: Any, onCompleteMessage: Any,
onFailureMessage: (Throwable) => Any = Status.Failure): Sink[T, NotUsed] = onFailureMessage: (Throwable) => Any = Status.Failure.apply): Sink[T, NotUsed] =
actorRefWithAck(ref, _ => identity, _ => onInitMessage, Some(ackMessage), onCompleteMessage, onFailureMessage) actorRefWithAck(ref, _ => identity, _ => onInitMessage, Some(ackMessage), onCompleteMessage, onFailureMessage)
/** /**

View file

@ -315,7 +315,7 @@ object Source {
*/ */
def fromGraph[T, M](g: Graph[SourceShape[T], M]): Source[T, M] = g match { def fromGraph[T, M](g: Graph[SourceShape[T], M]): Source[T, M] = g match {
case s: Source[T, M] => s case s: Source[T, M] => s
case s: javadsl.Source[T, M] => s.asScala case s: javadsl.Source[T, M] @unchecked => s.asScala
case g: GraphStageWithMaterializedValue[SourceShape[T], M] => case g: GraphStageWithMaterializedValue[SourceShape[T], M] =>
// move these from the stage itself to make the returned source // move these from the stage itself to make the returned source
// behave as it is the stage with regards to attributes // behave as it is the stage with regards to attributes
@ -783,7 +783,7 @@ object Source {
*/ */
def combineMat[T, U, M1, M2, M](first: Source[T, M1], second: Source[T, M2])( def combineMat[T, U, M1, M2, M](first: Source[T, M1], second: Source[T, M2])(
strategy: Int => Graph[UniformFanInShape[T, U], NotUsed])(matF: (M1, M2) => M): Source[U, M] = { strategy: Int => Graph[UniformFanInShape[T, U], NotUsed])(matF: (M1, M2) => M): Source[U, M] = {
val secondPartiallyCombined = GraphDSL.create(second) { implicit b => secondShape => val secondPartiallyCombined = GraphDSL.createGraph(second) { implicit b => secondShape =>
import GraphDSL.Implicits._ import GraphDSL.Implicits._
val c = b.add(strategy(2)) val c = b.add(strategy(2))
secondShape ~> c.in(1) secondShape ~> c.in(1)

View file

@ -135,7 +135,7 @@ object StreamConverters {
if (parallelism == 1) javaCollector[T, R](collectorFactory) if (parallelism == 1) javaCollector[T, R](collectorFactory)
else { else {
Sink Sink
.fromGraph(GraphDSL.create(Sink.head[R]) { implicit b => sink => .fromGraph(GraphDSL.createGraph(Sink.head[R]) { implicit b => sink =>
import GraphDSL.Implicits._ import GraphDSL.Implicits._
val factory = collectorFactory.asInstanceOf[() => Collector[T, Any, R]] val factory = collectorFactory.asInstanceOf[() => Collector[T, Any, R]]
val balance = b.add(Balance[T](parallelism)) val balance = b.add(Balance[T](parallelism))

View file

@ -9,7 +9,6 @@ import java.util.concurrent.TimeoutException
import javax.net.ssl.SSLContext import javax.net.ssl.SSLContext
import javax.net.ssl.SSLEngine import javax.net.ssl.SSLEngine
import javax.net.ssl.SSLSession import javax.net.ssl.SSLSession
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.Future import scala.concurrent.Future
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
@ -17,9 +16,7 @@ import scala.concurrent.duration.FiniteDuration
import scala.util.Success import scala.util.Success
import scala.util.Try import scala.util.Try
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import scala.annotation.nowarn import scala.annotation.nowarn
import akka.Done import akka.Done
import akka.NotUsed import akka.NotUsed
import akka.actor._ import akka.actor._
@ -30,6 +27,7 @@ import akka.io.Inet.SocketOption
import akka.stream._ import akka.stream._
import akka.stream.Attributes.Attribute import akka.stream.Attributes.Attribute
import akka.stream.TLSProtocol.NegotiateNewSession import akka.stream.TLSProtocol.NegotiateNewSession
import akka.stream.impl.TcpImplicitExtensionIdApply
import akka.stream.impl.fusing.GraphStages.detacher import akka.stream.impl.fusing.GraphStages.detacher
import akka.stream.impl.io.ConnectionSourceStage import akka.stream.impl.io.ConnectionSourceStage
import akka.stream.impl.io.OutgoingConnectionStage import akka.stream.impl.io.OutgoingConnectionStage
@ -38,7 +36,7 @@ import akka.util.ByteString
import akka.util.JavaDurationConverters._ import akka.util.JavaDurationConverters._
import akka.util.unused import akka.util.unused
object Tcp extends ExtensionId[Tcp] with ExtensionIdProvider { object Tcp extends ExtensionId[Tcp] with TcpImplicitExtensionIdApply with ExtensionIdProvider {
/** /**
* Represents a successful TCP server binding. * Represents a successful TCP server binding.
@ -79,8 +77,6 @@ object Tcp extends ExtensionId[Tcp] with ExtensionIdProvider {
*/ */
final case class OutgoingConnection(remoteAddress: InetSocketAddress, localAddress: InetSocketAddress) final case class OutgoingConnection(remoteAddress: InetSocketAddress, localAddress: InetSocketAddress)
def apply()(implicit system: ActorSystem): Tcp = super.apply(system)
override def get(system: ActorSystem): Tcp = super.get(system) override def get(system: ActorSystem): Tcp = super.get(system)
override def get(system: ClassicActorSystemProvider): Tcp = super.get(system) override def get(system: ClassicActorSystemProvider): Tcp = super.get(system)
@ -90,7 +86,7 @@ object Tcp extends ExtensionId[Tcp] with ExtensionIdProvider {
// just wraps/unwraps the TLS byte events to provide ByteString, ByteString flows // just wraps/unwraps the TLS byte events to provide ByteString, ByteString flows
private val tlsWrapping: BidiFlow[ByteString, TLSProtocol.SendBytes, TLSProtocol.SslTlsInbound, ByteString, NotUsed] = private val tlsWrapping: BidiFlow[ByteString, TLSProtocol.SendBytes, TLSProtocol.SslTlsInbound, ByteString, NotUsed] =
BidiFlow.fromFlows(Flow[ByteString].map(TLSProtocol.SendBytes), Flow[TLSProtocol.SslTlsInbound].collect { BidiFlow.fromFlows(Flow[ByteString].map(TLSProtocol.SendBytes.apply), Flow[TLSProtocol.SslTlsInbound].collect {
case sb: TLSProtocol.SessionBytes => sb.bytes case sb: TLSProtocol.SessionBytes => sb.bytes
// ignore other kinds of inbounds (currently only Truncated) // ignore other kinds of inbounds (currently only Truncated)
}) })

View file

@ -282,7 +282,7 @@ private[akka] object ConcurrentAsyncCallbackState {
// stream is initialized and so no threads can just send events without any synchronization overhead // stream is initialized and so no threads can just send events without any synchronization overhead
case object Initialized extends State[Nothing] case object Initialized extends State[Nothing]
// Event with feedback promise // Event with feedback promise
final case class Event[E](e: E, handlingPromise: Promise[Done]) final case class Event[+E](e: E, handlingPromise: Promise[Done])
val NoPendingEvents = Pending[Nothing](Nil) val NoPendingEvents = Pending[Nothing](Nil)
} }
@ -1243,7 +1243,7 @@ abstract class GraphStageLogic private[stream] (val inCount: Int, val outCount:
// started - can just dispatch async message to interpreter // started - can just dispatch async message to interpreter
onAsyncInput(event, promise) onAsyncInput(event, promise)
case list @ Pending(l) => case list @ Pending(l: List[Event[T]]) =>
// not started yet // not started yet
if (!currentState.compareAndSet(list, Pending[T](Event[T](event, promise) :: l))) if (!currentState.compareAndSet(list, Pending[T](Event[T](event, promise) :: l)))
invokeWithPromise(event, promise) invokeWithPromise(event, promise)

View file

@ -30,7 +30,7 @@ trait StageLogging { self: GraphStageLogic =>
if (_log eq null) { if (_log eq null) {
materializer match { materializer match {
case p: MaterializerLoggingProvider => case p: MaterializerLoggingProvider =>
_log = p.makeLogger(logSource) _log = p.makeLogger(logSource.asInstanceOf[Class[Any]])
case _ => case _ =>
_log = NoLogging _log = NoLogging
} }

View file

@ -8,23 +8,21 @@ import java.security.KeyStore
import java.security.cert.CertPathValidatorException import java.security.cert.CertPathValidatorException
import java.util.Collections import java.util.Collections
import javax.net.ssl._ import javax.net.ssl._
import com.typesafe.sslconfig.akka.util.AkkaLoggerFactory import com.typesafe.sslconfig.akka.util.AkkaLoggerFactory
import com.typesafe.sslconfig.ssl._ import com.typesafe.sslconfig.ssl._
import com.typesafe.sslconfig.util.LoggerFactory import com.typesafe.sslconfig.util.LoggerFactory
import akka.actor._ import akka.actor._
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.event.Logging import akka.event.Logging
import akka.stream.impl.AkkaSSLConfigExtensionIdApply
@deprecated("Use Tcp and TLS with SSLEngine parameters instead. Setup the SSLEngine with needed parameters.", "2.6.0") @deprecated("Use Tcp and TLS with SSLEngine parameters instead. Setup the SSLEngine with needed parameters.", "2.6.0")
object AkkaSSLConfig extends ExtensionId[AkkaSSLConfig] with ExtensionIdProvider { object AkkaSSLConfig extends ExtensionId[AkkaSSLConfig] with AkkaSSLConfigExtensionIdApply with ExtensionIdProvider {
//////////////////// EXTENSION SETUP /////////////////// //////////////////// EXTENSION SETUP ///////////////////
override def get(system: ActorSystem): AkkaSSLConfig = super.get(system) override def get(system: ActorSystem): AkkaSSLConfig = super.get(system)
override def get(system: ClassicActorSystemProvider): AkkaSSLConfig = super.get(system) override def get(system: ClassicActorSystemProvider): AkkaSSLConfig = super.get(system)
def apply()(implicit system: ActorSystem): AkkaSSLConfig = super.apply(system)
override def lookup = AkkaSSLConfig override def lookup = AkkaSSLConfig
@ -44,7 +42,7 @@ final class AkkaSSLConfig(system: ExtendedActorSystem, val config: SSLConfigSett
private val mkLogger = new AkkaLoggerFactory(system) private val mkLogger = new AkkaLoggerFactory(system)
private val log = Logging(system, getClass) private val log = Logging(system, classOf[AkkaSSLConfig])
log.debug("Initializing AkkaSSLConfig extension...") log.debug("Initializing AkkaSSLConfig extension...")
/** Can be used to modify the underlying config, most typically used to change a few values in the default config */ /** Can be used to modify the underlying config, most typically used to change a few values in the default config */

View file

@ -29,7 +29,7 @@ object Dependencies {
val scala212Version = "2.12.14" val scala212Version = "2.12.14"
val scala213Version = "2.13.5" val scala213Version = "2.13.5"
val scala3Version = "3.0.0" val scala3Version = "3.0.1-RC1"
val reactiveStreamsVersion = "1.0.3" val reactiveStreamsVersion = "1.0.3"