org.apache.pekko.stream

Members list

Packages

The architecture of Apache Pekko Streams internally consists of several distinct layers:

The architecture of Apache Pekko Streams internally consists of several distinct layers:

  • The DSLs like org.apache.pekko.stream.scaladsl.Flow, org.apache.pekko.stream.scaladsl.Source etc. are the user facing API for composing streams. These DSLs are a thin wrappers around the internal org.apache.pekko.stream.impl.TraversalBuilder builder classes. There are Java alternatives of these DSLs in javadsl which basically wrap their scala counterpart, delegating method calls.
  • The org.apache.pekko.stream.stage.GraphStage API is the user facing API for creating new stream operators. These classes are used by the org.apache.pekko.stream.impl.fusing.GraphInterpreter which executes islands (subgraphs) of these operators
  • The high level DSLs use the org.apache.pekko.stream.impl.TraversalBuilder classes to build instances of org.apache.pekko.stream.impl.Traversal which are the representation of a materializable stream description. These builders are immutable and safely shareable. Unlike the top-level DSLs, these are classic, i.e. elements are treated as Any.
  • The org.apache.pekko.stream.impl.Traversal is the immutable, efficient representation of a stream processing graph that can be materialized. The builders exist solely for the purpose of producing a traversal in the end.
  • The org.apache.pekko.stream.impl.PhasedFusingActorMaterializer is the class that is responsible for traversing and interpreting a org.apache.pekko.stream.impl.Traversal. It delegates the actual task of creating executable entities and Publishers/Producers to org.apache.pekko.stream.impl.PhaseIslands which are plugins that understand atomic operators in the graph and able to turn them into executable entities.
  • The org.apache.pekko.stream.impl.fusing.GraphInterpreter and its actor backed wrapper org.apache.pekko.stream.impl.fusing.ActorGraphInterpreter are used to execute synchronous islands (subgraphs) of org.apache.pekko.stream.stage.GraphStages.

For the execution layer, refer to org.apache.pekko.stream.impl.fusing.GraphInterpreter.

== Design goals ==

The central piece for both the DSLs and materialization is the org.apache.pekko.stream.impl.Traversal. This is the representation of a Pekko Stream, basically a org.apache.pekko.stream.scaladsl.RunnableGraph. The design goals for org.apache.pekko.stream.impl.Traversal are:

  • Be able to materialize a graph in one pass over the traversal
  • Unify materialization and fusing. The materializer should be able to construct all the necessary data structures for the interpreters and for connecting them in one go.
  • Avoid allocations as much as possible.
  • Biased implementation for the 90% case. Common cases should be as fast as possible:
    • wiring linear chains should be very fast.
    • assume that most graphs are mostly linear, with only a few generalized graph constructs thrown in.
    • materialization should not pay the price of island tracking if there is only a single island
    • assume that the number of islands is low in general
    • avoid "copiedModule" i.e. wrappers that exist solely for the purpose of establishing new port identities for operators that are used multiple times in the same graph.
  • Avoid hashmaps and prefer direct array lookup wherever possible

Semantically, a traversal is a list of commands that the materializer must execute to turn the description to a running stream. In fact, the traversal is nothing more than an immutable list, that is expressed as a tree. A tree is used to make immutable appends fast (immutable lists only have prepend as O(1) operation, append is O(N)). The materializer "recovers" the original sequence by using a local, mutable stack to properly traverse the tree structure. This is way cheaper than to immutably append to the traversal at each addition.

The "tree-ness" is expressed by explicit org.apache.pekko.stream.impl.Concat nodes that express that two traversals need to be traversed in a certain sequence, stashing away the second on a local stack until the first is fully traversed.

While traversing the traversal (basically following Concat nodes), the materializer will visit the following command types:

  • org.apache.pekko.stream.impl.MaterializeAtomic: An atomic module needs to be materialized. This node also contains wiring information which we discuss later.
  • Materialized value computation. This is a stack based "sublanguage" to compute the final materialized value on a stack, maintained by the materializer
    • org.apache.pekko.stream.impl.PushNotUsed push a NotUsed value on the stack
    • org.apache.pekko.stream.impl.Pop pop the top of the stack and throw away
    • org.apache.pekko.stream.impl.Transform take the top of the stack, transform it with the provided function and put the result back on the top of the stack
    • org.apache.pekko.stream.impl.Compose take the top two values of the stack, invoke the provided function with these values as arguments, then put the calculated value on the top of the stack
    • Materialized values of atomic operators when visiting a org.apache.pekko.stream.impl.MaterializeAtomic must be pushed to the stack automatically. There are no explicit PUSH commands for this
  • Attributes calculation. These also are a stack language, although much simpler than the materialized value commands. For any materialized operator, the top of the attributes stack should be provided as the current effective attributes.
    • org.apache.pekko.stream.impl.PushAttributes combines the attributes on the top of the stack with the given ones and puts the result on the attributes stack
    • org.apache.pekko.stream.impl.PopAttributes removes the top of the attributes stack.
  • Island tracking. Islands serve two purposes. First, they allow a large graph to be cut into parts that execute concurrently with each other, using asynchronous message passing between themselves. Second, they are an extension point where "plugins" (org.apache.pekko.stream.impl.PhaseIsland) can be used to specially handle subgraphs. Islands can be nested in each other. This makes "holes" in the parent island. Islands also need a stack as exiting a "hole" means returning to the parent, enclosing island and continuing where left.
    • org.apache.pekko.stream.impl.EnterIsland instructs the materializer that the following commands will belong to the materialization of a new island (a subgraph). The org.apache.pekko.stream.impl.IslandTag signals to the materializer which org.apache.pekko.stream.impl.PhaseIsland should be used to turn operators of this island into executable entities.
    • org.apache.pekko.stream.impl.ExitIsland instructs the materializer that the current island is done and the parent island is now the active one again.

Please note that the stack based materialized value computation eliminates the issues present in the older materializer which expressed these computations as an AST. We had to use optimizations for this tree so that long Keep.left chains don't explode the stack visiting a large AST. The stack based language sidesteps this issue completely as the number of these commands don't increase the stack space required to execute them, unless the computation itself requires it (which is not the case in any sane stream combination).

== Graph model, offsets, slots ==

As a mental model, the wiring part of the Traversal (i.e. excluding the stack based sub-commands tracking materialized values, attributes, islands, i.e. things that don't contribute to the wiring structure of the graph) translates everything to a single, global, contiguous Array. Every input and output port of each operator is mapped to exactly one slot of this "mental array". Input and output ports that are considered wired together simply map to the same slot. (In practice, these slots might not be mapped to an actual global array, but multiple local arrays using some translation logic, but we will explain this later)

Input ports are mapped simply to contiguous numbers in the order they are visited. Take for example a simple traversal:

Operator1[in1, in2, out] - Operator2[out] - Operator3[in]

This results in the following slot assignments:

  • Operator1.in1 -> 0
  • Operator1.in2 -> 1
  • Operator3.in -> 2

The materializer simply visits Stage1, Stage2, Stage3 in order, visiting the input ports of each operator in order. It then simply assigns numbers from a counter that is incremented after visiting an input port. (Please note that all org.apache.pekko.stream.impl.StreamLayout.AtomicModules maintain a stable order of their ports, so this global ordering is well defined)

Before explaining how output wiring works, it is important to settle some terminology. When we talk about ports we refer to their location in the "mental array" as slots. However, there are other entities that needs to reference various positions in this "mental array", but in these cases we use the term offset to signify that these are only used for bookkeeping, they have no "place" in the "array" themselves. In particular:

  • offset of a module: The offset of an org.apache.pekko.stream.impl.StreamLayout.AtomicModule is defined as the value of the input port counter when visiting the org.apache.pekko.stream.impl.MaterializeAtomic node to materialize that module. In other words, the offset of a module is the slot of its first input port (if there is any). Since modules might not have any input ports it can be that different modules share the same offset, simply because the the first one visited does not increase the input port counter.
  • offset of segments, islands: Defined similarly to module. The offset of an island or a segment is simply the value of the input port counter (or the first unallocated slot).

For example:

Module1[in1 = 0, in2 = 1] - Module2[out] - Module3[in = 2]

The offset of Module1 is 0, while Module2 and Module3 share the same offset of 2. Note that only input ports (slots) contribute to the offset of a module in a traversal.

Output ports are wired relative to the offset of the module they are contained in. When the materializer visits a org.apache.pekko.stream.impl.MaterializeAtomic node, it contains an Array that maps ports to a relative offset. To calculate the slot that is assigned to an output port the following formula is used:

slot = offsetOfModule + outToSlots(out.id)

Where outToSlots is the array contained in the org.apache.pekko.stream.impl.MaterializeAtomic node.

== Relative addressing ==

The power of this structure comes from the fact that slots are assigned in a relative manner:

  • input ports are assigned in sequence so the slots assigned to the input ports of a subgraph depend on the subgraph's position in the traversal
  • output ports are assigned relative to the offset of their owner module, which is in turn relative to its first (potential) input port (which is relative, too, because of the previous point)

This setup allows combining subgraphs without touching their internal wirings as all their internal wirings will properly resolve due to everything being relative:

 +---------------+                     +----+
 |               |                     |    |

|---------Graph1---------|--- .... ---|----Graph2----|

It is important to note that due to reusability, a Pekko Stream graph may contain the same atomic or composite multiple times in the same graph. Since these must be distinguished from each other somehow, they need port mapping (i.e. a new set of ports) to ensure that the ports of one graph are distinguishable from another. Because how the traversal relative addressing works, these are temporary though, once all internal wirings are ready, these mappings can be effectively dropped as the global slot assignments uniquely identify what is wired to what. For example since Graph1 is visited before Graph2 all of the slots or offsets it uses are different from Graph2 leaving no room for misinterpretation.

== Port mapping ==

Port mapping is the way how the DSL can distinguish between multiple instances of the same graph included multiple times. For example in the Graph DSL:

val merge1 = builder.add(Merge) val merge2 = builder.add(Merge)

the port merge1.out must be different from merge2.out.

For efficiency reasons, the linear and graph DSLs use different org.apache.pekko.stream.impl.TraversalBuilder types to build the org.apache.pekko.stream.impl.Traversal (we will discuss these next). One of the differences between the two builders are their approach to port mapping.

The simpler case is the org.apache.pekko.stream.impl.LinearTraversalBuilder. This builder only allows building linear chains of operators, hence, it can only have at most one OutPort and InPort unwired. Since there is no possible ambiguity between these two port types, there is no need for port mapping for these. Conversely, for those internal ports that are already wired, there is no need for port mapping as their relative wiring is not ambiguous (see previous section). As a result, the org.apache.pekko.stream.impl.LinearTraversalBuilder does not use any port mapping.

The generic graph builder class org.apache.pekko.stream.impl.CompositeTraversalBuilder needs port mapping as it allows adding any kind of builders in any order. When adding a module (encoded as another org.apache.pekko.stream.impl.TraversalBuilder) there are two entities in play:

  • The module (builder) to be added. This builder has a few ports unwired which are usually packaged in a Shape which is stored alongside with the builder in the Graph of the DSL. When invoking methods on this builder these set of ports must be used.
  • The module that we are growing. This module needs a new set of ports to be used as it might add this module multiple times and needs to disambiguate these ports.

Adding to the org.apache.pekko.stream.impl.CompositeTraversalBuilder involves the following steps (pseudocode):

val newShape = shape.deepCopy() // Copy the shape of the module we want to add val newBuilder = builder.add(submodule, newShape) // Add the module, and register it with the new shape newBuilder.wire(newShape.in, ...) // Use the new ports to wire

What happens in the background is that Shape.deepCopy creates copies of the ports, and fills their mappedTo field to point to their original port counterpart. Whenever we call wire in the outer module, it delegates calls to the submodule, but using the original port (as the submodule builder has no knowledge of the external mapping):

submodule.assign(port.mappedTo, ...) // enclosing module delegating to submodule, translating ports back

Visualizing this relationship:

+----------------------------------+ | in', in" ---------+ | in' and in" both resolve to in | | .mappedTo v .mappedTo | but they will be used on different builders | +-------------+ +-------------+ | | | in | | in | | (delegation happens recursively in AddedModule) | | AddedModule | | AddedModule | |

It is worth to note that the submodule might also continue this map-and-delegate chain to further submodules until a builder is reached that can directly perform the operation. In other words, the depth of nesting is equal to the length of mappedTo chaining.

IMPORTANT: When wiring in the enclosing module the new ports/shape MUST be used, using the original ports/shape will lead to incorrect state.

== TraversalBuilders ==

In order to understand why builders are needed, consider wiring two ports together. Actually, we don't need to wire input ports anywhere. Their slot is implicitly assigned by their position in the traversal, there is no additional state we need to track. On the other hand, we cannot build a org.apache.pekko.stream.impl.MaterializeAtomic node until the mapping array outToSlots is fully calculated. In other words, in reality, we don't wire input ports anywhere, we only assign output ports to slots. The builders exist mainly to keep track all the necessary information to be able to assign output ports, build the outToSlots array and finally the org.apache.pekko.stream.impl.MaterializeAtomic node. The consequence of this that a org.apache.pekko.stream.impl.Traversal can be constructed as soon as all output ports are wired ("unwired" inputs don't matter).

There is a specific builder that is used for the cases where all outputs have been wired: org.apache.pekko.stream.impl.CompletedTraversalBuilder. This builder type simply contains the completed traversal plus some additional data. The reason why this builder type exists is to keep auxiliary data structures required for output port mapping only while they are needed, and shed them as soon as they are not needed anymore. Since builders may recursively contain other builders, as soon as internals are completed those contained builders transition to completed state and drop all additional data structures. This is very GC friendly as many intermediate graphs exist only in a method call, and hence most of the additional data structures are dropped before method return and can be efficiently collected by the GC.

The most generic builder is org.apache.pekko.stream.impl.CompositeTraversalBuilder. There are two main considerations this builder needs to consider:

  • Enclosed modules (builders) must have a stable position in the final traversal for relative addressing to work. Since module offsets are calculated by traversal position, and outputs are wired relative to module offset, this is critical.
  • Enclosed builders might not be complete yet (i.e. have unwired outputs) and hence they cannot immediately give a Traversal.

The composite builder keeps a temporary list of traversal steps (in reverse order because of immutable lists) it needs to create once it is completed (all outputs wired). These steps refer to the traversal of submodules as a org.apache.pekko.stream.impl.BuilderKey which is just a placeholder where the traversal of the submodule will be stitched in. This org.apache.pekko.stream.impl.BuilderKey is also a key to a map which contains the evolving builder. The importance of this "preimage" traversal is that it keeps position of submodules stable, making relative addressing possible.

Once the composite is completed, it takes these steps (now reversing it back to normal), and builds the traversal using the submodule traversals referred to by org.apache.pekko.stream.impl.BuilderKey. Note that at this point all the submodules are org.apache.pekko.stream.impl.CompletedTraversalBuilders because there are no unwired outputs and hence the Traversal can be assembled. As the builder evolves over time, more and more of its org.apache.pekko.stream.impl.BuilderKeys will refer to org.apache.pekko.stream.impl.CompletedTraversalBuilders, shedding much of the temporary data structures.

Refer to org.apache.pekko.stream.impl.CompositeTraversalBuilder for more details.

The org.apache.pekko.stream.impl.LinearTraversalBuilder is a much simpler beast. For efficiency, it tries to work as much as possible directly on the org.apache.pekko.stream.impl.Traversal avoiding auxiliary structures. The two main considerations for this builder are:

  • org.apache.pekko.stream.scaladsl.Source and org.apache.pekko.stream.scaladsl.Flow contain an unwired output port. Yet, we would like to build the traversal directly as much as possible, even though the builder is not yet completed
  • org.apache.pekko.stream.impl.CompositeTraversalBuilders might be included in a linear chain. These cannot provide a traversal before they are fully completed.

The linear builder, although it is one class, comes in basically two flavors:

  • Purely linear builder: this contains only other linear builders, or all the composites that it includes have been fully wired before and hence their traversal is now fully incorporated. Basically this kind of builder only contains the org.apache.pekko.stream.impl.Traversal and only a couple of extra fields.
  • Linear builder with an incomplete composite at the end (output): In this case, we have an incomplete composite. It can only be at the end, since this is the only position where an output port can be unwired. We need to carry this builder with us until the output port is finally wired, in which case we incorporate its traversal into the already complete one, and hopefully transition to a purely linear builder.

If we consider the purely linear case, we still need to figure out how can we provide a traversal even though the last output port is unwired. The trick that is used is to wire this output port optimistically to the relative address -1 which is almost always correct (why -1? explained a bit later). If it turns out to be incorrect later, we fix it by the helper method org.apache.pekko.stream.impl.Traversal.rewireFirstTo which tears down the traversal until the wrong module is found, then fixes the port assignment. This is only possible on purely linear layouts though. Again, this is an example of the 90% rule. Most appends will not need this rewiring and hence be as fast as possible while the rarer cases suffering a minor penalty.

In the case where the last module is a composite, the above trick would not work as nothing guarantees that the module that exposed its output port is at an expected position in the traversal. Instead, we simply keep around this composite and delay construction of its part of the traversal. For details see org.apache.pekko.stream.impl.LinearTraversalBuilder as these cases are heavily commented and explained in the code.

There is another peculiarity of the linear builder we need to explain. Namely, it builds the traversal in reverse order, i.e. from Sinks towards Sources. THIS CAN BE SUPER CONFUSING AT TIMES SO PAY ATTENTION! There are two important reasons why this is needed:

  • Prepending to immutable lists is more efficient. Even though we encode our traversal list as a tree, we would need stack space at materialization time as much as the length of the list if we would append to it instead of prepending.
  • Prepending means that most output ports refer to slots visited before, i.e. output relative offsets are negative. This means that during materialization, output ports will be wired to slots that the materializer visited before which enables an efficient one-pass materialization design. The importance of this is discussed later below.

To visualize this, imagine a simple stream:

[Source.out] -> [Map.in, Map.out] -> [Sink.in]

The traversal:

offs = 0 offs = 1 offs = 1 [Sink.in = 0] <- [Map.in = 1, Map.out = -1] <- [Source.out = -1]

Since the traversal steps are reversed compared to the DSL order, it is important to reverse materialized value computation, too.

== Islands and local slots ==

All what we have discussed so far referred to the "mental array", the global address space in which slots are assigned to ports. This model describes the wiring of the graph perfectly, but it does not map to the local data structures needed by materialization when there are islands present. One of the important goals of this layout data structure is to be able to produce the data structures used by the org.apache.pekko.stream.impl.fusing.GraphInterpreter directly, without much translation. Unfortunately if there is an island inside a traversal, it might leave gaps in the address space:

|----Island1-----|----Island2(enclosed)----|-----Island1-----|

Since we visit Island2 before returning to Island1, the naive approach would leave a large gap between the last input port visited before entering Island2 and the first input port visited when returning to Island1. What we would like to have instead is a contiguous slot assignment from the viewpoint of Island1. This is where org.apache.pekko.stream.impl.PhasedFusingActorMaterializer and its org.apache.pekko.stream.impl.IslandTracking helper comes into the picture. These classes do the heavy-lifting of traversing the traversal and then mapping global slots to slots local to the island, delegating then the local wiring to org.apache.pekko.stream.impl.PhaseIsland implementations. For example the org.apache.pekko.stream.impl.GraphStageIsland sees only a contigous slot-space and hence it can directly construct the array for the interpreter. It is not aware of the presence of other islands or how it is represented in the global slot-space.

== Materialization ==

Materialzation is orchestrated by the org.apache.pekko.stream.impl.PhasedFusingActorMaterializer. It basically decodes the traversal and handles islands. This top-level materializer does not really handle the wiring inside an island, it only handles wiring of Publishers and Subscribers that connect islands. Instead it delegates in-island wiring to org.apache.pekko.stream.impl.PhaseIslands. For example a default fused island will be actually wired by org.apache.pekko.stream.impl.GraphStageIsland.

First, look at a traversal that has two islands:

|----Island1-----|----Island2(enclosed)----|-----Island1-----|

In this traversal, we have two islands, and three, so called segments. Segments are simply contiguous range of slots between org.apache.pekko.stream.impl.EnterIsland or org.apache.pekko.stream.impl.ExitIsland tags (in any combination). When the materializer encounters either an enter or exit command, it saves various information about the segment it just completed (what is its offset, how long it is measured in input slots, etc.). This information is later used to figure out if a wiring crosses island boundaries or is it local to the island.

It is important to note that the data structure for this is only allocated when there are islands. This is again the 90% rule in action. In addition, these data structures are java.util.ArrayList instances, where lookups according to some value are implemented as simple linear scans. Since in 90% of the cases these structures are very short, this is the most efficient approach. Cases where this can be a performance problem are very-very special and likely not happen in practice (no graph should contain more than a dozen of islands for example).

When it comes to deciding whether a wiring is cross-island or local, there are two cases possible:

  • we encountered an output port that is wired backwards (relative address is negative). In this case we already have all the data necessary to resolve the question.
  • we encountered an output port that is wired forward (relative address is positive). In this case we have not yet visited that part of the traversal where the assignment points.

If we want to keep the one-pass design of the materializer, we need to delay forward wirings until we have all the information needed, i.e. we visit the target in port. The org.apache.pekko.stream.impl.PhasedFusingActorMaterializer has a data structure for tracking forward wires which it consults whenever it visits an input port. Again, this is only allocated if needed, and it is again an array with linear scan lookup. Once the target input port have been found, the rules of the wiring are the same as for backwards wiring.

  backward wire (to the visited part) <------+ +------> forward wire (into the unknown)
                                             | |

|----Island1-----|----Island2(enclosed)-------- ... (this is where we are now)

Remember, the org.apache.pekko.stream.impl.LinearTraversalBuilder builds its org.apache.pekko.stream.impl.Traversal in backwards order, so since most of the graphs are constructed by the linear DSLs almost all wirings will be backwards (90% rule in action again).

=== Backward wirings ===

When it comes to resolving wirings and calculating the local slots for all the islands involved there are three distinct cases.

A wiring can be in-segment:

                   +--------------+
                   |              |

|----Island1-----|----Island2(enclosed)----|-----Island1-----|

This means that the slot assigned to the output port still belongs to the current segment. This is easy to detect as the org.apache.pekko.stream.impl.IslandTracking class tracks the offset of the current segment. If the target input slot is larger or equal than this offset, and the wiring is backwards, then the wiring is strictly local to the island. The materializer will simply delegate to the org.apache.pekko.stream.impl.PhaseIsland to do the internal wiring. Since we know the offset of the segment in the local space of this island, calculating the local slot for the org.apache.pekko.stream.impl.PhaseIsland is simple. (This is fully documented with diagrams in org.apache.pekko.stream.impl.IslandTracking)

A wiring can be cross-segment, in-island:

          +---------------------------------+
          |                                 |

|----Island1-----|----Island2(enclosed)----|-----Island1-----|

In this case, the target slot is in another, but already visited segment. The org.apache.pekko.stream.impl.IslandTracking class needs to first find the segment in which the target slot is. Since each segment keeps a reference to its org.apache.pekko.stream.impl.PhaseIsland instance that handles the internal wiring a simple reference equality check will tell us if the target segment is in the same island or not. In this case it is, so all we need is to compensate for any possible holes (punched by enclosed islands) to calculate the local slot for the island and call the appropriate callback on the org.apache.pekko.stream.impl.PhaseIsland. (This is fully documented with diagrams in org.apache.pekko.stream.impl.IslandTracking)

Finally a wiring can be cross-segment, cross-island:

                   +------------------------+
                   |                        |

|----Island1-----|----Island2(enclosed)----|-----Island1-----|

This means, that the steps were similar as in the previous case until that point where we check the reference equality of the current org.apache.pekko.stream.impl.PhaseIsland with that of the target segment (we have already found the target segment). In this case, we need to calculate the local slot in the target island (similar to the previous case) and try to wire the two islands together. Now, instead of delegating the wiring to the phases, we ask the output org.apache.pekko.stream.impl.PhaseIsland to provide a Publisher and then we ask the target island to take this Publisher.

Refer to org.apache.pekko.stream.impl.IslandTracking for all the nasty details of local slot resolution. It is also recommended to try out a few examples with org.apache.pekko.stream.impl.PhasedFusingActorMaterializer.Debug turned on, it will detail every step of the island tracking and slot resolution steps.

== Utilities ==

Useful utilities are:

  • org.apache.pekko.stream.impl.PhasedFusingActorMaterializer.Debug: if this flag is turned on, the materializer will log the steps it takes
  • org.apache.pekko.stream.impl.TraversalBuilder.printTraversal: Prints the Traversal in a readable format
  • org.apache.pekko.stream.impl.TraversalBuilder.printWiring: Prints the calculated port assignments. Useful for debugging if everything is wired to the right thing.

Attributes

Scala API: The flow DSL allows the formulation of stream transformations based on some input. The starting point is called Source and can be a collection, an iterator, a block of code which is evaluated repeatedly or a org.reactivestreams.Publisher. A flow with an attached input and open output is also a Source.

Scala API: The flow DSL allows the formulation of stream transformations based on some input. The starting point is called Source and can be a collection, an iterator, a block of code which is evaluated repeatedly or a org.reactivestreams.Publisher. A flow with an attached input and open output is also a Source.

A flow may also be defined without an attached input or output and that is then a Flow. The Flow can be connected to the Source later by using Source#via with the flow as argument, and it remains a Source.

Transformations can be appended to Source and Flow with the operations defined in FlowOps. Each DSL element produces a new flow that can be further transformed, building up a description of the complete transformation pipeline.

The termination point of a flow is called Sink and can for example be a Future or org.reactivestreams.Subscriber. A flow with an attached output and open input is also a Sink.

If a flow has both an attached input and an attached output it becomes a RunnableGraph. In order to execute this pipeline the flow must be materialized by calling RunnableGraph#run on it.

You can create your Source, Flow and Sink in any order and then wire them together before they are materialized by connecting them using Flow#via and Flow#to, or connecting them into a GraphDSL with fan-in and fan-out elements.

See Reactive Streams for details on org.reactivestreams.Publisher and org.reactivestreams.Subscriber.

It should be noted that the streams modeled by this library are “hot”, meaning that they asynchronously flow through a series of processors without detailed control by the user. In particular it is not predictable how many elements a given transformation step might buffer before handing elements downstream, which means that transformation functions may be invoked more often than for corresponding transformations on strict collections like scala.collection.immutable.List. An important consequence is that elements that were produced into a stream may be discarded by later processors, e.g. when using the #take operator.

By default every operation is executed within its own org.apache.pekko.actor.Actor to enable full pipelining of the chained set of computations. This behavior is determined by the org.apache.pekko.stream.Materializer which is required by those methods that materialize the Flow into a series of org.reactivestreams.Processor instances. The returned reactive stream is fully started and active.

Attributes

Type members

Classlikes

Signal that the operator was abruptly terminated, usually seen as a call to postStop of the GraphStageLogic without 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.

Signal that the operator was abruptly terminated, usually seen as a call to postStop of the GraphStageLogic without 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.

Attributes

Source
ActorMaterializer.scala
Supertypes
trait NoStackTrace
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
sealed class AbruptStreamTerminationException(msg: String, cause: Throwable) extends RuntimeException, NoStackTrace

A base exception for abrupt stream termination.

A base exception for abrupt stream termination.

Attributes

Source
ActorMaterializer.scala
Supertypes
trait NoStackTrace
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
Known subtypes

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.

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.

Attributes

Source
ActorMaterializer.scala
Supertypes
trait Product
trait Equals
trait NoStackTrace
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
abstract class AbstractShape extends Shape

Java API for creating custom Shape types.

Java API for creating custom Shape types.

Attributes

Source
Shape.scala
Supertypes
class Shape
class Object
trait Matchable
class Any

Attributes for the Materializer. Note that more attributes defined in Attributes.

Attributes for the Materializer. Note that more attributes defined in Attributes.

Attributes

Source
Attributes.scala
Supertypes
class Object
trait Matchable
class Any
Self type

Attributes

Companion
class
Source
ActorMaterializer.scala
Supertypes
class Object
trait Matchable
class Any
Self type

Attributes

Companion
class
Source
ActorMaterializer.scala
Supertypes
class Object
trait Matchable
class Any
Self type

This class describes the configurable properties of the ActorMaterializer. Please refer to the withX methods for descriptions of the individual settings.

This class describes the configurable properties of the ActorMaterializer. Please refer to the withX methods for descriptions of the individual settings.

The constructor is not public API, use create or apply on the ActorMaterializerSettings companion instead.

Attributes

Companion
object
Source
ActorMaterializer.scala
Supertypes
class Object
trait Matchable
class Any
case class AmorphousShape(inlets: Seq[Inlet[_]], outlets: Seq[Outlet[_]]) extends Shape

This type of Shape can express any number of inputs and outputs at the expense of forgetting about their specific types. It is used mainly in the implementation of the Graph builders and typically replaced by a more meaningful type of Shape when the building is finished.

This type of Shape can express any number of inputs and outputs at the expense of forgetting about their specific types. It is used mainly in the implementation of the Graph builders and typically replaced by a more meaningful type of Shape when the building is finished.

Attributes

Source
Shape.scala
Supertypes
trait Serializable
trait Product
trait Equals
class Shape
class Object
trait Matchable
class Any
Show all
final case class Attributes(attributeList: List[Attribute])

Holds attributes which can be used to alter pekko.stream.scaladsl.Flow / pekko.stream.javadsl.Flow or pekko.stream.scaladsl.GraphDSL / pekko.stream.javadsl.GraphDSL materialization.

Note that more attributes for the Materializer are defined in ActorAttributes.

The attributeList is ordered with the most specific attribute first, least specific last. Note that the order was the opposite in Akka 2.4.x.

Operators should in general not access the attributeList but instead use get to get the expected value of an attribute.

Attributes

Companion
object
Source
Attributes.scala
Supertypes
trait Serializable
trait Product
trait Equals
class Object
trait Matchable
class Any
Show all
object Attributes

Note that more attributes for the Materializer are defined in ActorAttributes.

Note that more attributes for the Materializer are defined in ActorAttributes.

Attributes

Companion
class
Source
Attributes.scala
Supertypes
trait Product
trait Mirror
class Object
trait Matchable
class Any
Self type
Attributes.type

Attributes

Source
StreamTimeoutException.scala
Supertypes
trait NoStackTrace
class TimeoutException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
final case class BidiShape[-In1, +Out1, -In2, +Out2](in1: Inlet[In1], out1: Outlet[Out1], in2: Inlet[In2], out2: Outlet[Out2]) extends Shape

A bidirectional flow of elements that consequently has two inputs and two outputs, arranged like this:

A bidirectional flow of elements that consequently has two inputs and two outputs, arranged like this:

      +------+
In1 ~>|      |~> Out1
      | bidi |
Out2 <~|      |<~ In2
      +------+

Attributes

Companion
object
Source
Shape.scala
Supertypes
trait Serializable
trait Product
trait Equals
class Shape
class Object
trait Matchable
class Any
Show all
object BidiShape

Attributes

Companion
class
Source
Shape.scala
Supertypes
trait Product
trait Mirror
class Object
trait Matchable
class Any
Self type
BidiShape.type

Attributes

Companion
object
Source
StreamTcpException.scala
Supertypes
trait NoStackTrace
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
Known subtypes

A queue of the given size that gives immediate feedback whether an element could be enqueued or not.

A queue of the given size that gives immediate feedback whether an element could be enqueued or not.

Not for user extension

Attributes

Source
BoundedSourceQueue.scala
Supertypes
class Object
trait Matchable
class Any
final case class BufferOverflowException(msg: String) extends RuntimeException

Attributes

Source
OverflowStrategy.scala
Supertypes
trait Product
trait Equals
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
sealed abstract class Client extends TLSRole

The client is usually the side that consumes the service provided by its interlocutor. The precise interpretation of this role is protocol specific.

The client is usually the side that consumes the service provided by its interlocutor. The precise interpretation of this role is protocol specific.

Attributes

Companion
object
Source
SslTlsOptions.scala
Supertypes
class TLSRole
class Object
trait Matchable
class Any
Known subtypes
object Client
case object Client extends Client

Attributes

Companion
class
Source
SslTlsOptions.scala
Supertypes
trait Singleton
trait Product
trait Mirror
trait Serializable
trait Product
trait Equals
class Client
class TLSRole
class Object
trait Matchable
class Any
Show all
Self type
Client.type
sealed abstract class ClosedShape extends Shape

This Shape is used for graphs that have neither open inputs nor open outputs. Only such a Graph can be materialized by a Materializer.

This Shape is used for graphs that have neither open inputs nor open outputs. Only such a Graph can be materialized by a Materializer.

Attributes

Companion
object
Source
Shape.scala
Supertypes
class Shape
class Object
trait Matchable
class Any
Known subtypes
object ClosedShape
object ClosedShape extends ClosedShape

Attributes

Companion
class
Source
Shape.scala
Supertypes
trait Sum
trait Mirror
class ClosedShape
class Shape
class Object
trait Matchable
class Any
Show all
Self type

Attributes

Companion
object
Source
CompletionStrategy.scala
Supertypes
class Object
trait Matchable
class Any
case object CompletionStrategy

Attributes

Companion
trait
Source
CompletionStrategy.scala
Supertypes
trait Singleton
trait Product
trait Mirror
trait Serializable
trait Product
trait Equals
class Object
trait Matchable
class Any
Show all
Self type

Attributes

Source
StreamTimeoutException.scala
Supertypes
trait NoStackTrace
class TimeoutException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all

Attributes

Source
StreamTcpException.scala
Supertypes
trait NoStackTrace
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
sealed abstract class DelayOverflowStrategy extends Serializable

Represents a strategy that decides how to deal with a buffer of time based operator that is full but is about to receive a new element.

Represents a strategy that decides how to deal with a buffer of time based operator that is full but is about to receive a new element.

Attributes

Companion
object
Source
OverflowStrategy.scala
Supertypes
trait Serializable
class Object
trait Matchable
class Any
Known subtypes

Attributes

Companion
class
Source
OverflowStrategy.scala
Supertypes
trait Sum
trait Mirror
class Object
trait Matchable
class Any
Self type
sealed abstract class EagerClose extends TLSClosing

Attributes

Companion
object
Source
SslTlsOptions.scala
Supertypes
class TLSClosing
class Object
trait Matchable
class Any
Known subtypes
object EagerClose
case object EagerClose extends EagerClose

Attributes

Companion
class
Source
SslTlsOptions.scala
Supertypes
trait Singleton
trait Product
trait Mirror
trait Serializable
trait Product
trait Equals
class EagerClose
class TLSClosing
class Object
trait Matchable
class Any
Show all
Self type
EagerClose.type
object FanInShape

Attributes

Companion
class
Source
FanInShape.scala
Supertypes
class Object
trait Matchable
class Any
Self type
FanInShape.type
abstract class FanInShape[+O] extends Shape

Attributes

Companion
object
Source
FanInShape.scala
Supertypes
class Shape
class Object
trait Matchable
class Any
Known subtypes
class FanInShape10[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, O]
class FanInShape11[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, O]
class FanInShape12[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, O]
class FanInShape13[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, O]
class FanInShape14[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, O]
class FanInShape15[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, O]
class FanInShape16[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, O]
class FanInShape17[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, O]
class FanInShape18[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, O]
class FanInShape19[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, O]
class FanInShape1N[T0, T1, O]
class FanInShape2[T0, T1, O]
class FanInShape20[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, O]
class FanInShape21[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, O]
class FanInShape22[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, O]
class FanInShape3[T0, T1, T2, O]
class FanInShape4[T0, T1, T2, T3, O]
class FanInShape5[T0, T1, T2, T3, T4, O]
class FanInShape6[T0, T1, T2, T3, T4, T5, O]
class FanInShape7[T0, T1, T2, T3, T4, T5, T6, O]
class FanInShape8[T0, T1, T2, T3, T4, T5, T6, T7, O]
class FanInShape9[T0, T1, T2, T3, T4, T5, T6, T7, T8, O]
class UniformFanInShape[T, O]
Show all
class FanInShape10[-T0, -T1, -T2, -T3, -T4, -T5, -T6, -T7, -T8, -T9, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape11[-T0, -T1, -T2, -T3, -T4, -T5, -T6, -T7, -T8, -T9, -T10, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape12[-T0, -T1, -T2, -T3, -T4, -T5, -T6, -T7, -T8, -T9, -T10, -T11, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape13[-T0, -T1, -T2, -T3, -T4, -T5, -T6, -T7, -T8, -T9, -T10, -T11, -T12, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape14[-T0, -T1, -T2, -T3, -T4, -T5, -T6, -T7, -T8, -T9, -T10, -T11, -T12, -T13, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape15[-T0, -T1, -T2, -T3, -T4, -T5, -T6, -T7, -T8, -T9, -T10, -T11, -T12, -T13, -T14, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape16[-T0, -T1, -T2, -T3, -T4, -T5, -T6, -T7, -T8, -T9, -T10, -T11, -T12, -T13, -T14, -T15, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape17[-T0, -T1, -T2, -T3, -T4, -T5, -T6, -T7, -T8, -T9, -T10, -T11, -T12, -T13, -T14, -T15, -T16, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape18[-T0, -T1, -T2, -T3, -T4, -T5, -T6, -T7, -T8, -T9, -T10, -T11, -T12, -T13, -T14, -T15, -T16, -T17, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape19[-T0, -T1, -T2, -T3, -T4, -T5, -T6, -T7, -T8, -T9, -T10, -T11, -T12, -T13, -T14, -T15, -T16, -T17, -T18, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape2[-T0, -T1, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape20[-T0, -T1, -T2, -T3, -T4, -T5, -T6, -T7, -T8, -T9, -T10, -T11, -T12, -T13, -T14, -T15, -T16, -T17, -T18, -T19, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape21[-T0, -T1, -T2, -T3, -T4, -T5, -T6, -T7, -T8, -T9, -T10, -T11, -T12, -T13, -T14, -T15, -T16, -T17, -T18, -T19, -T20, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape22[-T0, -T1, -T2, -T3, -T4, -T5, -T6, -T7, -T8, -T9, -T10, -T11, -T12, -T13, -T14, -T15, -T16, -T17, -T18, -T19, -T20, -T21, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape3[-T0, -T1, -T2, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape4[-T0, -T1, -T2, -T3, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape5[-T0, -T1, -T2, -T3, -T4, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape6[-T0, -T1, -T2, -T3, -T4, -T5, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape7[-T0, -T1, -T2, -T3, -T4, -T5, -T6, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape8[-T0, -T1, -T2, -T3, -T4, -T5, -T6, -T7, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
class FanInShape9[-T0, -T1, -T2, -T3, -T4, -T5, -T6, -T7, -T8, +O](_init: Init[O]) extends FanInShape[O]

Attributes

Source
FanInShapeN.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
object FanOutShape

Attributes

Companion
class
Source
FanOutShape.scala
Supertypes
class Object
trait Matchable
class Any
Self type
abstract class FanOutShape[-I] extends Shape

Attributes

Companion
object
Source
FanOutShape.scala
Supertypes
class Shape
class Object
trait Matchable
class Any
Known subtypes
class FanOutShape10[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9]
class FanOutShape11[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10]
class FanOutShape12[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11]
class FanOutShape13[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12]
class FanOutShape14[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13]
class FanOutShape15[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14]
class FanOutShape16[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14, O15]
class FanOutShape17[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14, O15, O16]
class FanOutShape18[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14, O15, O16, O17]
class FanOutShape19[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14, O15, O16, O17, O18]
class FanOutShape2[I, O0, O1]
class FanOutShape20[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14, O15, O16, O17, O18, O19]
class FanOutShape21[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14, O15, O16, O17, O18, O19, O20]
class FanOutShape22[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14, O15, O16, O17, O18, O19, O20, O21]
class FanOutShape3[I, O0, O1, O2]
class FanOutShape4[I, O0, O1, O2, O3]
class FanOutShape5[I, O0, O1, O2, O3, O4]
class FanOutShape6[I, O0, O1, O2, O3, O4, O5]
class FanOutShape7[I, O0, O1, O2, O3, O4, O5, O6]
class FanOutShape8[I, O0, O1, O2, O3, O4, O5, O6, O7]
class FanOutShape9[I, O0, O1, O2, O3, O4, O5, O6, O7, O8]
class UniformFanOutShape[I, O]
Show all
class FanOutShape10[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6, +O7, +O8, +O9](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape11[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6, +O7, +O8, +O9, +O10](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape12[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6, +O7, +O8, +O9, +O10, +O11](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape13[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6, +O7, +O8, +O9, +O10, +O11, +O12](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape14[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6, +O7, +O8, +O9, +O10, +O11, +O12, +O13](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape15[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6, +O7, +O8, +O9, +O10, +O11, +O12, +O13, +O14](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape16[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6, +O7, +O8, +O9, +O10, +O11, +O12, +O13, +O14, +O15](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape17[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6, +O7, +O8, +O9, +O10, +O11, +O12, +O13, +O14, +O15, +O16](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape18[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6, +O7, +O8, +O9, +O10, +O11, +O12, +O13, +O14, +O15, +O16, +O17](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape19[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6, +O7, +O8, +O9, +O10, +O11, +O12, +O13, +O14, +O15, +O16, +O17, +O18](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape2[-I, +O0, +O1](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape20[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6, +O7, +O8, +O9, +O10, +O11, +O12, +O13, +O14, +O15, +O16, +O17, +O18, +O19](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape21[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6, +O7, +O8, +O9, +O10, +O11, +O12, +O13, +O14, +O15, +O16, +O17, +O18, +O19, +O20](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape22[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6, +O7, +O8, +O9, +O10, +O11, +O12, +O13, +O14, +O15, +O16, +O17, +O18, +O19, +O20, +O21](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape3[-I, +O0, +O1, +O2](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape4[-I, +O0, +O1, +O2, +O3](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape5[-I, +O0, +O1, +O2, +O3, +O4](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape6[-I, +O0, +O1, +O2, +O3, +O4, +O5](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape7[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape8[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6, +O7](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
class FanOutShape9[-I, +O0, +O1, +O2, +O3, +O4, +O5, +O6, +O7, +O8](_init: Init[I]) extends FanOutShape[I]

Attributes

Source
FanOutShapeN.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
trait FlowMonitor[+T]

Used to monitor the state of a stream

Used to monitor the state of a stream

Type parameters

T

Type of messages passed by the stream

Attributes

Source
FlowMonitor.scala
Supertypes
class Object
trait Matchable
class Any

Attributes

Source
FlowMonitor.scala
Supertypes
class Object
trait Matchable
class Any
Self type
final case class FlowShape[-I, +O](in: Inlet[I], out: Outlet[O]) extends Shape

A Flow Shape has exactly one input and one output, it looks from the outside like a pipe (but it can be a complex topology of streams within of course).

A Flow Shape has exactly one input and one output, it looks from the outside like a pipe (but it can be a complex topology of streams within of course).

Attributes

Companion
object
Source
Shape.scala
Supertypes
trait Serializable
trait Product
trait Equals
class Shape
class Object
trait Matchable
class Any
Show all
object FlowShape

Attributes

Companion
class
Source
Shape.scala
Supertypes
trait Product
trait Mirror
class Object
trait Matchable
class Any
Self type
FlowShape.type
trait Graph[+S <: Shape, +M]

Not intended to be directly extended by user classes

Not intended to be directly extended by user classes

Attributes

See also
Companion
object
Source
Graph.scala
Supertypes
class Object
trait Matchable
class Any
Known subtypes
class BidiFlow[I1, O1, I2, O2, Mat]
class Flow[In, Out, Mat]
class RunnableGraph[Mat]
class Sink[In, Mat]
class Source[Out, Mat]
class BidiFlow[I1, O1, I2, O2, Mat]
class Flow[In, Out, Mat]
class RunnableGraph[Mat]
class Sink[In, Mat]
class Source[Out, Mat]
class GraphStage[S]
class Balance[T]
class Broadcast[T]
class Concat[T]
class Interleave[T]
class Merge[T]
class MergeLatest[T, M]
class MergePreferred[T]
class MergePrioritized[T]
class MergeSequence[T]
class MergeSorted[T]
class Partition[T]
class UnzipWith10[In, A1, A2, A3, A4, A5, A6, A7, A8, A9, A10]
class UnzipWith11[In, A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11]
class UnzipWith12[In, A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12]
class UnzipWith13[In, A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13]
class UnzipWith14[In, A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14]
class UnzipWith15[In, A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15]
class UnzipWith16[In, A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16]
class UnzipWith17[In, A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17]
class UnzipWith18[In, A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18]
class UnzipWith19[In, A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19]
class UnzipWith2[In, A1, A2]
class Unzip[A, B]
class UnzipWith20[In, A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20]
class UnzipWith21[In, A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21]
class UnzipWith22[In, A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22]
class UnzipWith3[In, A1, A2, A3]
class UnzipWith4[In, A1, A2, A3, A4]
class UnzipWith5[In, A1, A2, A3, A4, A5]
class UnzipWith6[In, A1, A2, A3, A4, A5, A6]
class UnzipWith7[In, A1, A2, A3, A4, A5, A6, A7]
class UnzipWith8[In, A1, A2, A3, A4, A5, A6, A7, A8]
class UnzipWith9[In, A1, A2, A3, A4, A5, A6, A7, A8, A9]
class ZipLatestWith10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, O]
class ZipLatestWith11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, O]
class ZipLatestWith12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, O]
class ZipLatestWith13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, O]
class ZipLatestWith14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, O]
class ZipLatestWith15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, O]
class ZipLatestWith16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, O]
class ZipLatestWith17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, O]
class ZipLatestWith18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, O]
class ZipLatestWith19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, O]
class ZipLatestWith2[A1, A2, O]
class ZipLatest[A, B]
class ZipLatestWith20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, O]
class ZipLatestWith21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, O]
class ZipLatestWith22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, O]
class ZipLatestWith3[A1, A2, A3, O]
class ZipLatestWith4[A1, A2, A3, A4, O]
class ZipLatestWith5[A1, A2, A3, A4, A5, O]
class ZipLatestWith6[A1, A2, A3, A4, A5, A6, O]
class ZipLatestWith7[A1, A2, A3, A4, A5, A6, A7, O]
class ZipLatestWith8[A1, A2, A3, A4, A5, A6, A7, A8, O]
class ZipLatestWith9[A1, A2, A3, A4, A5, A6, A7, A8, A9, O]
class ZipWith10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, O]
class ZipWith11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, O]
class ZipWith12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, O]
class ZipWith13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, O]
class ZipWith14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, O]
class ZipWith15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, O]
class ZipWith16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, O]
class ZipWith17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, O]
class ZipWith18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, O]
class ZipWith19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, O]
class ZipWith2[A1, A2, O]
class Zip[A, B]
class ZipWith20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, O]
class ZipWith21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, O]
class ZipWith22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, O]
class ZipWith3[A1, A2, A3, O]
class ZipWith4[A1, A2, A3, A4, O]
class ZipWith5[A1, A2, A3, A4, A5, O]
class ZipWith6[A1, A2, A3, A4, A5, A6, O]
class ZipWith7[A1, A2, A3, A4, A5, A6, A7, O]
class ZipWith8[A1, A2, A3, A4, A5, A6, A7, A8, O]
class ZipWith9[A1, A2, A3, A4, A5, A6, A7, A8, A9, O]
class ZipWithN[A, O]
class ZipN[A]
Show all
object Graph

Attributes

Companion
trait
Source
Graph.scala
Supertypes
class Object
trait Matchable
class Any
Self type
Graph.type
final class IOOperationIncompleteException(message: String, val count: Long, cause: Throwable) extends RuntimeException

This exception signals that a stream has been completed or has an error while there was still IO operations in progress

This exception signals that a stream has been completed or has an error while there was still IO operations in progress

Value parameters

cause

cause

count

The number of bytes read/written up until the error

Attributes

Source
IOResult.scala
Supertypes
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
final case class IOResult(count: Long, status: Try[Done])

Holds a result of an IO operation.

Holds a result of an IO operation.

Value parameters

count

Numeric value depending on context, for example IO operations performed or bytes processed.

status

Status of the result. Can be either pekko.Done or an exception.

Attributes

Companion
object
Source
IOResult.scala
Supertypes
trait Serializable
trait Product
trait Equals
class Object
trait Matchable
class Any
Show all
object IOResult

Attributes

Companion
class
Source
IOResult.scala
Supertypes
trait Product
trait Mirror
class Object
trait Matchable
class Any
Self type
IOResult.type
object IOSettings

Attributes

Companion
class
Source
ActorMaterializer.scala
Supertypes
class Object
trait Matchable
class Any
Self type
IOSettings.type
final class IOSettings

Attributes

Companion
object
Source
ActorMaterializer.scala
Supertypes
class Object
trait Matchable
class Any
sealed abstract class IgnoreBoth extends TLSClosing

Attributes

Companion
object
Source
SslTlsOptions.scala
Supertypes
class TLSClosing
class Object
trait Matchable
class Any
Known subtypes
object IgnoreBoth
case object IgnoreBoth extends IgnoreBoth

Attributes

Companion
class
Source
SslTlsOptions.scala
Supertypes
trait Singleton
trait Product
trait Mirror
trait Serializable
trait Product
trait Equals
class IgnoreBoth
class TLSClosing
class Object
trait Matchable
class Any
Show all
Self type
IgnoreBoth.type
sealed abstract class IgnoreCancel extends TLSClosing

Attributes

Companion
object
Source
SslTlsOptions.scala
Supertypes
class TLSClosing
class Object
trait Matchable
class Any
Known subtypes
object IgnoreCancel
case object IgnoreCancel extends IgnoreCancel

Attributes

Companion
class
Source
SslTlsOptions.scala
Supertypes
trait Singleton
trait Product
trait Mirror
trait Serializable
trait Product
trait Equals
class IgnoreCancel
class TLSClosing
class Object
trait Matchable
class Any
Show all
Self type
sealed abstract class IgnoreComplete extends TLSClosing

Attributes

Companion
object
Source
SslTlsOptions.scala
Supertypes
class TLSClosing
class Object
trait Matchable
class Any
Known subtypes
case object IgnoreComplete extends IgnoreComplete

Attributes

Companion
class
Source
SslTlsOptions.scala
Supertypes
trait Singleton
trait Product
trait Mirror
trait Serializable
trait Product
trait Equals
class TLSClosing
class Object
trait Matchable
class Any
Show all
Self type
sealed abstract class InPort

An input port of a StreamLayout.Module. This type logically belongs into the impl package but must live here due to how sealed works. It is also used in the Java DSL for “classic Inlets” as a work-around for otherwise unreasonable existential types.

An input port of a StreamLayout.Module. This type logically belongs into the impl package but must live here due to how sealed works. It is also used in the Java DSL for “classic Inlets” as a work-around for otherwise unreasonable existential types.

Attributes

Source
Shape.scala
Supertypes
class Object
trait Matchable
class Any
Known subtypes
class Inlet[T]
Self type
Inlet[_]

Attributes

Source
StreamTimeoutException.scala
Supertypes
trait NoStackTrace
class TimeoutException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
object Inlet

An Inlet is a typed input to a Shape. Its partner in the Module view is the InPort (which does not bear an element type because Modules only express the internal structural hierarchy of stream topologies).

An Inlet is a typed input to a Shape. Its partner in the Module view is the InPort (which does not bear an element type because Modules only express the internal structural hierarchy of stream topologies).

Attributes

Companion
class
Source
Shape.scala
Supertypes
class Object
trait Matchable
class Any
Self type
Inlet.type
final class Inlet[T] extends InPort

Attributes

Companion
object
Source
Shape.scala
Supertypes
class InPort
class Object
trait Matchable
class Any
final case class InvalidPartnerActorException(expectedRef: ActorRef, gotRef: ActorRef, msg: String) extends IllegalStateException

Stream refs establish a connection between a local and remote actor, representing the origin and remote sides of a stream. Each such actor refers to the other side as its "partner". We make sure that no other actor than the initial partner can send demand/messages to the other side accidentally.

Stream refs establish a connection between a local and remote actor, representing the origin and remote sides of a stream. Each such actor refers to the other side as its "partner". We make sure that no other actor than the initial partner can send demand/messages to the other side accidentally.

This exception is thrown when a message is received from a non-partner actor, which could mean a bug or some actively malicient behavior from the other side.

This is not meant as a security feature, but rather as plain sanity-check.

Attributes

Source
StreamRefs.scala
Supertypes
trait Product
trait Equals
class IllegalStateException
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
final case class InvalidSequenceNumberException(expectedSeqNr: Long, gotSeqNr: Long, msg: String) extends IllegalStateException

Attributes

Source
StreamRefs.scala
Supertypes
trait Product
trait Equals
class IllegalStateException
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
trait KillSwitch

A KillSwitch allows completion of Graphs from the outside by completing Graphs of FlowShape linked to the switch. Depending on whether the KillSwitch is a UniqueKillSwitch or a SharedKillSwitch one or multiple streams might be linked with the switch. For details see the documentation of the concrete subclasses of this interface.

A KillSwitch allows completion of Graphs from the outside by completing Graphs of FlowShape linked to the switch. Depending on whether the KillSwitch is a UniqueKillSwitch or a SharedKillSwitch one or multiple streams might be linked with the switch. For details see the documentation of the concrete subclasses of this interface.

Attributes

Source
KillSwitch.scala
Supertypes
class Object
trait Matchable
class Any
Known subtypes
object KillSwitches

Creates shared or single kill switches which can be used to control completion of graphs from the outside.

Creates shared or single kill switches which can be used to control completion of graphs from the outside.

  • The factory shared() returns a SharedKillSwitch which provides a Graph of FlowShape that can be used in arbitrary number of graphs and materializations. The switch simultaneously controls completion in all of those graphs.
  • The factory single() returns a Graph of FlowShape that materializes to a UniqueKillSwitch which is always unique to that materialized Flow itself.

Creates a SharedKillSwitch that can be used to externally control the completion of various streams.

Attributes

Source
KillSwitch.scala
Supertypes
class Object
trait Matchable
class Any
Self type
class MaterializationException(msg: String, cause: Throwable) extends RuntimeException

This exception or subtypes thereof should be used to signal materialization failures.

This exception or subtypes thereof should be used to signal materialization failures.

Attributes

Source
ActorMaterializer.scala
Supertypes
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
abstract class Materializer

The Materializer is the component responsible for turning a stream blueprint into a running stream. In general the system wide materializer should be preferred over creating instances manually.

The Materializer is the component responsible for turning a stream blueprint into a running stream. In general the system wide materializer should be preferred over creating instances manually.

Not for user extension

Attributes

Companion
object
Source
Materializer.scala
Supertypes
class Object
trait Matchable
class Any
Known subtypes
object Materializer

Attributes

Companion
class
Source
Materializer.scala
Supertypes
class Object
trait Matchable
class Any
Self type

Not for user extension

Not for user extension

Attributes

Source
MaterializerLoggingProvider.scala
Supertypes
class Object
trait Matchable
class Any
Known subtypes
Self type
final class NeverMaterializedException(cause: Throwable) extends RuntimeException

Attributes

Source
NeverMaterializedException.scala
Supertypes
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
sealed abstract class OutPort

An output port of a StreamLayout.Module. This type logically belongs into the impl package but must live here due to how sealed works. It is also used in the Java DSL for “classic Outlets” as a work-around for otherwise unreasonable existential types.

An output port of a StreamLayout.Module. This type logically belongs into the impl package but must live here due to how sealed works. It is also used in the Java DSL for “classic Outlets” as a work-around for otherwise unreasonable existential types.

Attributes

Source
Shape.scala
Supertypes
class Object
trait Matchable
class Any
Known subtypes
class Outlet[T]
Self type
Outlet[_]
object Outlet

An Outlet is a typed output to a Shape. Its partner in the Module view is the OutPort (which does not bear an element type because Modules only express the internal structural hierarchy of stream topologies).

An Outlet is a typed output to a Shape. Its partner in the Module view is the OutPort (which does not bear an element type because Modules only express the internal structural hierarchy of stream topologies).

Attributes

Companion
class
Source
Shape.scala
Supertypes
class Object
trait Matchable
class Any
Self type
Outlet.type
final class Outlet[T] extends OutPort

Attributes

Companion
object
Source
Shape.scala
Supertypes
class OutPort
class Object
trait Matchable
class Any
sealed abstract class OverflowStrategy extends DelayOverflowStrategy

Represents a strategy that decides how to deal with a buffer that is full but is about to receive a new element.

Represents a strategy that decides how to deal with a buffer that is full but is about to receive a new element.

Attributes

Companion
object
Source
OverflowStrategy.scala
Supertypes
trait Serializable
class Object
trait Matchable
class Any

Attributes

Companion
class
Source
OverflowStrategy.scala
Supertypes
trait Sum
trait Mirror
class Object
trait Matchable
class Any
Self type
sealed abstract class QueueCompletionResult extends QueueOfferResult

Not for user extension

Not for user extension

Attributes

Source
QueueOfferResult.scala
Supertypes
class Object
trait Matchable
class Any
Known subtypes
class Failure
object QueueClosed
sealed abstract class QueueOfferResult

Not for user extension

Not for user extension

Attributes

Companion
object
Source
QueueOfferResult.scala
Supertypes
class Object
trait Matchable
class Any
Known subtypes
class Failure
object QueueClosed
object Dropped
object Enqueued

Contains types that is used as return types for streams Source queues

Contains types that is used as return types for streams Source queues

Attributes

Companion
class
Source
QueueOfferResult.scala
Supertypes
trait Sum
trait Mirror
class Object
trait Matchable
class Any
Self type
class RateExceededException(msg: String) extends RuntimeException

Exception that is thrown when rated controlled by stream is exceeded

Exception that is thrown when rated controlled by stream is exceeded

Attributes

Source
ThrottleMode.scala
Supertypes
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
final case class RemoteStreamRefActorTerminatedException(msg: String) extends RuntimeException

Attributes

Source
StreamRefs.scala
Supertypes
trait Product
trait Equals
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
final class RestartSettings

Attributes

Companion
object
Source
RestartSettings.scala
Supertypes
class Object
trait Matchable
class Any

Attributes

Companion
class
Source
RestartSettings.scala
Supertypes
class Object
trait Matchable
class Any
Self type
sealed abstract class Server extends TLSRole

The server is usually the side the provides the service to its interlocutor. The precise interpretation of this role is protocol specific.

The server is usually the side the provides the service to its interlocutor. The precise interpretation of this role is protocol specific.

Attributes

Companion
object
Source
SslTlsOptions.scala
Supertypes
class TLSRole
class Object
trait Matchable
class Any
Known subtypes
object Server
case object Server extends Server

Attributes

Companion
class
Source
SslTlsOptions.scala
Supertypes
trait Singleton
trait Product
trait Mirror
trait Serializable
trait Product
trait Equals
class Server
class TLSRole
class Object
trait Matchable
class Any
Show all
Self type
Server.type
abstract class Shape

A Shape describes the inlets and outlets of a Graph. In keeping with the philosophy that a Graph is a freely reusable blueprint, everything that matters from the outside are the connections that can be made with it, otherwise it is just a black box.

A Shape describes the inlets and outlets of a Graph. In keeping with the philosophy that a Graph is a freely reusable blueprint, everything that matters from the outside are the connections that can be made with it, otherwise it is just a black box.

Attributes

Source
Shape.scala
Supertypes
class Object
trait Matchable
class Any
Known subtypes
class BidiShape[In1, Out1, In2, Out2]
class ClosedShape
object ClosedShape
class FanInShape[O]
class FanInShape10[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, O]
class FanInShape11[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, O]
class FanInShape12[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, O]
class FanInShape13[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, O]
class FanInShape14[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, O]
class FanInShape15[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, O]
class FanInShape16[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, O]
class FanInShape17[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, O]
class FanInShape18[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, O]
class FanInShape19[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, O]
class FanInShape1N[T0, T1, O]
class FanInShape2[T0, T1, O]
class FanInShape20[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, O]
class FanInShape21[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, O]
class FanInShape22[T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, O]
class FanInShape3[T0, T1, T2, O]
class FanInShape4[T0, T1, T2, T3, O]
class FanInShape5[T0, T1, T2, T3, T4, O]
class FanInShape6[T0, T1, T2, T3, T4, T5, O]
class FanInShape7[T0, T1, T2, T3, T4, T5, T6, O]
class FanInShape8[T0, T1, T2, T3, T4, T5, T6, T7, O]
class FanInShape9[T0, T1, T2, T3, T4, T5, T6, T7, T8, O]
class UniformFanInShape[T, O]
class FanOutShape[I]
class FanOutShape10[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9]
class FanOutShape11[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10]
class FanOutShape12[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11]
class FanOutShape13[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12]
class FanOutShape14[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13]
class FanOutShape15[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14]
class FanOutShape16[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14, O15]
class FanOutShape17[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14, O15, O16]
class FanOutShape18[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14, O15, O16, O17]
class FanOutShape19[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14, O15, O16, O17, O18]
class FanOutShape2[I, O0, O1]
class FanOutShape20[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14, O15, O16, O17, O18, O19]
class FanOutShape21[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14, O15, O16, O17, O18, O19, O20]
class FanOutShape22[I, O0, O1, O2, O3, O4, O5, O6, O7, O8, O9, O10, O11, O12, O13, O14, O15, O16, O17, O18, O19, O20, O21]
class FanOutShape3[I, O0, O1, O2]
class FanOutShape4[I, O0, O1, O2, O3]
class FanOutShape5[I, O0, O1, O2, O3, O4]
class FanOutShape6[I, O0, O1, O2, O3, O4, O5]
class FanOutShape7[I, O0, O1, O2, O3, O4, O5, O6]
class FanOutShape8[I, O0, O1, O2, O3, O4, O5, O6, O7]
class FanOutShape9[I, O0, O1, O2, O3, O4, O5, O6, O7, O8]
class UniformFanOutShape[I, O]
class FlowShape[I, O]
class SinkShape[T]
class SourceShape[T]
Show all
final class SharedKillSwitch extends KillSwitch

A SharedKillSwitch is a provider for Graphs of FlowShape that can be completed or failed from the outside. A Graph returned by the switch can be materialized arbitrary amount of times: every newly materialized Graph belongs to the switch from which it was acquired. Multiple SharedKillSwitch instances are isolated from each other, shutting down or aborting on instance does not affect the Graphs provided by another instance.

A SharedKillSwitch is a provider for Graphs of FlowShape that can be completed or failed from the outside. A Graph returned by the switch can be materialized arbitrary amount of times: every newly materialized Graph belongs to the switch from which it was acquired. Multiple SharedKillSwitch instances are isolated from each other, shutting down or aborting on instance does not affect the Graphs provided by another instance.

After calling SharedKillSwitch#shutdown all materialized, running instances of all Graphs provided by the SharedKillSwitch will complete their downstreams and cancel their upstreams (unless if finished or failed already in which case the command is ignored). Subsequent invocations of SharedKillSwitch#shutdown and SharedKillSwitch#abort will be ignored.

After calling SharedKillSwitch#abort all materialized, running instances of all Graphs provided by the SharedKillSwitch will fail their downstreams with the provided exception and cancel their upstreams (unless it finished or failed already in which case the command is ignored). Subsequent invocations of SharedKillSwitch#shutdown and SharedKillSwitch#abort will be ignored.

The Graphs provided by the SharedKillSwitch do not modify the passed through elements in any way or affect backpressure in the stream. All provided Graphs provide the parent SharedKillSwitch as materialized value.

This class is thread-safe, the instance can be passed safely among threads and its methods may be invoked concurrently.

Attributes

Source
KillSwitch.scala
Supertypes
trait KillSwitch
class Object
trait Matchable
class Any
object SinkRef

See full documentation on SinkRef.

See full documentation on SinkRef.

Attributes

Companion
trait
Source
StreamRefs.scala
Supertypes
class Object
trait Matchable
class Any
Self type
SinkRef.type

A SinkRef allows sharing a "reference" to a scaladsl.Sink with others, with the main purpose of crossing a network boundary. Usually obtaining a SinkRef would be done via Actor messaging, in which one system asks a remote one, to accept some data from it, and the remote one decides to accept the request to send data in a back-pressured streaming fashion -- using a sink ref.

A SinkRef allows sharing a "reference" to a scaladsl.Sink with others, with the main purpose of crossing a network boundary. Usually obtaining a SinkRef would be done via Actor messaging, in which one system asks a remote one, to accept some data from it, and the remote one decides to accept the request to send data in a back-pressured streaming fashion -- using a sink ref.

To create a SinkRef you have to materialize the Sink that you want to obtain a reference to by attaching it to a StreamRefs.sinkRef().

Stream refs can be seen as Reactive Streams over network boundaries. See also pekko.stream.SourceRef which is the dual of a SinkRef.

For additional configuration see reference.conf as well as pekko.stream.StreamRefAttributes.

Not for user extension.

Attributes

Companion
object
Source
StreamRefs.scala
Supertypes
class Object
trait Matchable
class Any
final case class SinkShape[-T](in: Inlet[T]) extends Shape

A Sink Shape has exactly one input and no outputs, it models a data sink.

A Sink Shape has exactly one input and no outputs, it models a data sink.

Attributes

Companion
object
Source
Shape.scala
Supertypes
trait Serializable
trait Product
trait Equals
class Shape
class Object
trait Matchable
class Any
Show all
object SinkShape

Attributes

Companion
class
Source
Shape.scala
Supertypes
trait Product
trait Mirror
class Object
trait Matchable
class Any
Self type
SinkShape.type
object SourceRef

See full documentation on SourceRef.

See full documentation on SourceRef.

Attributes

Companion
trait
Source
StreamRefs.scala
Supertypes
class Object
trait Matchable
class Any
Self type
SourceRef.type

A SourceRef allows sharing a "reference" with others, with the main purpose of crossing a network boundary. Usually obtaining a SourceRef would be done via Actor messaging, in which one system asks a remote one, to share some data with it, and the remote one decides to do so in a back-pressured streaming fashion -- using a stream ref.

A SourceRef allows sharing a "reference" with others, with the main purpose of crossing a network boundary. Usually obtaining a SourceRef would be done via Actor messaging, in which one system asks a remote one, to share some data with it, and the remote one decides to do so in a back-pressured streaming fashion -- using a stream ref.

To create a SourceRef you have to materialize the Source that you want to obtain a reference to by attaching it to a Sink.sourceRef.

Stream refs can be seen as Reactive Streams over network boundaries. See also pekko.stream.SinkRef which is the dual of a SourceRef.

For additional configuration see reference.conf as well as pekko.stream.StreamRefAttributes.

Not for user extension.

Attributes

Companion
object
Source
StreamRefs.scala
Supertypes
class Object
trait Matchable
class Any
final case class SourceShape[+T](out: Outlet[T]) extends Shape

A Source Shape has exactly one output and no inputs, it models a source of data.

A Source Shape has exactly one output and no inputs, it models a source of data.

Attributes

Companion
object
Source
Shape.scala
Supertypes
trait Serializable
trait Product
trait Equals
class Shape
class Object
trait Matchable
class Any
Show all
object SourceShape

Attributes

Companion
class
Source
Shape.scala
Supertypes
trait Product
trait Mirror
class Object
trait Matchable
class Any
Self type
final class StreamDetachedException(message: String) extends RuntimeException, NoStackTrace

This exception signals that materialized value is already detached from stream. This usually happens when stream is completed and an ActorSystem is shut down while materialized object is still available.

This exception signals that materialized value is already detached from stream. This usually happens when stream is completed and an ActorSystem is shut down while materialized object is still available.

Attributes

Source
StreamDetachedException.scala
Supertypes
trait NoStackTrace
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all

Attributes

Source
StreamTimeoutException.scala
Supertypes
trait NoStackTrace
class TimeoutException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
class StreamLimitReachedException(val n: Long) extends RuntimeException

Attributes

Source
StreamLimitReachedException.scala
Supertypes
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all

Attributes for stream refs (pekko.stream.SourceRef and pekko.stream.SinkRef). Note that more attributes defined in Attributes and ActorAttributes.

Attributes for stream refs (pekko.stream.SourceRef and pekko.stream.SinkRef). Note that more attributes defined in Attributes and ActorAttributes.

Attributes

Source
Attributes.scala
Supertypes
class Object
trait Matchable
class Any
Self type

The stream ref resolver extension provides a way to serialize and deserialize streamrefs in user serializers.

The stream ref resolver extension provides a way to serialize and deserialize streamrefs in user serializers.

Attributes

Companion
trait
Source
StreamRefs.scala
Supertypes
class Object
trait Matchable
class Any
Self type

The stream ref resolver provides a way to serialize and deserialize streamrefs in user serializers.

The stream ref resolver provides a way to serialize and deserialize streamrefs in user serializers.

Not for user extension

Attributes

Companion
object
Source
StreamRefs.scala
Supertypes
trait Extension
class Object
trait Matchable
class Any

Attributes

Companion
trait
Source
StreamRefSettings.scala
Supertypes
class Object
trait Matchable
class Any
Self type

Settings specific to SourceRef and SinkRef. More detailed documentation about each of the settings is available in reference.conf.

Settings specific to SourceRef and SinkRef. More detailed documentation about each of the settings is available in reference.conf.

Attributes

Companion
object
Source
StreamRefSettings.scala
Supertypes
class Object
trait Matchable
class Any
final case class StreamRefSubscriptionTimeoutException(msg: String) extends IllegalStateException

Attributes

Source
StreamRefs.scala
Supertypes
trait Product
trait Equals
class IllegalStateException
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all

Attributes

Companion
class
Source
ActorMaterializer.scala
Supertypes
class Object
trait Matchable
class Any
Self type

Leaked publishers and subscribers are cleaned up when they are not used within a given deadline, configured by StreamSubscriptionTimeoutSettings.

Leaked publishers and subscribers are cleaned up when they are not used within a given deadline, configured by StreamSubscriptionTimeoutSettings.

Attributes

Companion
object
Source
ActorMaterializer.scala
Supertypes
class Object
trait Matchable
class Any

This mode describes what shall happen when the subscription timeout expires for substream Publishers created by operations like prefixAndTail.

This mode describes what shall happen when the subscription timeout expires for substream Publishers created by operations like prefixAndTail.

Attributes

Companion
object
Source
ActorMaterializer.scala
Supertypes
class Object
trait Matchable
class Any
Known subtypes

Attributes

Companion
class
Source
ActorMaterializer.scala
Supertypes
trait Sum
trait Mirror
class Object
trait Matchable
class Any
Self type
class StreamTcpException(msg: String) extends RuntimeException, NoStackTrace

Attributes

Source
StreamTcpException.scala
Supertypes
trait NoStackTrace
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
Known subtypes
sealed class StreamTimeoutException(msg: String) extends TimeoutException, NoStackTrace

Base class for timeout exceptions specific to Pekko Streams

Base class for timeout exceptions specific to Pekko Streams

Not for user extension

Attributes

Source
StreamTimeoutException.scala
Supertypes
trait NoStackTrace
class TimeoutException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
Known subtypes

Extension of Subscription that allows to pass a cause when a subscription is cancelled.

Extension of Subscription that allows to pass a cause when a subscription is cancelled.

Subscribers can check for this trait and use its cancel(cause) method instead of the regular cancel method to pass a cancellation cause.

Not for user extension.

Attributes

Companion
object
Source
SubscriptionWithCancelException.scala
Supertypes
trait Subscription
class Object
trait Matchable
class Any

Attributes

Companion
trait
Source
SubscriptionWithCancelException.scala
Supertypes
class Object
trait Matchable
class Any
Self type
object Supervision

Attributes

Source
Supervision.scala
Supertypes
class Object
trait Matchable
class Any
Self type

The system materializer is a default materializer to use for most cases running streams, it is a single instance per actor system that is tied to the lifecycle of that system.

The system materializer is a default materializer to use for most cases running streams, it is a single instance per actor system that is tied to the lifecycle of that system.

Not intended to be manually used in user code.

Attributes

Companion
class
Source
SystemMaterializer.scala
Supertypes
class Object
trait Matchable
class Any
Self type
final class SystemMaterializer(system: ExtendedActorSystem) extends Extension

Attributes

Companion
object
Source
SystemMaterializer.scala
Supertypes
trait Extension
class Object
trait Matchable
class Any
sealed abstract class TLSClientAuth

An SSLEngine can either demand, allow or ignore its peer’s authentication (via certificates), where Need will fail the handshake if the peer does not provide valid credentials, Want allows the peer to send credentials and verifies them if provided, and None disables peer certificate verification.

An SSLEngine can either demand, allow or ignore its peer’s authentication (via certificates), where Need will fail the handshake if the peer does not provide valid credentials, Want allows the peer to send credentials and verifies them if provided, and None disables peer certificate verification.

See the documentation for SSLEngine::setWantClientAuth for more information.

Attributes

Companion
object
Source
SslTlsOptions.scala
Supertypes
class Object
trait Matchable
class Any
Known subtypes
object Need
object None
object Want
object TLSClientAuth

Attributes

Companion
class
Source
SslTlsOptions.scala
Supertypes
trait Sum
trait Mirror
class Object
trait Matchable
class Any
Self type
sealed abstract class TLSClosing

All streams in Pekko are unidirectional: while in a complex flow graph data may flow in multiple directions these individual flows are independent from each other. The difference between two half-duplex connections in opposite directions and a full-duplex connection is that the underlying transport is shared in the latter and tearing it down will end the data transfer in both directions.

All streams in Pekko are unidirectional: while in a complex flow graph data may flow in multiple directions these individual flows are independent from each other. The difference between two half-duplex connections in opposite directions and a full-duplex connection is that the underlying transport is shared in the latter and tearing it down will end the data transfer in both directions.

When integrating a full-duplex transport medium that does not support half-closing (which means ending one direction of data transfer without ending the other) into a stream topology, there can be unexpected effects. Feeding a finite Source into this medium will close the connection after all elements have been sent, which means that possible replies may not be received in full. To support this type of usage, the sending and receiving of data on the same side (e.g. on the Client) need to be coordinated such that it is known when all replies have been received. Only then should the transport be shut down.

To support these scenarios it is recommended that the full-duplex transport integration is configurable in terms of termination handling, which means that the user can optionally suppress the normal (closing) reaction to completion or cancellation events, as is expressed by the possible values of this type:

  • EagerClose means to not ignore signals
  • IgnoreCancel means to not react to cancellation of the receiving side unless the sending side has already completed
  • IgnoreComplete means to not react to the completion of the sending side unless the receiving side has already canceled
  • IgnoreBoth means to ignore the first termination signal—be that cancellation or completion—and only act upon the second one

Attributes

Companion
object
Source
SslTlsOptions.scala
Supertypes
class Object
trait Matchable
class Any
Known subtypes
class EagerClose
object EagerClose
class IgnoreBoth
object IgnoreBoth
class IgnoreCancel
object IgnoreCancel
Show all
object TLSClosing

Attributes

Companion
class
Source
SslTlsOptions.scala
Supertypes
trait Sum
trait Mirror
class Object
trait Matchable
class Any
Self type
TLSClosing.type
object TLSProtocol

Attributes

Source
SslTlsOptions.scala
Supertypes
class Object
trait Matchable
class Any
Self type
object TLSRole

Many protocols are asymmetric and distinguish between the client and the server, where the latter listens passively for messages and the former actively initiates the exchange.

Many protocols are asymmetric and distinguish between the client and the server, where the latter listens passively for messages and the former actively initiates the exchange.

Attributes

Companion
class
Source
SslTlsOptions.scala
Supertypes
trait Sum
trait Mirror
class Object
trait Matchable
class Any
Self type
TLSRole.type
sealed abstract class TLSRole

Attributes

Companion
object
Source
SslTlsOptions.scala
Supertypes
class Object
trait Matchable
class Any
Known subtypes
class Client
object Client
class Server
object Server
final case class TargetRefNotInitializedYetException() extends IllegalStateException

Attributes

Source
StreamRefs.scala
Supertypes
trait Product
trait Equals
class IllegalStateException
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
sealed abstract class ThrottleMode

Represents a mode that decides how to deal exceed rate for Throttle operator

Represents a mode that decides how to deal exceed rate for Throttle operator

Attributes

Companion
object
Source
ThrottleMode.scala
Supertypes
class Object
trait Matchable
class Any
Known subtypes
object Enforcing
object Shaping
object ThrottleMode

Attributes

Companion
class
Source
ThrottleMode.scala
Supertypes
trait Sum
trait Mirror
class Object
trait Matchable
class Any
Self type
final class TooManySubstreamsOpenException extends IllegalStateException, NoStackTrace

This exception signals that the maximum number of substreams declared has been exceeded. A finite limit is imposed so that memory usage is controlled.

This exception signals that the maximum number of substreams declared has been exceeded. A finite limit is imposed so that memory usage is controlled.

Attributes

Source
TooManySubstreamsOpenException.scala
Supertypes
trait NoStackTrace
class IllegalStateException
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all

Attributes

Companion
class
Source
UniformFanInShape.scala
Supertypes
class Object
trait Matchable
class Any
Self type
class UniformFanInShape[-T, +O](val n: Int, _init: Init[O]) extends FanInShape[O]

Attributes

Companion
object
Source
UniformFanInShape.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
Known subtypes

Attributes

Companion
class
Source
UniformFanOutShape.scala
Supertypes
class Object
trait Matchable
class Any
Self type
class UniformFanOutShape[-I, +O](n: Int, _init: Init[I]) extends FanOutShape[I]

Attributes

Companion
object
Source
UniformFanOutShape.scala
Supertypes
class FanOutShape[I]
class Shape
class Object
trait Matchable
class Any
final class UniqueKillSwitch extends KillSwitch

A UniqueKillSwitch is always a result of a materialization (unlike SharedKillSwitch which is constructed before any materialization) and it always controls that graph and operator which yielded the materialized value.

A UniqueKillSwitch is always a result of a materialization (unlike SharedKillSwitch which is constructed before any materialization) and it always controls that graph and operator which yielded the materialized value.

After calling UniqueKillSwitch#shutdown the running instance of the Graph of FlowShape that materialized to the UniqueKillSwitch will complete its downstream and cancel its upstream (unless if finished or failed already in which case the command is ignored). Subsequent invocations of completion commands will be ignored.

After calling UniqueKillSwitch#abort the running instance of the Graph of FlowShape that materialized to the UniqueKillSwitch will fail its downstream with the provided exception and cancel its upstream (unless if finished or failed already in which case the command is ignored). Subsequent invocations of completion commands will be ignored.

It is also possible to individually cancel, complete or fail upstream and downstream parts by calling the corresponding methods.

Attributes

Source
KillSwitch.scala
Supertypes
trait KillSwitch
class Object
trait Matchable
class Any
final class WatchedActorTerminatedException(val watchingStageName: String, val ref: ActorRef) extends RuntimeException

Used as failure exception by an ask operator if the target actor terminates. See Flow.ask and Flow.watch.

Used as failure exception by an ask operator if the target actor terminates. See Flow.ask and Flow.watch.

Attributes

Source
WatchedActorTerminatedException.scala
Supertypes
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all

Deprecated classlikes

final case class AbruptIOTerminationException(ioResult: IOResult, cause: Throwable) extends RuntimeException, NoStackTrace

This exception signals that a stream has been completed by an onError signal while there was still IO operations in progress.

This exception signals that a stream has been completed by an onError signal while there was still IO operations in progress.

Attributes

Deprecated
true
Source
IOResult.scala
Supertypes
trait Product
trait Equals
trait NoStackTrace
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all

An ActorMaterializer takes a stream blueprint and turns it into a running stream.

An ActorMaterializer takes a stream blueprint and turns it into a running stream.

Attributes

Companion
object
Deprecated
true
Source
ActorMaterializer.scala
Supertypes
class Materializer
class Object
trait Matchable
class Any

Attributes

Companion
class
Deprecated
true
Source
StreamTcpException.scala
Supertypes
trait Singleton
trait Product
trait Mirror
trait Product
trait Equals
trait NoStackTrace
class RuntimeException
class Exception
class Throwable
trait Serializable
class Object
trait Matchable
class Any
Show all
Self type
class FanInShape1N[-T0, -T1, +O](val n: Int, _init: Init[O]) extends FanInShape[O]

Attributes

Deprecated
true
Source
FanInShape1N.scala
Supertypes
class FanInShape[O]
class Shape
class Object
trait Matchable
class Any
sealed abstract class SubstreamCancelStrategy

Represents a strategy that decides how to deal with substream events.

Represents a strategy that decides how to deal with substream events.

Attributes

Companion
object
Deprecated
[Since version 1.1.0]
Source
SubstreamCancelStrategy.scala
Supertypes
class Object
trait Matchable
class Any

Attributes

Companion
class
Deprecated
[Since version 1.1.0]
Source
SubstreamCancelStrategy.scala
Supertypes
trait Sum
trait Mirror
class Object
trait Matchable
class Any
Self type