2018-10-29 17:19:37 +08:00
|
|
|
/*
|
2019-01-02 18:55:26 +08:00
|
|
|
* Copyright (C) 2014-2019 Lightbend Inc. <https://www.lightbend.com>
|
2014-04-01 15:19:42 +02:00
|
|
|
*/
|
2018-03-13 23:45:55 +09:00
|
|
|
|
2014-04-01 15:19:42 +02:00
|
|
|
package akka.stream
|
|
|
|
|
|
2014-11-03 15:29:02 +01:00
|
|
|
import java.util.concurrent.TimeUnit
|
2016-05-03 18:58:26 -07:00
|
|
|
import java.util.concurrent.atomic.AtomicBoolean
|
2015-04-16 02:24:01 +02:00
|
|
|
|
2018-02-22 13:42:59 +01:00
|
|
|
import akka.actor.{ ActorContext, ActorRef, ActorRefFactory, ActorSystem, ExtendedActorSystem, Props }
|
|
|
|
|
import akka.annotation.InternalApi
|
2015-11-23 15:57:09 +01:00
|
|
|
import akka.event.LoggingAdapter
|
2016-04-28 14:32:46 +01:00
|
|
|
import akka.util.Helpers.toRootLowerCase
|
2014-11-09 21:09:50 +01:00
|
|
|
import akka.stream.impl._
|
2018-01-04 17:21:47 +01:00
|
|
|
import com.typesafe.config.{ Config, ConfigFactory }
|
2015-04-16 02:24:01 +02:00
|
|
|
|
2014-11-03 15:29:02 +01:00
|
|
|
import scala.concurrent.duration._
|
2015-04-09 12:21:12 +02:00
|
|
|
import akka.japi.function
|
2017-04-28 11:11:50 +02:00
|
|
|
import akka.stream.stage.GraphStageLogic
|
2014-11-03 15:29:02 +01:00
|
|
|
|
2015-05-12 15:44:18 +02:00
|
|
|
import scala.util.control.NoStackTrace
|
|
|
|
|
|
2015-06-23 18:28:53 +02:00
|
|
|
object ActorMaterializer {
|
2014-05-08 19:34:58 +02:00
|
|
|
|
2014-04-01 19:35:56 +02:00
|
|
|
/**
|
2017-11-22 13:51:24 +01:00
|
|
|
* Scala API: Creates an ActorMaterializer that can materialize stream blueprints as running streams.
|
|
|
|
|
*
|
|
|
|
|
* The required [[akka.actor.ActorRefFactory]] (which can be either an [[akka.actor.ActorSystem]] or an [[akka.actor.ActorContext]])
|
2014-09-01 13:12:18 +02:00
|
|
|
* will be used to create one actor that in turn creates actors for the transformation steps.
|
2014-05-08 19:34:58 +02:00
|
|
|
*
|
2015-06-23 18:28:53 +02:00
|
|
|
* The materializer's [[akka.stream.ActorMaterializerSettings]] will be obtained from the
|
2014-08-26 09:03:48 +02:00
|
|
|
* configuration of the `context`'s underlying [[akka.actor.ActorSystem]].
|
|
|
|
|
*
|
2014-05-08 19:34:58 +02:00
|
|
|
* The `namePrefix` is used as the first part of the names of the actors running
|
|
|
|
|
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
|
|
|
|
|
* `namePrefix-flowNumber-flowStepNumber-stepName`.
|
2014-04-01 19:35:56 +02:00
|
|
|
*/
|
2019-03-11 10:38:24 +01:00
|
|
|
def apply(materializerSettings: Option[ActorMaterializerSettings] = None, namePrefix: Option[String] = None)(
|
|
|
|
|
implicit context: ActorRefFactory): ActorMaterializer = {
|
2014-08-26 09:03:48 +02:00
|
|
|
val system = actorSystemOf(context)
|
|
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
val settings = materializerSettings.getOrElse(ActorMaterializerSettings(system))
|
2015-10-31 14:46:10 +01:00
|
|
|
apply(settings, namePrefix.getOrElse("flow"))(context)
|
2014-08-26 09:03:48 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2017-11-22 13:51:24 +01:00
|
|
|
* Scala API: Creates an ActorMaterializer that can materialize stream blueprints as running streams.
|
|
|
|
|
*
|
|
|
|
|
* The required [[akka.actor.ActorRefFactory]]
|
2014-08-26 09:03:48 +02:00
|
|
|
* (which can be either an [[akka.actor.ActorSystem]] or an [[akka.actor.ActorContext]])
|
|
|
|
|
* will be used to create these actors, therefore it is *forbidden* to pass this object
|
|
|
|
|
* to another actor if the factory is an ActorContext.
|
|
|
|
|
*
|
|
|
|
|
* The `namePrefix` is used as the first part of the names of the actors running
|
|
|
|
|
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
|
|
|
|
|
* `namePrefix-flowNumber-flowStepNumber-stepName`.
|
|
|
|
|
*/
|
2019-03-11 10:38:24 +01:00
|
|
|
def apply(materializerSettings: ActorMaterializerSettings, namePrefix: String)(
|
|
|
|
|
implicit context: ActorRefFactory): ActorMaterializer = {
|
2015-06-25 12:54:29 +02:00
|
|
|
val haveShutDown = new AtomicBoolean(false)
|
2014-08-26 09:03:48 +02:00
|
|
|
val system = actorSystemOf(context)
|
2014-08-21 12:35:38 +02:00
|
|
|
|
2019-03-13 10:56:20 +01:00
|
|
|
new PhasedFusingActorMaterializer(
|
|
|
|
|
system,
|
|
|
|
|
materializerSettings,
|
|
|
|
|
system.dispatchers,
|
|
|
|
|
actorOfStreamSupervisor(materializerSettings, context, haveShutDown),
|
|
|
|
|
haveShutDown,
|
|
|
|
|
FlowNames(system).name.copy(namePrefix))
|
2014-08-21 12:35:38 +02:00
|
|
|
}
|
2014-05-08 19:34:58 +02:00
|
|
|
|
2019-03-13 10:56:20 +01:00
|
|
|
private def actorOfStreamSupervisor(
|
|
|
|
|
materializerSettings: ActorMaterializerSettings,
|
|
|
|
|
context: ActorRefFactory,
|
|
|
|
|
haveShutDown: AtomicBoolean) = {
|
2018-11-05 13:10:54 +00:00
|
|
|
val props = StreamSupervisor.props(materializerSettings, haveShutDown)
|
2018-01-22 21:51:49 +09:00
|
|
|
context match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case s: ExtendedActorSystem => s.systemActorOf(props, StreamSupervisor.nextName())
|
|
|
|
|
case a: ActorContext => a.actorOf(props, StreamSupervisor.nextName())
|
2018-01-22 21:51:49 +09:00
|
|
|
}
|
2018-11-05 13:10:54 +00:00
|
|
|
}
|
2018-01-22 21:51:49 +09:00
|
|
|
|
2014-08-26 09:03:48 +02:00
|
|
|
/**
|
2018-03-07 15:12:34 +01:00
|
|
|
* Scala API: Creates an ActorMaterializer that can materialize stream blueprints as running streams.
|
2017-11-22 13:51:24 +01:00
|
|
|
*
|
|
|
|
|
* The required [[akka.actor.ActorRefFactory]]
|
2014-08-26 09:03:48 +02:00
|
|
|
* (which can be either an [[akka.actor.ActorSystem]] or an [[akka.actor.ActorContext]])
|
|
|
|
|
* will be used to create these actors, therefore it is *forbidden* to pass this object
|
|
|
|
|
* to another actor if the factory is an ActorContext.
|
|
|
|
|
*
|
|
|
|
|
* The `namePrefix` is used as the first part of the names of the actors running
|
|
|
|
|
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
|
|
|
|
|
* `namePrefix-flowNumber-flowStepNumber-stepName`.
|
|
|
|
|
*/
|
2015-06-23 18:28:53 +02:00
|
|
|
def apply(materializerSettings: ActorMaterializerSettings)(implicit context: ActorRefFactory): ActorMaterializer =
|
2014-08-26 09:03:48 +02:00
|
|
|
apply(Some(materializerSettings), None)
|
|
|
|
|
|
2016-06-23 18:11:56 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API: Creates the `StreamSupervisor` as a system actor.
|
|
|
|
|
*/
|
2019-03-13 10:56:20 +01:00
|
|
|
private[akka] def systemMaterializer(
|
|
|
|
|
materializerSettings: ActorMaterializerSettings,
|
|
|
|
|
namePrefix: String,
|
|
|
|
|
system: ExtendedActorSystem): ActorMaterializer = {
|
2016-06-23 18:11:56 +02:00
|
|
|
val haveShutDown = new AtomicBoolean(false)
|
2016-07-27 13:29:23 +02:00
|
|
|
new PhasedFusingActorMaterializer(
|
2016-06-23 18:11:56 +02:00
|
|
|
system,
|
|
|
|
|
materializerSettings,
|
|
|
|
|
system.dispatchers,
|
2019-03-11 10:38:24 +01:00
|
|
|
system.systemActorOf(StreamSupervisor.props(materializerSettings, haveShutDown), StreamSupervisor.nextName()),
|
2016-06-23 18:11:56 +02:00
|
|
|
haveShutDown,
|
|
|
|
|
FlowNames(system).name.copy(namePrefix))
|
|
|
|
|
}
|
|
|
|
|
|
2014-08-26 09:03:48 +02:00
|
|
|
/**
|
2017-11-22 13:51:24 +01:00
|
|
|
* Java API: Creates an ActorMaterializer that can materialize stream blueprints as running streams.
|
|
|
|
|
*
|
|
|
|
|
* The required [[akka.actor.ActorRefFactory]]
|
2014-08-26 09:03:48 +02:00
|
|
|
* (which can be either an [[akka.actor.ActorSystem]] or an [[akka.actor.ActorContext]])
|
|
|
|
|
* will be used to create these actors, therefore it is *forbidden* to pass this object
|
|
|
|
|
* to another actor if the factory is an ActorContext.
|
|
|
|
|
*
|
|
|
|
|
* Defaults the actor name prefix used to name actors running the processing steps to `"flow"`.
|
|
|
|
|
* The actor names are built up of `namePrefix-flowNumber-flowStepNumber-stepName`.
|
|
|
|
|
*/
|
2015-06-23 18:28:53 +02:00
|
|
|
def create(context: ActorRefFactory): ActorMaterializer =
|
2014-08-26 09:03:48 +02:00
|
|
|
apply()(context)
|
|
|
|
|
|
2014-04-23 10:05:09 +02:00
|
|
|
/**
|
2017-11-22 13:51:24 +01:00
|
|
|
* Java API: Creates an ActorMaterializer that can materialize stream blueprints as running streams.
|
|
|
|
|
*
|
|
|
|
|
* The required [[akka.actor.ActorRefFactory]]
|
2014-04-23 10:05:09 +02:00
|
|
|
* (which can be either an [[akka.actor.ActorSystem]] or an [[akka.actor.ActorContext]])
|
2014-09-01 13:12:18 +02:00
|
|
|
* will be used to create one actor that in turn creates actors for the transformation steps.
|
2014-04-23 10:05:09 +02:00
|
|
|
*/
|
2015-06-23 18:28:53 +02:00
|
|
|
def create(settings: ActorMaterializerSettings, context: ActorRefFactory): ActorMaterializer =
|
2014-08-26 09:03:48 +02:00
|
|
|
apply(Option(settings), None)(context)
|
|
|
|
|
|
|
|
|
|
/**
|
2017-11-22 13:51:24 +01:00
|
|
|
* Java API: Creates an ActorMaterializer that can materialize stream blueprints as running streams.
|
|
|
|
|
*
|
|
|
|
|
* The required [[akka.actor.ActorRefFactory]]
|
2014-08-26 09:03:48 +02:00
|
|
|
* (which can be either an [[akka.actor.ActorSystem]] or an [[akka.actor.ActorContext]])
|
|
|
|
|
* will be used to create these actors, therefore it is *forbidden* to pass this object
|
|
|
|
|
* to another actor if the factory is an ActorContext.
|
|
|
|
|
*
|
|
|
|
|
* The `namePrefix` is used as the first part of the names of the actors running
|
|
|
|
|
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
|
|
|
|
|
* `namePrefix-flowNumber-flowStepNumber-stepName`.
|
|
|
|
|
*/
|
2015-06-23 18:28:53 +02:00
|
|
|
def create(settings: ActorMaterializerSettings, context: ActorRefFactory, namePrefix: String): ActorMaterializer =
|
2014-08-26 09:03:48 +02:00
|
|
|
apply(Option(settings), Option(namePrefix))(context)
|
|
|
|
|
|
|
|
|
|
private def actorSystemOf(context: ActorRefFactory): ActorSystem = {
|
|
|
|
|
val system = context match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case s: ExtendedActorSystem => s
|
|
|
|
|
case c: ActorContext => c.system
|
|
|
|
|
case null => throw new IllegalArgumentException("ActorRefFactory context must be defined")
|
|
|
|
|
case _ =>
|
2019-03-11 10:38:24 +01:00
|
|
|
throw new IllegalArgumentException(
|
|
|
|
|
s"ActorRefFactory context must be an ActorSystem or ActorContext, got [${context.getClass.getName}]")
|
2014-08-26 09:03:48 +02:00
|
|
|
}
|
|
|
|
|
system
|
|
|
|
|
}
|
|
|
|
|
|
2016-05-03 18:58:26 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] object ActorMaterializerHelper {
|
2019-03-11 10:38:24 +01:00
|
|
|
|
2015-04-10 14:39:48 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2015-06-23 18:28:53 +02:00
|
|
|
private[akka] def downcast(materializer: Materializer): ActorMaterializer =
|
2015-07-06 22:00:21 +02:00
|
|
|
materializer match { //FIXME this method is going to cause trouble for other Materializer implementations
|
2019-02-09 15:25:39 +01:00
|
|
|
case m: ActorMaterializer => m
|
2019-03-11 10:38:24 +01:00
|
|
|
case _ =>
|
|
|
|
|
throw new IllegalArgumentException(
|
|
|
|
|
s"required [${classOf[ActorMaterializer].getName}] " +
|
|
|
|
|
s"but got [${materializer.getClass.getName}]")
|
2015-04-10 14:39:48 +02:00
|
|
|
}
|
2014-04-01 15:19:42 +02:00
|
|
|
}
|
|
|
|
|
|
2014-04-01 19:35:56 +02:00
|
|
|
/**
|
2017-11-22 13:51:24 +01:00
|
|
|
* An ActorMaterializer takes a stream blueprint and turns it into a running stream.
|
2014-04-01 19:35:56 +02:00
|
|
|
*/
|
2016-10-28 16:05:56 +02:00
|
|
|
abstract class ActorMaterializer extends Materializer with MaterializerLoggingProvider {
|
2015-01-27 18:29:20 +01:00
|
|
|
|
2015-06-23 18:28:53 +02:00
|
|
|
def settings: ActorMaterializerSettings
|
2015-01-27 18:29:20 +01:00
|
|
|
|
2015-06-19 17:15:50 +02:00
|
|
|
/**
|
2018-06-09 17:42:56 +09:00
|
|
|
* Shuts down this materializer and all the operators that have been materialized through this materializer. After
|
|
|
|
|
* having shut down, this materializer cannot be used again. Any attempt to materialize operators after having
|
2015-06-19 17:15:50 +02:00
|
|
|
* shut down will result in an IllegalStateException being thrown at materialization time.
|
|
|
|
|
*/
|
|
|
|
|
def shutdown(): Unit
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Indicates if the materializer has been shut down.
|
|
|
|
|
*/
|
|
|
|
|
def isShutdown: Boolean
|
|
|
|
|
|
2015-01-27 18:29:20 +01:00
|
|
|
/**
|
2016-05-03 18:58:26 -07:00
|
|
|
* INTERNAL API
|
2015-01-27 18:29:20 +01:00
|
|
|
*/
|
2015-04-10 14:39:48 +02:00
|
|
|
private[akka] def actorOf(context: MaterializationContext, props: Props): ActorRef
|
2015-01-27 18:29:20 +01:00
|
|
|
|
2015-04-09 12:21:12 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2016-05-03 18:58:26 -07:00
|
|
|
def system: ActorSystem
|
2015-04-09 12:21:12 +02:00
|
|
|
|
2015-11-23 15:57:09 +01:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] def logger: LoggingAdapter
|
|
|
|
|
|
2016-05-03 18:58:26 -07:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2015-05-27 00:27:05 +02:00
|
|
|
private[akka] def supervisor: ActorRef
|
|
|
|
|
|
2015-01-27 18:29:20 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* This exception or subtypes thereof should be used to signal materialization
|
|
|
|
|
* failures.
|
|
|
|
|
*/
|
|
|
|
|
class MaterializationException(msg: String, cause: Throwable = null) extends RuntimeException(msg, cause)
|
|
|
|
|
|
2015-05-12 15:44:18 +02:00
|
|
|
/**
|
|
|
|
|
* This exception signals that an actor implementing a Reactive Streams Subscriber, Publisher or Processor
|
|
|
|
|
* has been terminated without being notified by an onError, onComplete or cancel signal. This usually happens
|
|
|
|
|
* when an ActorSystem is shut down while stream processing actors are still running.
|
|
|
|
|
*/
|
|
|
|
|
final case class AbruptTerminationException(actor: ActorRef)
|
2019-03-11 10:38:24 +01:00
|
|
|
extends RuntimeException(s"Processor actor [$actor] terminated abruptly")
|
|
|
|
|
with NoStackTrace
|
2015-05-12 15:44:18 +02:00
|
|
|
|
2017-04-28 11:11:50 +02:00
|
|
|
/**
|
2018-06-09 17:42:56 +09:00
|
|
|
* Signal that the operator was abruptly terminated, usually seen as a call to `postStop` of the `GraphStageLogic` without
|
2017-04-28 11:11:50 +02:00
|
|
|
* any of the handler callbacks seeing completion or failure from upstream or cancellation from downstream. This can happen when
|
|
|
|
|
* the actor running the graph is killed, which happens when the materializer or actor system is terminated.
|
|
|
|
|
*/
|
|
|
|
|
final class AbruptStageTerminationException(logic: GraphStageLogic)
|
2019-03-11 10:38:24 +01:00
|
|
|
extends RuntimeException(
|
|
|
|
|
s"GraphStage [$logic] terminated abruptly, caused by for example materializer or actor system termination.")
|
|
|
|
|
with NoStackTrace
|
2017-04-28 11:11:50 +02:00
|
|
|
|
2015-06-23 18:28:53 +02:00
|
|
|
object ActorMaterializerSettings {
|
2015-05-05 12:32:49 +02:00
|
|
|
|
2015-12-14 17:02:00 +01:00
|
|
|
/**
|
|
|
|
|
* Create [[ActorMaterializerSettings]] from individual settings (Scala).
|
|
|
|
|
*/
|
2018-01-04 17:21:47 +01:00
|
|
|
@Deprecated
|
2019-03-11 10:38:24 +01:00
|
|
|
@deprecated(
|
|
|
|
|
"Create the settings using the apply(system) or apply(config) method, and then modify them using the .with methods.",
|
|
|
|
|
since = "2.5.10")
|
2019-03-13 10:56:20 +01:00
|
|
|
def apply(
|
|
|
|
|
initialInputBufferSize: Int,
|
|
|
|
|
maxInputBufferSize: Int,
|
|
|
|
|
dispatcher: String,
|
|
|
|
|
supervisionDecider: Supervision.Decider,
|
|
|
|
|
subscriptionTimeoutSettings: StreamSubscriptionTimeoutSettings,
|
|
|
|
|
debugLogging: Boolean,
|
|
|
|
|
outputBurstLimit: Int,
|
|
|
|
|
fuzzingMode: Boolean,
|
|
|
|
|
autoFusing: Boolean,
|
|
|
|
|
maxFixedBufferSize: Int) = {
|
2018-03-07 15:12:34 +01:00
|
|
|
// these sins were committed in the name of bin comp:
|
2018-02-22 13:42:59 +01:00
|
|
|
val config = ConfigFactory.defaultReference
|
2019-03-13 10:56:20 +01:00
|
|
|
new ActorMaterializerSettings(
|
|
|
|
|
initialInputBufferSize,
|
|
|
|
|
maxInputBufferSize,
|
|
|
|
|
dispatcher,
|
|
|
|
|
supervisionDecider,
|
|
|
|
|
subscriptionTimeoutSettings,
|
|
|
|
|
debugLogging,
|
|
|
|
|
outputBurstLimit,
|
|
|
|
|
fuzzingMode,
|
|
|
|
|
autoFusing,
|
|
|
|
|
maxFixedBufferSize,
|
|
|
|
|
1000,
|
|
|
|
|
IOSettings(tcpWriteBufferSize = 16 * 1024),
|
|
|
|
|
StreamRefSettings(config.getConfig("akka.stream.materializer.stream-ref")),
|
|
|
|
|
config.getString(ActorAttributes.IODispatcher.dispatcher))
|
2018-02-22 13:42:59 +01:00
|
|
|
}
|
2015-05-05 12:32:49 +02:00
|
|
|
|
2014-04-23 10:05:09 +02:00
|
|
|
/**
|
2015-12-14 17:02:00 +01:00
|
|
|
* Create [[ActorMaterializerSettings]] from the settings of an [[akka.actor.ActorSystem]] (Scala).
|
2014-04-23 10:05:09 +02:00
|
|
|
*/
|
2015-06-23 18:28:53 +02:00
|
|
|
def apply(system: ActorSystem): ActorMaterializerSettings =
|
2014-08-26 09:03:48 +02:00
|
|
|
apply(system.settings.config.getConfig("akka.stream.materializer"))
|
|
|
|
|
|
|
|
|
|
/**
|
2015-12-14 17:02:00 +01:00
|
|
|
* Create [[ActorMaterializerSettings]] from a Config subsection (Scala).
|
2014-08-26 09:03:48 +02:00
|
|
|
*/
|
2015-06-23 18:28:53 +02:00
|
|
|
def apply(config: Config): ActorMaterializerSettings =
|
2019-03-13 10:56:20 +01:00
|
|
|
new ActorMaterializerSettings(
|
|
|
|
|
initialInputBufferSize = config.getInt("initial-input-buffer-size"),
|
|
|
|
|
maxInputBufferSize = config.getInt("max-input-buffer-size"),
|
|
|
|
|
dispatcher = config.getString("dispatcher"),
|
|
|
|
|
supervisionDecider = Supervision.stoppingDecider,
|
|
|
|
|
subscriptionTimeoutSettings = StreamSubscriptionTimeoutSettings(config),
|
|
|
|
|
debugLogging = config.getBoolean("debug-logging"),
|
|
|
|
|
outputBurstLimit = config.getInt("output-burst-limit"),
|
|
|
|
|
fuzzingMode = config.getBoolean("debug.fuzzing-mode"),
|
|
|
|
|
autoFusing = config.getBoolean("auto-fusing"),
|
|
|
|
|
maxFixedBufferSize = config.getInt("max-fixed-buffer-size"),
|
|
|
|
|
syncProcessingLimit = config.getInt("sync-processing-limit"),
|
|
|
|
|
ioSettings = IOSettings(config.getConfig("io")),
|
|
|
|
|
streamRefSettings = StreamRefSettings(config.getConfig("stream-ref")),
|
|
|
|
|
blockingIoDispatcher = config.getString("blocking-io-dispatcher"))
|
2014-08-26 09:03:48 +02:00
|
|
|
|
|
|
|
|
/**
|
2015-12-14 17:02:00 +01:00
|
|
|
* Create [[ActorMaterializerSettings]] from individual settings (Java).
|
|
|
|
|
*/
|
2018-01-04 17:21:47 +01:00
|
|
|
@Deprecated
|
2019-03-11 10:38:24 +01:00
|
|
|
@deprecated(
|
|
|
|
|
"Create the settings using the create(system) or create(config) method, and then modify them using the .with methods.",
|
|
|
|
|
since = "2.5.10")
|
2019-03-13 10:56:20 +01:00
|
|
|
def create(
|
|
|
|
|
initialInputBufferSize: Int,
|
|
|
|
|
maxInputBufferSize: Int,
|
|
|
|
|
dispatcher: String,
|
|
|
|
|
supervisionDecider: Supervision.Decider,
|
|
|
|
|
subscriptionTimeoutSettings: StreamSubscriptionTimeoutSettings,
|
|
|
|
|
debugLogging: Boolean,
|
|
|
|
|
outputBurstLimit: Int,
|
|
|
|
|
fuzzingMode: Boolean,
|
|
|
|
|
autoFusing: Boolean,
|
|
|
|
|
maxFixedBufferSize: Int) = {
|
2018-03-07 15:12:34 +01:00
|
|
|
// these sins were committed in the name of bin comp:
|
2018-02-22 13:42:59 +01:00
|
|
|
val config = ConfigFactory.defaultReference
|
2019-03-13 10:56:20 +01:00
|
|
|
new ActorMaterializerSettings(
|
|
|
|
|
initialInputBufferSize,
|
|
|
|
|
maxInputBufferSize,
|
|
|
|
|
dispatcher,
|
|
|
|
|
supervisionDecider,
|
|
|
|
|
subscriptionTimeoutSettings,
|
|
|
|
|
debugLogging,
|
|
|
|
|
outputBurstLimit,
|
|
|
|
|
fuzzingMode,
|
|
|
|
|
autoFusing,
|
|
|
|
|
maxFixedBufferSize,
|
|
|
|
|
1000,
|
|
|
|
|
IOSettings(tcpWriteBufferSize = 16 * 1024),
|
|
|
|
|
StreamRefSettings(config.getConfig("akka.stream.materializer.stream-ref")),
|
|
|
|
|
config.getString(ActorAttributes.IODispatcher.dispatcher))
|
2018-02-22 13:42:59 +01:00
|
|
|
}
|
2015-12-14 17:02:00 +01:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Create [[ActorMaterializerSettings]] from the settings of an [[akka.actor.ActorSystem]] (Java).
|
2014-08-26 09:03:48 +02:00
|
|
|
*/
|
2015-06-23 18:28:53 +02:00
|
|
|
def create(system: ActorSystem): ActorMaterializerSettings =
|
2014-08-26 09:03:48 +02:00
|
|
|
apply(system)
|
|
|
|
|
|
|
|
|
|
/**
|
2015-12-14 17:02:00 +01:00
|
|
|
* Create [[ActorMaterializerSettings]] from a Config subsection (Java).
|
2014-08-26 09:03:48 +02:00
|
|
|
*/
|
2015-06-23 18:28:53 +02:00
|
|
|
def create(config: Config): ActorMaterializerSettings =
|
2014-08-26 09:03:48 +02:00
|
|
|
apply(config)
|
2015-11-09 16:19:12 +01:00
|
|
|
|
2014-04-23 10:05:09 +02:00
|
|
|
}
|
|
|
|
|
|
2014-04-01 19:35:56 +02:00
|
|
|
/**
|
2015-12-14 17:02:00 +01:00
|
|
|
* This class describes the configurable properties of the [[ActorMaterializer]].
|
|
|
|
|
* Please refer to the `withX` methods for descriptions of the individual settings.
|
2018-02-22 13:42:59 +01:00
|
|
|
*
|
|
|
|
|
* The constructor is not public API, use create or apply on the [[ActorMaterializerSettings]] companion instead.
|
2014-04-01 19:35:56 +02:00
|
|
|
*/
|
2018-02-22 13:42:59 +01:00
|
|
|
final class ActorMaterializerSettings @InternalApi private (
|
2019-03-11 10:38:24 +01:00
|
|
|
/*
|
|
|
|
|
* Important note: `initialInputBufferSize`, `maxInputBufferSize`, `dispatcher` and
|
|
|
|
|
* `supervisionDecider` must not be used as values in the materializer, or anything the materializer phases use
|
|
|
|
|
* since these settings allow for overriding using [[Attributes]]. They must always be gotten from the effective
|
|
|
|
|
* attributes.
|
|
|
|
|
*/
|
|
|
|
|
val initialInputBufferSize: Int,
|
|
|
|
|
val maxInputBufferSize: Int,
|
|
|
|
|
val dispatcher: String,
|
|
|
|
|
val supervisionDecider: Supervision.Decider,
|
|
|
|
|
val subscriptionTimeoutSettings: StreamSubscriptionTimeoutSettings,
|
|
|
|
|
val debugLogging: Boolean,
|
|
|
|
|
val outputBurstLimit: Int,
|
|
|
|
|
val fuzzingMode: Boolean,
|
|
|
|
|
val autoFusing: Boolean,
|
|
|
|
|
val maxFixedBufferSize: Int,
|
|
|
|
|
val syncProcessingLimit: Int,
|
|
|
|
|
val ioSettings: IOSettings,
|
|
|
|
|
val streamRefSettings: StreamRefSettings,
|
|
|
|
|
val blockingIoDispatcher: String) {
|
2014-08-26 09:03:48 +02:00
|
|
|
|
|
|
|
|
require(initialInputBufferSize > 0, "initialInputBufferSize must be > 0")
|
2016-02-02 16:39:47 -05:00
|
|
|
require(syncProcessingLimit > 0, "syncProcessingLimit must be > 0")
|
2014-08-26 09:03:48 +02:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
requirePowerOfTwo(maxInputBufferSize, "maxInputBufferSize")
|
2019-03-13 10:56:20 +01:00
|
|
|
require(
|
|
|
|
|
initialInputBufferSize <= maxInputBufferSize,
|
|
|
|
|
s"initialInputBufferSize($initialInputBufferSize) must be <= maxInputBufferSize($maxInputBufferSize)")
|
2014-04-01 15:19:42 +02:00
|
|
|
|
2018-01-22 19:13:40 +09:00
|
|
|
// backwards compatibility when added IOSettings, shouldn't be needed since private, but added to satisfy mima
|
2018-02-22 13:42:59 +01:00
|
|
|
@deprecated("Use ActorMaterializerSettings.apply or ActorMaterializerSettings.create instead", "2.5.10")
|
2019-03-13 10:56:20 +01:00
|
|
|
def this(
|
|
|
|
|
initialInputBufferSize: Int,
|
|
|
|
|
maxInputBufferSize: Int,
|
|
|
|
|
dispatcher: String,
|
|
|
|
|
supervisionDecider: Supervision.Decider,
|
|
|
|
|
subscriptionTimeoutSettings: StreamSubscriptionTimeoutSettings,
|
|
|
|
|
debugLogging: Boolean,
|
|
|
|
|
outputBurstLimit: Int,
|
|
|
|
|
fuzzingMode: Boolean,
|
|
|
|
|
autoFusing: Boolean,
|
|
|
|
|
maxFixedBufferSize: Int,
|
|
|
|
|
syncProcessingLimit: Int,
|
|
|
|
|
ioSettings: IOSettings) =
|
2018-02-22 13:42:59 +01:00
|
|
|
// using config like this is not quite right but the only way to solve backwards comp without hard coding settings
|
2019-03-13 10:56:20 +01:00
|
|
|
this(
|
|
|
|
|
initialInputBufferSize,
|
|
|
|
|
maxInputBufferSize,
|
|
|
|
|
dispatcher,
|
|
|
|
|
supervisionDecider,
|
|
|
|
|
subscriptionTimeoutSettings,
|
|
|
|
|
debugLogging,
|
|
|
|
|
outputBurstLimit,
|
|
|
|
|
fuzzingMode,
|
|
|
|
|
autoFusing,
|
|
|
|
|
maxFixedBufferSize,
|
|
|
|
|
syncProcessingLimit,
|
|
|
|
|
ioSettings,
|
|
|
|
|
StreamRefSettings(ConfigFactory.defaultReference().getConfig("akka.stream.materializer.stream-ref")),
|
|
|
|
|
ConfigFactory.defaultReference().getString(ActorAttributes.IODispatcher.dispatcher))
|
2018-01-22 19:13:40 +09:00
|
|
|
|
|
|
|
|
// backwards compatibility when added IOSettings, shouldn't be needed since private, but added to satisfy mima
|
2018-02-22 13:42:59 +01:00
|
|
|
@deprecated("Use ActorMaterializerSettings.apply or ActorMaterializerSettings.create instead", "2.5.10")
|
2019-03-13 10:56:20 +01:00
|
|
|
def this(
|
|
|
|
|
initialInputBufferSize: Int,
|
|
|
|
|
maxInputBufferSize: Int,
|
|
|
|
|
dispatcher: String,
|
|
|
|
|
supervisionDecider: Supervision.Decider,
|
|
|
|
|
subscriptionTimeoutSettings: StreamSubscriptionTimeoutSettings,
|
|
|
|
|
debugLogging: Boolean,
|
|
|
|
|
outputBurstLimit: Int,
|
|
|
|
|
fuzzingMode: Boolean,
|
|
|
|
|
autoFusing: Boolean,
|
|
|
|
|
maxFixedBufferSize: Int,
|
|
|
|
|
syncProcessingLimit: Int) =
|
2018-02-22 13:42:59 +01:00
|
|
|
// using config like this is not quite right but the only way to solve backwards comp without hard coding settings
|
2019-03-13 10:56:20 +01:00
|
|
|
this(
|
|
|
|
|
initialInputBufferSize,
|
|
|
|
|
maxInputBufferSize,
|
|
|
|
|
dispatcher,
|
|
|
|
|
supervisionDecider,
|
|
|
|
|
subscriptionTimeoutSettings,
|
|
|
|
|
debugLogging,
|
|
|
|
|
outputBurstLimit,
|
|
|
|
|
fuzzingMode,
|
|
|
|
|
autoFusing,
|
|
|
|
|
maxFixedBufferSize,
|
|
|
|
|
syncProcessingLimit,
|
|
|
|
|
IOSettings(tcpWriteBufferSize = 16 * 1024),
|
|
|
|
|
StreamRefSettings(ConfigFactory.defaultReference().getConfig("akka.stream.materializer.stream-ref")),
|
|
|
|
|
ConfigFactory.defaultReference().getString(ActorAttributes.IODispatcher.dispatcher))
|
2018-01-22 19:13:40 +09:00
|
|
|
|
|
|
|
|
// backwards compatibility when added IOSettings, shouldn't be needed since private, but added to satisfy mima
|
2018-02-22 13:42:59 +01:00
|
|
|
@deprecated("Use ActorMaterializerSettings.apply or ActorMaterializerSettings.create instead", "2.5.10")
|
2019-03-13 10:56:20 +01:00
|
|
|
def this(
|
|
|
|
|
initialInputBufferSize: Int,
|
|
|
|
|
maxInputBufferSize: Int,
|
|
|
|
|
dispatcher: String,
|
|
|
|
|
supervisionDecider: Supervision.Decider,
|
|
|
|
|
subscriptionTimeoutSettings: StreamSubscriptionTimeoutSettings,
|
|
|
|
|
debugLogging: Boolean,
|
|
|
|
|
outputBurstLimit: Int,
|
|
|
|
|
fuzzingMode: Boolean,
|
|
|
|
|
autoFusing: Boolean,
|
|
|
|
|
maxFixedBufferSize: Int) =
|
2018-02-22 13:42:59 +01:00
|
|
|
// using config like this is not quite right but the only way to solve backwards comp without hard coding settings
|
2019-03-13 10:56:20 +01:00
|
|
|
this(
|
|
|
|
|
initialInputBufferSize,
|
|
|
|
|
maxInputBufferSize,
|
|
|
|
|
dispatcher,
|
|
|
|
|
supervisionDecider,
|
|
|
|
|
subscriptionTimeoutSettings,
|
|
|
|
|
debugLogging,
|
|
|
|
|
outputBurstLimit,
|
|
|
|
|
fuzzingMode,
|
|
|
|
|
autoFusing,
|
|
|
|
|
maxFixedBufferSize,
|
|
|
|
|
1000,
|
|
|
|
|
IOSettings(tcpWriteBufferSize = 16 * 1024),
|
|
|
|
|
StreamRefSettings(ConfigFactory.defaultReference().getConfig("akka.stream.materializer.stream-ref")),
|
|
|
|
|
ConfigFactory.defaultReference().getString(ActorAttributes.IODispatcher.dispatcher))
|
|
|
|
|
|
|
|
|
|
private def copy(
|
|
|
|
|
initialInputBufferSize: Int = this.initialInputBufferSize,
|
|
|
|
|
maxInputBufferSize: Int = this.maxInputBufferSize,
|
|
|
|
|
dispatcher: String = this.dispatcher,
|
|
|
|
|
supervisionDecider: Supervision.Decider = this.supervisionDecider,
|
|
|
|
|
subscriptionTimeoutSettings: StreamSubscriptionTimeoutSettings = this.subscriptionTimeoutSettings,
|
|
|
|
|
debugLogging: Boolean = this.debugLogging,
|
|
|
|
|
outputBurstLimit: Int = this.outputBurstLimit,
|
|
|
|
|
fuzzingMode: Boolean = this.fuzzingMode,
|
|
|
|
|
autoFusing: Boolean = this.autoFusing,
|
|
|
|
|
maxFixedBufferSize: Int = this.maxFixedBufferSize,
|
|
|
|
|
syncProcessingLimit: Int = this.syncProcessingLimit,
|
|
|
|
|
ioSettings: IOSettings = this.ioSettings,
|
|
|
|
|
streamRefSettings: StreamRefSettings = this.streamRefSettings,
|
|
|
|
|
blockingIoDispatcher: String = this.blockingIoDispatcher) = {
|
|
|
|
|
new ActorMaterializerSettings(
|
|
|
|
|
initialInputBufferSize,
|
|
|
|
|
maxInputBufferSize,
|
|
|
|
|
dispatcher,
|
|
|
|
|
supervisionDecider,
|
|
|
|
|
subscriptionTimeoutSettings,
|
|
|
|
|
debugLogging,
|
|
|
|
|
outputBurstLimit,
|
|
|
|
|
fuzzingMode,
|
|
|
|
|
autoFusing,
|
|
|
|
|
maxFixedBufferSize,
|
|
|
|
|
syncProcessingLimit,
|
|
|
|
|
ioSettings,
|
|
|
|
|
streamRefSettings,
|
|
|
|
|
blockingIoDispatcher)
|
2016-02-02 16:39:47 -05:00
|
|
|
}
|
2015-05-05 12:32:49 +02:00
|
|
|
|
2015-12-14 17:02:00 +01:00
|
|
|
/**
|
|
|
|
|
* Each asynchronous piece of a materialized stream topology is executed by one Actor
|
|
|
|
|
* that manages an input buffer for all inlets of its shape. This setting configures
|
2017-11-22 13:51:24 +01:00
|
|
|
* the default for initial and maximal input buffer in number of elements for each inlet.
|
|
|
|
|
* This can be overridden for individual parts of the
|
|
|
|
|
* stream topology by using [[akka.stream.Attributes#inputBuffer]].
|
2015-12-14 17:02:00 +01:00
|
|
|
*
|
2017-11-22 13:51:24 +01:00
|
|
|
* FIXME: this is used for all kinds of buffers, not only the stream actor, some use initial some use max,
|
|
|
|
|
* document and or fix if it should not be like that. Search for get[Attributes.InputBuffer] to see how it is used
|
2015-12-14 17:02:00 +01:00
|
|
|
*/
|
2015-11-09 16:19:12 +01:00
|
|
|
def withInputBuffer(initialSize: Int, maxSize: Int): ActorMaterializerSettings = {
|
|
|
|
|
if (initialSize == this.initialInputBufferSize && maxSize == this.maxInputBufferSize) this
|
|
|
|
|
else copy(initialInputBufferSize = initialSize, maxInputBufferSize = maxSize)
|
|
|
|
|
}
|
2014-04-23 10:05:09 +02:00
|
|
|
|
2015-12-14 17:02:00 +01:00
|
|
|
/**
|
|
|
|
|
* This setting configures the default dispatcher to be used by streams materialized
|
|
|
|
|
* with the [[ActorMaterializer]]. This can be overridden for individual parts of the
|
|
|
|
|
* stream topology by using [[akka.stream.Attributes#dispatcher]].
|
|
|
|
|
*/
|
2015-11-09 16:19:12 +01:00
|
|
|
def withDispatcher(dispatcher: String): ActorMaterializerSettings = {
|
|
|
|
|
if (this.dispatcher == dispatcher) this
|
|
|
|
|
else copy(dispatcher = dispatcher)
|
|
|
|
|
}
|
2014-05-13 17:17:33 +02:00
|
|
|
|
2015-02-04 09:26:32 +01:00
|
|
|
/**
|
|
|
|
|
* Scala API: Decides how exceptions from application code are to be handled, unless
|
2015-04-09 15:16:59 +02:00
|
|
|
* overridden for specific flows of the stream operations with
|
2015-06-23 17:32:55 +02:00
|
|
|
* [[akka.stream.Attributes#supervisionStrategy]].
|
2017-11-22 13:51:24 +01:00
|
|
|
*
|
2018-06-09 17:42:56 +09:00
|
|
|
* Note that supervision in streams are implemented on a per operator basis and is not supported
|
|
|
|
|
* by every operator.
|
2015-02-04 09:26:32 +01:00
|
|
|
*/
|
2015-11-09 16:19:12 +01:00
|
|
|
def withSupervisionStrategy(decider: Supervision.Decider): ActorMaterializerSettings = {
|
|
|
|
|
if (decider eq this.supervisionDecider) this
|
|
|
|
|
else copy(supervisionDecider = decider)
|
|
|
|
|
}
|
2015-02-04 09:26:32 +01:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Java API: Decides how exceptions from application code are to be handled, unless
|
2015-04-09 15:16:59 +02:00
|
|
|
* overridden for specific flows of the stream operations with
|
2015-06-23 17:32:55 +02:00
|
|
|
* [[akka.stream.Attributes#supervisionStrategy]].
|
2017-11-22 13:51:24 +01:00
|
|
|
*
|
2018-06-09 17:42:56 +09:00
|
|
|
* Note that supervision in streams are implemented on a per operator basis and is not supported
|
|
|
|
|
* by every operator.
|
2015-02-04 09:26:32 +01:00
|
|
|
*/
|
2019-03-11 10:38:24 +01:00
|
|
|
def withSupervisionStrategy(
|
|
|
|
|
decider: function.Function[Throwable, Supervision.Directive]): ActorMaterializerSettings = {
|
2015-02-23 11:54:02 +01:00
|
|
|
import Supervision._
|
|
|
|
|
copy(supervisionDecider = decider match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case `resumingDecider` => resumingDecider
|
|
|
|
|
case `restartingDecider` => restartingDecider
|
|
|
|
|
case `stoppingDecider` => stoppingDecider
|
|
|
|
|
case other => other.apply _
|
2015-02-23 11:54:02 +01:00
|
|
|
})
|
|
|
|
|
}
|
2015-02-04 09:26:32 +01:00
|
|
|
|
2015-12-14 17:02:00 +01:00
|
|
|
/**
|
|
|
|
|
* Test utility: fuzzing mode means that GraphStage events are not processed
|
|
|
|
|
* in FIFO order within a fused subgraph, but randomized.
|
|
|
|
|
*/
|
|
|
|
|
def withFuzzing(enable: Boolean): ActorMaterializerSettings =
|
2015-11-09 16:19:12 +01:00
|
|
|
if (enable == this.fuzzingMode) this
|
|
|
|
|
else copy(fuzzingMode = enable)
|
|
|
|
|
|
2015-12-14 17:02:00 +01:00
|
|
|
/**
|
|
|
|
|
* Maximum number of elements emitted in batch if downstream signals large demand.
|
|
|
|
|
*/
|
|
|
|
|
def withOutputBurstLimit(limit: Int): ActorMaterializerSettings =
|
|
|
|
|
if (limit == this.outputBurstLimit) this
|
|
|
|
|
else copy(outputBurstLimit = limit)
|
|
|
|
|
|
2016-02-02 16:39:47 -05:00
|
|
|
/**
|
|
|
|
|
* Limit for number of messages that can be processed synchronously in stream to substream communication
|
|
|
|
|
*/
|
|
|
|
|
def withSyncProcessingLimit(limit: Int): ActorMaterializerSettings =
|
|
|
|
|
if (limit == this.syncProcessingLimit) this
|
|
|
|
|
else copy(syncProcessingLimit = limit)
|
|
|
|
|
|
2015-12-14 17:02:00 +01:00
|
|
|
/**
|
|
|
|
|
* Enable to log all elements that are dropped due to failures (at DEBUG level).
|
|
|
|
|
*/
|
|
|
|
|
def withDebugLogging(enable: Boolean): ActorMaterializerSettings =
|
2015-11-09 16:19:12 +01:00
|
|
|
if (enable == this.debugLogging) this
|
|
|
|
|
else copy(debugLogging = enable)
|
2015-12-14 17:02:00 +01:00
|
|
|
|
2016-02-07 14:54:48 +01:00
|
|
|
/**
|
|
|
|
|
* Configure the maximum buffer size for which a FixedSizeBuffer will be preallocated.
|
|
|
|
|
* This defaults to a large value because it is usually better to fail early when
|
|
|
|
|
* system memory is not sufficient to hold the buffer.
|
|
|
|
|
*/
|
|
|
|
|
def withMaxFixedBufferSize(size: Int): ActorMaterializerSettings =
|
|
|
|
|
if (size == this.maxFixedBufferSize) this
|
|
|
|
|
else copy(maxFixedBufferSize = size)
|
|
|
|
|
|
2015-12-14 17:02:00 +01:00
|
|
|
/**
|
|
|
|
|
* Leaked publishers and subscribers are cleaned up when they are not used within a given
|
|
|
|
|
* deadline, configured by [[StreamSubscriptionTimeoutSettings]].
|
|
|
|
|
*/
|
|
|
|
|
def withSubscriptionTimeoutSettings(settings: StreamSubscriptionTimeoutSettings): ActorMaterializerSettings =
|
|
|
|
|
if (settings == this.subscriptionTimeoutSettings) this
|
|
|
|
|
else copy(subscriptionTimeoutSettings = settings)
|
2015-01-27 13:36:13 +01:00
|
|
|
|
2017-11-07 03:07:29 +01:00
|
|
|
def withIOSettings(ioSettings: IOSettings): ActorMaterializerSettings =
|
|
|
|
|
if (ioSettings == this.ioSettings) this
|
|
|
|
|
else copy(ioSettings = ioSettings)
|
|
|
|
|
|
2018-01-04 17:21:47 +01:00
|
|
|
/** Change settings specific to [[SourceRef]] and [[SinkRef]]. */
|
|
|
|
|
def withStreamRefSettings(streamRefSettings: StreamRefSettings): ActorMaterializerSettings =
|
|
|
|
|
if (streamRefSettings == this.streamRefSettings) this
|
|
|
|
|
else copy(streamRefSettings = streamRefSettings)
|
|
|
|
|
|
2018-02-22 13:42:59 +01:00
|
|
|
def withBlockingIoDispatcher(newBlockingIoDispatcher: String): ActorMaterializerSettings =
|
|
|
|
|
if (newBlockingIoDispatcher == blockingIoDispatcher) this
|
|
|
|
|
else copy(blockingIoDispatcher = newBlockingIoDispatcher)
|
|
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
private def requirePowerOfTwo(n: Integer, name: String): Unit = {
|
|
|
|
|
require(n > 0, s"$name must be > 0")
|
|
|
|
|
require((n & (n - 1)) == 0, s"$name must be a power of two")
|
|
|
|
|
}
|
2015-12-14 17:02:00 +01:00
|
|
|
|
|
|
|
|
override def equals(other: Any): Boolean = other match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case s: ActorMaterializerSettings =>
|
2015-12-14 17:02:00 +01:00
|
|
|
s.initialInputBufferSize == initialInputBufferSize &&
|
2019-03-11 10:38:24 +01:00
|
|
|
s.maxInputBufferSize == maxInputBufferSize &&
|
|
|
|
|
s.dispatcher == dispatcher &&
|
|
|
|
|
s.supervisionDecider == supervisionDecider &&
|
|
|
|
|
s.subscriptionTimeoutSettings == subscriptionTimeoutSettings &&
|
|
|
|
|
s.debugLogging == debugLogging &&
|
|
|
|
|
s.outputBurstLimit == outputBurstLimit &&
|
|
|
|
|
s.syncProcessingLimit == syncProcessingLimit &&
|
|
|
|
|
s.fuzzingMode == fuzzingMode &&
|
|
|
|
|
s.autoFusing == autoFusing &&
|
|
|
|
|
s.ioSettings == ioSettings &&
|
|
|
|
|
s.blockingIoDispatcher == blockingIoDispatcher
|
2019-02-09 15:25:39 +01:00
|
|
|
case _ => false
|
2015-12-14 17:02:00 +01:00
|
|
|
}
|
|
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
override def toString: String =
|
|
|
|
|
s"ActorMaterializerSettings($initialInputBufferSize,$maxInputBufferSize," +
|
2017-11-07 03:07:29 +01:00
|
|
|
s"$dispatcher,$supervisionDecider,$subscriptionTimeoutSettings,$debugLogging,$outputBurstLimit," +
|
|
|
|
|
s"$syncProcessingLimit,$fuzzingMode,$autoFusing,$ioSettings)"
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
object IOSettings {
|
|
|
|
|
def apply(system: ActorSystem): IOSettings =
|
|
|
|
|
apply(system.settings.config.getConfig("akka.stream.materializer.io"))
|
|
|
|
|
|
|
|
|
|
def apply(config: Config): IOSettings =
|
2019-03-11 10:38:24 +01:00
|
|
|
new IOSettings(tcpWriteBufferSize = math.min(Int.MaxValue, config.getBytes("tcp.write-buffer-size")).toInt)
|
2017-11-07 03:07:29 +01:00
|
|
|
|
|
|
|
|
def apply(tcpWriteBufferSize: Int): IOSettings =
|
|
|
|
|
new IOSettings(tcpWriteBufferSize)
|
|
|
|
|
|
|
|
|
|
/** Java API */
|
|
|
|
|
def create(config: Config) = apply(config)
|
|
|
|
|
|
|
|
|
|
/** Java API */
|
|
|
|
|
def create(system: ActorSystem) = apply(system)
|
|
|
|
|
|
|
|
|
|
/** Java API */
|
|
|
|
|
def create(tcpWriteBufferSize: Int): IOSettings =
|
|
|
|
|
apply(tcpWriteBufferSize)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
final class IOSettings private (val tcpWriteBufferSize: Int) {
|
|
|
|
|
|
|
|
|
|
def withTcpWriteBufferSize(value: Int): IOSettings = copy(tcpWriteBufferSize = value)
|
|
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
private def copy(tcpWriteBufferSize: Int): IOSettings = new IOSettings(tcpWriteBufferSize = tcpWriteBufferSize)
|
2017-11-07 03:07:29 +01:00
|
|
|
|
|
|
|
|
override def equals(other: Any): Boolean = other match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case s: IOSettings => s.tcpWriteBufferSize == tcpWriteBufferSize
|
|
|
|
|
case _ => false
|
2017-11-07 03:07:29 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def toString =
|
|
|
|
|
s"""IoSettings(${tcpWriteBufferSize})"""
|
2014-04-01 15:19:42 +02:00
|
|
|
}
|
2014-11-03 15:29:02 +01:00
|
|
|
|
|
|
|
|
object StreamSubscriptionTimeoutSettings {
|
2015-01-28 14:19:50 +01:00
|
|
|
import akka.stream.StreamSubscriptionTimeoutTerminationMode._
|
2014-11-03 15:29:02 +01:00
|
|
|
|
2015-12-14 17:02:00 +01:00
|
|
|
/**
|
|
|
|
|
* Create settings from individual values (Java).
|
|
|
|
|
*/
|
2019-03-13 10:56:20 +01:00
|
|
|
def create(
|
|
|
|
|
mode: StreamSubscriptionTimeoutTerminationMode,
|
|
|
|
|
timeout: FiniteDuration): StreamSubscriptionTimeoutSettings =
|
2015-12-14 17:02:00 +01:00
|
|
|
new StreamSubscriptionTimeoutSettings(mode, timeout)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Create settings from individual values (Scala).
|
|
|
|
|
*/
|
2019-03-13 10:56:20 +01:00
|
|
|
def apply(
|
|
|
|
|
mode: StreamSubscriptionTimeoutTerminationMode,
|
|
|
|
|
timeout: FiniteDuration): StreamSubscriptionTimeoutSettings =
|
2015-05-05 12:32:49 +02:00
|
|
|
new StreamSubscriptionTimeoutSettings(mode, timeout)
|
|
|
|
|
|
2015-12-14 17:02:00 +01:00
|
|
|
/**
|
|
|
|
|
* Create settings from a Config subsection (Java).
|
|
|
|
|
*/
|
2014-11-03 15:29:02 +01:00
|
|
|
def create(config: Config): StreamSubscriptionTimeoutSettings =
|
|
|
|
|
apply(config)
|
|
|
|
|
|
2015-12-14 17:02:00 +01:00
|
|
|
/**
|
|
|
|
|
* Create settings from a Config subsection (Scala).
|
|
|
|
|
*/
|
2014-11-03 15:29:02 +01:00
|
|
|
def apply(config: Config): StreamSubscriptionTimeoutSettings = {
|
|
|
|
|
val c = config.getConfig("subscription-timeout")
|
2019-03-11 10:38:24 +01:00
|
|
|
StreamSubscriptionTimeoutSettings(mode = toRootLowerCase(c.getString("mode")) match {
|
|
|
|
|
case "no" | "off" | "false" | "noop" => NoopTermination
|
|
|
|
|
case "warn" => WarnTermination
|
|
|
|
|
case "cancel" => CancelTermination
|
|
|
|
|
}, timeout = c.getDuration("timeout", TimeUnit.MILLISECONDS).millis)
|
2014-11-03 15:29:02 +01:00
|
|
|
}
|
|
|
|
|
}
|
2015-05-05 12:32:49 +02:00
|
|
|
|
2015-12-14 17:02:00 +01:00
|
|
|
/**
|
|
|
|
|
* Leaked publishers and subscribers are cleaned up when they are not used within a given
|
|
|
|
|
* deadline, configured by [[StreamSubscriptionTimeoutSettings]].
|
|
|
|
|
*/
|
2019-03-13 10:56:20 +01:00
|
|
|
final class StreamSubscriptionTimeoutSettings(
|
|
|
|
|
val mode: StreamSubscriptionTimeoutTerminationMode,
|
|
|
|
|
val timeout: FiniteDuration) {
|
2015-12-14 17:02:00 +01:00
|
|
|
override def equals(other: Any): Boolean = other match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case s: StreamSubscriptionTimeoutSettings => s.mode == mode && s.timeout == timeout
|
|
|
|
|
case _ => false
|
2015-12-14 17:02:00 +01:00
|
|
|
}
|
|
|
|
|
override def toString: String = s"StreamSubscriptionTimeoutSettings($mode,$timeout)"
|
|
|
|
|
}
|
2014-11-03 15:29:02 +01:00
|
|
|
|
2015-12-14 17:02:00 +01:00
|
|
|
/**
|
|
|
|
|
* This mode describes what shall happen when the subscription timeout expires for
|
|
|
|
|
* substream Publishers created by operations like `prefixAndTail`.
|
|
|
|
|
*/
|
2014-11-03 15:29:02 +01:00
|
|
|
sealed abstract class StreamSubscriptionTimeoutTerminationMode
|
2014-11-12 13:05:57 +01:00
|
|
|
|
2014-11-03 15:29:02 +01:00
|
|
|
object StreamSubscriptionTimeoutTerminationMode {
|
2014-11-12 13:05:57 +01:00
|
|
|
case object NoopTermination extends StreamSubscriptionTimeoutTerminationMode
|
|
|
|
|
case object WarnTermination extends StreamSubscriptionTimeoutTerminationMode
|
|
|
|
|
case object CancelTermination extends StreamSubscriptionTimeoutTerminationMode
|
|
|
|
|
|
2015-12-14 17:02:00 +01:00
|
|
|
/**
|
|
|
|
|
* Do not do anything when timeout expires.
|
|
|
|
|
*/
|
|
|
|
|
def noop: StreamSubscriptionTimeoutTerminationMode = NoopTermination
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Log a warning when the timeout expires.
|
|
|
|
|
*/
|
|
|
|
|
def warn: StreamSubscriptionTimeoutTerminationMode = WarnTermination
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* When the timeout expires attach a Subscriber that will immediately cancel its subscription.
|
|
|
|
|
*/
|
|
|
|
|
def cancel: StreamSubscriptionTimeoutTerminationMode = CancelTermination
|
2014-11-12 13:05:57 +01:00
|
|
|
|
2014-11-03 15:29:02 +01:00
|
|
|
}
|