/*
 * Copyright (c) 2011-2018 Pivotal Software Inc, All Rights Reserved.
 *
 * Licensed under the Apache License, Version 2.0 (the "License");
 * you may not use this file except in compliance with the License.
 * You may obtain a copy of the License at
 *
 *       http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package reactor.core.publisher;

import java.time.Duration;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Objects;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.BiConsumer;
import java.util.function.BiFunction;
import java.util.function.BiPredicate;
import java.util.function.BooleanSupplier;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.function.LongConsumer;
import java.util.function.Predicate;
import java.util.function.Supplier;
import java.util.logging.Level;
import java.util.stream.Collector;
import java.util.stream.Stream;

import org.reactivestreams.Publisher;
import org.reactivestreams.Subscriber;
import org.reactivestreams.Subscription;
import reactor.core.CoreSubscriber;
import reactor.core.Disposable;
import reactor.core.Exceptions;
import reactor.core.Fuseable;
import reactor.core.Scannable;
import reactor.core.publisher.FluxSink.OverflowStrategy;
import reactor.core.scheduler.Scheduler;
import reactor.core.scheduler.Scheduler.Worker;
import reactor.core.scheduler.Schedulers;
import reactor.util.Logger;
import reactor.util.annotation.Nullable;
import reactor.util.concurrent.Queues;
import reactor.util.context.Context;
import reactor.util.function.Tuple2;
import reactor.util.function.Tuple3;
import reactor.util.function.Tuple4;
import reactor.util.function.Tuple5;
import reactor.util.function.Tuple6;
import reactor.util.function.Tuple7;
import reactor.util.function.Tuple8;
import reactor.util.function.Tuples;

A Reactive Streams Publisher with rx operators that emits 0 to N elements, and then completes (successfully or with an error).

The recommended way to learn about the Flux API and discover new operators is through the reference documentation, rather than through this javadoc (as opposed to learning more about individual operators). See the "which operator do I need?" appendix.

It is intended to be used in implementations and return types. Input parameters should keep using raw Publisher as much as possible.

If it is known that the underlying Publisher will emit 0 or 1 element, Mono should be used instead.

Note that using state in the java.util.function / lambdas used within Flux operators should be avoided, as these may be shared between several Subscribers.

subscribe(CoreSubscriber) is an internal extension to subscribe(Subscriber) used internally for Context passing. User provided Subscriber may be passed to this "subscribe" extension but will loose the available per-subscribe @link Hooks#onLastOperator}.

Author:Sebastien Deleuze, Stephane Maldini, David Karnok, Simon Baslé
Type parameters:
  • <T> – the element type of this Reactive Streams Publisher
See Also:
/** * A Reactive Streams {@link Publisher} with rx operators that emits 0 to N elements, and then completes * (successfully or with an error). * <p> * The recommended way to learn about the {@link Flux} API and discover new operators is * through the reference documentation, rather than through this javadoc (as opposed to * learning more about individual operators). See the <a href="http://projectreactor.io/docs/core/release/reference/docs/index.html#which-operator"> * "which operator do I need?" appendix</a>. * * <p> * <img width="640" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/flux.png" alt=""> * <p> * * <p>It is intended to be used in implementations and return types. Input parameters should keep using raw * {@link Publisher} as much as possible. * * <p>If it is known that the underlying {@link Publisher} will emit 0 or 1 element, {@link Mono} should be used * instead. * * <p>Note that using state in the {@code java.util.function} / lambdas used within Flux operators * should be avoided, as these may be shared between several {@link Subscriber Subscribers}. * * <p> {@link #subscribe(CoreSubscriber)} is an internal extension to * {@link #subscribe(Subscriber)} used internally for {@link Context} passing. User * provided {@link Subscriber} may * be passed to this "subscribe" extension but will loose the available * per-subscribe @link Hooks#onLastOperator}. * * @param <T> the element type of this Reactive Streams {@link Publisher} * * @author Sebastien Deleuze * @author Stephane Maldini * @author David Karnok * @author Simon Baslé * * @see Mono */
public abstract class Flux<T> implements Publisher<T> { // ============================================================================================================== // Static Generators // ==============================================================================================================
Build a Flux whose data are generated by the combination of the most recently published value from each of the Publisher sources.

Params:
  • sources – The Publisher sources to combine values from
  • combinator – The aggregate function that will receive the latest value from each upstream and return the value to signal downstream
Type parameters:
  • <T> – type of the value from sources
  • <V> – The produced output after transformation by the given combinator
@reactor.discardThis operator is NOT suited for types that need guaranteed discard of unpropagated elements, as it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and will be passed to the combinator multiple times.
Returns:a Flux based on the produced combinations
/** * Build a {@link Flux} whose data are generated by the combination of <strong>the * most recently published</strong> value from each of the {@link Publisher} sources. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/combinelatest.png" * alt=""> * * @reactor.discard This operator is NOT suited for types that need guaranteed discard of unpropagated elements, as * it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and * will be passed to the combinator multiple times. * * @param sources The {@link Publisher} sources to combine values from * @param combinator The aggregate function that will receive the latest value from each upstream and return the value * to signal downstream * @param <T> type of the value from sources * @param <V> The produced output after transformation by the given combinator * * @return a {@link Flux} based on the produced combinations */
@SafeVarargs public static <T, V> Flux<V> combineLatest(Function<Object[], V> combinator, Publisher<? extends T>... sources) { return combineLatest(combinator, Queues.XS_BUFFER_SIZE, sources); }
Build a Flux whose data are generated by the combination of the most recently published value from each of the Publisher sources.

Params:
  • sources – The Publisher sources to combine values from
  • prefetch – The demand sent to each combined source Publisher
  • combinator – The aggregate function that will receive the latest value from each upstream and return the value to signal downstream
Type parameters:
  • <T> – type of the value from sources
  • <V> – The produced output after transformation by the given combinator
@reactor.discardThis operator is NOT suited for types that need guaranteed discard of unpropagated elements, as it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and will be passed to the combinator multiple times.
Returns:a Flux based on the produced combinations
/** * Build a {@link Flux} whose data are generated by the combination of <strong>the * most recently published</strong> value from each of the {@link Publisher} sources. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/combinelatest.png" * alt=""> * * @reactor.discard This operator is NOT suited for types that need guaranteed discard of unpropagated elements, as * it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and * will be passed to the combinator multiple times. * * @param sources The {@link Publisher} sources to combine values from * @param prefetch The demand sent to each combined source {@link Publisher} * @param combinator The aggregate function that will receive the latest value from each upstream and return the value * to signal downstream * @param <T> type of the value from sources * @param <V> The produced output after transformation by the given combinator * * @return a {@link Flux} based on the produced combinations */
@SafeVarargs public static <T, V> Flux<V> combineLatest(Function<Object[], V> combinator, int prefetch, Publisher<? extends T>... sources) { if (sources.length == 0) { return empty(); } if (sources.length == 1) { Publisher<? extends T> source = sources[0]; if (source instanceof Fuseable) { return onAssembly(new FluxMapFuseable<>(from(source), v -> combinator.apply(new Object[]{v}))); } return onAssembly(new FluxMap<>(from(source), v -> combinator.apply(new Object[]{v}))); } return onAssembly(new FluxCombineLatest<>(sources, combinator, Queues.get(prefetch), prefetch)); }
Build a Flux whose data are generated by the combination of the most recently published value from each of two Publisher sources.

Params:
  • source1 – The first Publisher source to combine values from
  • source2 – The second Publisher source to combine values from
  • combinator – The aggregate function that will receive the latest value from each upstream and return the value to signal downstream
Type parameters:
  • <T1> – type of the value from source1
  • <T2> – type of the value from source2
  • <V> – The produced output after transformation by the given combinator
@reactor.discardThis operator is NOT suited for types that need guaranteed discard of unpropagated elements, as it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and will be passed to the combinator multiple times.
Returns:a Flux based on the produced combinations
/** * Build a {@link Flux} whose data are generated by the combination of <strong>the * most recently published</strong> value from each of two {@link Publisher} sources. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/combinelatest.png" * alt=""> * * @reactor.discard This operator is NOT suited for types that need guaranteed discard of unpropagated elements, as * it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and * will be passed to the combinator multiple times. * * @param source1 The first {@link Publisher} source to combine values from * @param source2 The second {@link Publisher} source to combine values from * @param combinator The aggregate function that will receive the latest value from each upstream and return the value * to signal downstream * @param <T1> type of the value from source1 * @param <T2> type of the value from source2 * @param <V> The produced output after transformation by the given combinator * * @return a {@link Flux} based on the produced combinations */
@SuppressWarnings("unchecked") public static <T1, T2, V> Flux<V> combineLatest(Publisher<? extends T1> source1, Publisher<? extends T2> source2, BiFunction<? super T1, ? super T2, ? extends V> combinator) { return combineLatest(tuple -> combinator.apply((T1)tuple[0], (T2)tuple[1]), source1, source2); }
Build a Flux whose data are generated by the combination of the most recently published value from each of three Publisher sources.

Params:
  • source1 – The first Publisher source to combine values from
  • source2 – The second Publisher source to combine values from
  • source3 – The third Publisher source to combine values from
  • combinator – The aggregate function that will receive the latest value from each upstream and return the value to signal downstream
Type parameters:
  • <T1> – type of the value from source1
  • <T2> – type of the value from source2
  • <T3> – type of the value from source3
  • <V> – The produced output after transformation by the given combinator
@reactor.discardThis operator is NOT suited for types that need guaranteed discard of unpropagated elements, as it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and will be passed to the combinator multiple times.
Returns:a Flux based on the produced combinations
/** * Build a {@link Flux} whose data are generated by the combination of <strong>the * most recently published</strong> value from each of three {@link Publisher} sources. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/combinelatest.png" * alt=""> * * @reactor.discard This operator is NOT suited for types that need guaranteed discard of unpropagated elements, as * it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and * will be passed to the combinator multiple times. * * @param source1 The first {@link Publisher} source to combine values from * @param source2 The second {@link Publisher} source to combine values from * @param source3 The third {@link Publisher} source to combine values from * @param combinator The aggregate function that will receive the latest value from each upstream and return the value * to signal downstream * @param <T1> type of the value from source1 * @param <T2> type of the value from source2 * @param <T3> type of the value from source3 * @param <V> The produced output after transformation by the given combinator * * @return a {@link Flux} based on the produced combinations */
public static <T1, T2, T3, V> Flux<V> combineLatest(Publisher<? extends T1> source1, Publisher<? extends T2> source2, Publisher<? extends T3> source3, Function<Object[], V> combinator) { return combineLatest(combinator, source1, source2, source3); }
Build a Flux whose data are generated by the combination of the most recently published value from each of four Publisher sources.

Params:
  • source1 – The first Publisher source to combine values from
  • source2 – The second Publisher source to combine values from
  • source3 – The third Publisher source to combine values from
  • source4 – The fourth Publisher source to combine values from
  • combinator – The aggregate function that will receive the latest value from each upstream and return the value to signal downstream
Type parameters:
  • <T1> – type of the value from source1
  • <T2> – type of the value from source2
  • <T3> – type of the value from source3
  • <T4> – type of the value from source4
  • <V> – The produced output after transformation by the given combinator
@reactor.discardThis operator is NOT suited for types that need guaranteed discard of unpropagated elements, as it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and will be passed to the combinator multiple times.
Returns:a Flux based on the produced combinations
/** * Build a {@link Flux} whose data are generated by the combination of <strong>the * most recently published</strong> value from each of four {@link Publisher} sources. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/combinelatest.png" * alt=""> * * @reactor.discard This operator is NOT suited for types that need guaranteed discard of unpropagated elements, as * it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and * will be passed to the combinator multiple times. * * @param source1 The first {@link Publisher} source to combine values from * @param source2 The second {@link Publisher} source to combine values from * @param source3 The third {@link Publisher} source to combine values from * @param source4 The fourth {@link Publisher} source to combine values from * @param combinator The aggregate function that will receive the latest value from each upstream and return the value * to signal downstream * @param <T1> type of the value from source1 * @param <T2> type of the value from source2 * @param <T3> type of the value from source3 * @param <T4> type of the value from source4 * @param <V> The produced output after transformation by the given combinator * * @return a {@link Flux} based on the produced combinations */
public static <T1, T2, T3, T4, V> Flux<V> combineLatest(Publisher<? extends T1> source1, Publisher<? extends T2> source2, Publisher<? extends T3> source3, Publisher<? extends T4> source4, Function<Object[], V> combinator) { return combineLatest(combinator, source1, source2, source3, source4); }
Build a Flux whose data are generated by the combination of the most recently published value from each of five Publisher sources.

Params:
  • source1 – The first Publisher source to combine values from
  • source2 – The second Publisher source to combine values from
  • source3 – The third Publisher source to combine values from
  • source4 – The fourth Publisher source to combine values from
  • source5 – The fifth Publisher source to combine values from
  • combinator – The aggregate function that will receive the latest value from each upstream and return the value to signal downstream
Type parameters:
  • <T1> – type of the value from source1
  • <T2> – type of the value from source2
  • <T3> – type of the value from source3
  • <T4> – type of the value from source4
  • <T5> – type of the value from source5
  • <V> – The produced output after transformation by the given combinator
@reactor.discardThis operator is NOT suited for types that need guaranteed discard of unpropagated elements, as it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and will be passed to the combinator multiple times.
Returns:a Flux based on the produced combinations
/** * Build a {@link Flux} whose data are generated by the combination of <strong>the * most recently published</strong> value from each of five {@link Publisher} sources. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/combinelatest.png" * alt=""> * * @reactor.discard This operator is NOT suited for types that need guaranteed discard of unpropagated elements, as * it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and * will be passed to the combinator multiple times. * * @param source1 The first {@link Publisher} source to combine values from * @param source2 The second {@link Publisher} source to combine values from * @param source3 The third {@link Publisher} source to combine values from * @param source4 The fourth {@link Publisher} source to combine values from * @param source5 The fifth {@link Publisher} source to combine values from * @param combinator The aggregate function that will receive the latest value from each upstream and return the value * to signal downstream * @param <T1> type of the value from source1 * @param <T2> type of the value from source2 * @param <T3> type of the value from source3 * @param <T4> type of the value from source4 * @param <T5> type of the value from source5 * @param <V> The produced output after transformation by the given combinator * * @return a {@link Flux} based on the produced combinations */
public static <T1, T2, T3, T4, T5, V> Flux<V> combineLatest(Publisher<? extends T1> source1, Publisher<? extends T2> source2, Publisher<? extends T3> source3, Publisher<? extends T4> source4, Publisher<? extends T5> source5, Function<Object[], V> combinator) { return combineLatest(combinator, source1, source2, source3, source4, source5); }
Build a Flux whose data are generated by the combination of the most recently published value from each of six Publisher sources.

Params:
  • source1 – The first Publisher source to combine values from
  • source2 – The second Publisher source to combine values from
  • source3 – The third Publisher source to combine values from
  • source4 – The fourth Publisher source to combine values from
  • source5 – The fifth Publisher source to combine values from
  • source6 – The sixth Publisher source to combine values from
  • combinator – The aggregate function that will receive the latest value from each upstream and return the value to signal downstream
Type parameters:
  • <T1> – type of the value from source1
  • <T2> – type of the value from source2
  • <T3> – type of the value from source3
  • <T4> – type of the value from source4
  • <T5> – type of the value from source5
  • <T6> – type of the value from source6
  • <V> – The produced output after transformation by the given combinator
@reactor.discardThis operator is NOT suited for types that need guaranteed discard of unpropagated elements, as it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and will be passed to the combinator multiple times.
Returns:a Flux based on the produced combinations
/** * Build a {@link Flux} whose data are generated by the combination of <strong>the * most recently published</strong> value from each of six {@link Publisher} sources. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/combinelatest.png" * alt=""> * * @reactor.discard This operator is NOT suited for types that need guaranteed discard of unpropagated elements, as * it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and * will be passed to the combinator multiple times. * * @param source1 The first {@link Publisher} source to combine values from * @param source2 The second {@link Publisher} source to combine values from * @param source3 The third {@link Publisher} source to combine values from * @param source4 The fourth {@link Publisher} source to combine values from * @param source5 The fifth {@link Publisher} source to combine values from * @param source6 The sixth {@link Publisher} source to combine values from * @param combinator The aggregate function that will receive the latest value from each upstream and return the value * to signal downstream * @param <T1> type of the value from source1 * @param <T2> type of the value from source2 * @param <T3> type of the value from source3 * @param <T4> type of the value from source4 * @param <T5> type of the value from source5 * @param <T6> type of the value from source6 * @param <V> The produced output after transformation by the given combinator * * @return a {@link Flux} based on the produced combinations */
public static <T1, T2, T3, T4, T5, T6, V> Flux<V> combineLatest(Publisher<? extends T1> source1, Publisher<? extends T2> source2, Publisher<? extends T3> source3, Publisher<? extends T4> source4, Publisher<? extends T5> source5, Publisher<? extends T6> source6, Function<Object[], V> combinator) { return combineLatest(combinator, source1, source2, source3, source4, source5, source6); }
Build a Flux whose data are generated by the combination of the most recently published value from each of the Publisher sources provided in an Iterable.

Params:
  • sources – The list of Publisher sources to combine values from
  • combinator – The aggregate function that will receive the latest value from each upstream and return the value to signal downstream
Type parameters:
  • <T> – The common base type of the values from sources
  • <V> – The produced output after transformation by the given combinator
@reactor.discardThis operator is NOT suited for types that need guaranteed discard of unpropagated elements, as it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and will be passed to the combinator multiple times.
Returns:a Flux based on the produced combinations
/** * Build a {@link Flux} whose data are generated by the combination of <strong>the * most recently published</strong> value from each * of the {@link Publisher} sources provided in an {@link Iterable}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/combinelatest.png" * alt=""> * * @reactor.discard This operator is NOT suited for types that need guaranteed discard of unpropagated elements, as * it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and * will be passed to the combinator multiple times. * * @param sources The list of {@link Publisher} sources to combine values from * @param combinator The aggregate function that will receive the latest value from each upstream and return the value * to signal downstream * @param <T> The common base type of the values from sources * @param <V> The produced output after transformation by the given combinator * * @return a {@link Flux} based on the produced combinations */
public static <T, V> Flux<V> combineLatest(Iterable<? extends Publisher<? extends T>> sources, Function<Object[], V> combinator) { return combineLatest(sources, Queues.XS_BUFFER_SIZE, combinator); }
Build a Flux whose data are generated by the combination of the most recently published value from each of the Publisher sources provided in an Iterable.

Params:
  • sources – The list of Publisher sources to combine values from
  • prefetch – demand produced to each combined source Publisher
  • combinator – The aggregate function that will receive the latest value from each upstream and return the value to signal downstream
Type parameters:
  • <T> – The common base type of the values from sources
  • <V> – The produced output after transformation by the given combinator
@reactor.discardThis operator is NOT suited for types that need guaranteed discard of unpropagated elements, as it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and will be passed to the combinator multiple times.
Returns:a Flux based on the produced combinations
/** * Build a {@link Flux} whose data are generated by the combination of <strong>the * most recently published</strong> value from each * of the {@link Publisher} sources provided in an {@link Iterable}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/combinelatest.png" * alt=""> * * @reactor.discard This operator is NOT suited for types that need guaranteed discard of unpropagated elements, as * it doesn't track which elements have been used by the combinator and which haven't. Furthermore, elements can and * will be passed to the combinator multiple times. * * @param sources The list of {@link Publisher} sources to combine values from * @param prefetch demand produced to each combined source {@link Publisher} * @param combinator The aggregate function that will receive the latest value from each upstream and return the value * to signal downstream * @param <T> The common base type of the values from sources * @param <V> The produced output after transformation by the given combinator * * @return a {@link Flux} based on the produced combinations */
public static <T, V> Flux<V> combineLatest(Iterable<? extends Publisher<? extends T>> sources, int prefetch, Function<Object[], V> combinator) { return onAssembly(new FluxCombineLatest<T, V>(sources, combinator, Queues.get(prefetch), prefetch)); }
Concatenate all sources provided in an Iterable, forwarding elements emitted by the sources downstream.

Concatenation is achieved by sequentially subscribing to the first source then waiting for it to complete before subscribing to the next, and so on until the last source completes. Any error interrupts the sequence immediately and is forwarded downstream.

Params:
Type parameters:
  • <T> – The type of values in both source and output sequences
Returns:a new Flux concatenating all source sequences
/** * Concatenate all sources provided in an {@link Iterable}, forwarding elements * emitted by the sources downstream. * <p> * Concatenation is achieved by sequentially subscribing to the first source then * waiting for it to complete before subscribing to the next, and so on until the * last source completes. Any error interrupts the sequence immediately and is * forwarded downstream. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concat.png" alt=""> * * @param sources The {@link Iterable} of {@link Publisher} to concatenate * @param <T> The type of values in both source and output sequences * * @return a new {@link Flux} concatenating all source sequences */
public static <T> Flux<T> concat(Iterable<? extends Publisher<? extends T>> sources) { return onAssembly(new FluxConcatIterable<>(sources)); }
Concatenates the values to the end of the Flux

Params:
  • values – The values to concatenate
Returns:a new Flux concatenating all source sequences
/** * Concatenates the values to the end of the {@link Flux} * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.2.0.M2/src/docs/marble/concatwithvalues.png" alt=""> * <p> * @param values The values to concatenate * * @return a new {@link Flux} concatenating all source sequences */
@SafeVarargs public final Flux<T> concatWithValues(T... values) { return concatWith(Flux.fromArray(values)); }
Concatenate all sources emitted as an onNext signal from a parent Publisher, forwarding elements emitted by the sources downstream.

Concatenation is achieved by sequentially subscribing to the first source then waiting for it to complete before subscribing to the next, and so on until the last source completes. Any error interrupts the sequence immediately and is forwarded downstream.

Params:
Type parameters:
  • <T> – The type of values in both source and output sequences
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation.
Returns:a new Flux concatenating all inner sources sequences
/** * Concatenate all sources emitted as an onNext signal from a parent {@link Publisher}, * forwarding elements emitted by the sources downstream. * <p> * Concatenation is achieved by sequentially subscribing to the first source then * waiting for it to complete before subscribing to the next, and so on until the * last source completes. Any error interrupts the sequence immediately and is * forwarded downstream. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concatinner.png" alt=""> * <p> * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation. * * @param sources The {@link Publisher} of {@link Publisher} to concatenate * @param <T> The type of values in both source and output sequences * * @return a new {@link Flux} concatenating all inner sources sequences */
public static <T> Flux<T> concat(Publisher<? extends Publisher<? extends T>> sources) { return concat(sources, Queues.XS_BUFFER_SIZE); }
Concatenate all sources emitted as an onNext signal from a parent Publisher, forwarding elements emitted by the sources downstream.

Concatenation is achieved by sequentially subscribing to the first source then waiting for it to complete before subscribing to the next, and so on until the last source completes. Any error interrupts the sequence immediately and is forwarded downstream.

Params:
  • sources – The Publisher of Publisher to concatenate
  • prefetch – the inner source request size
Type parameters:
  • <T> – The type of values in both source and output sequences
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation.
Returns:a new Flux concatenating all inner sources sequences
/** * Concatenate all sources emitted as an onNext signal from a parent {@link Publisher}, * forwarding elements emitted by the sources downstream. * <p> * Concatenation is achieved by sequentially subscribing to the first source then * waiting for it to complete before subscribing to the next, and so on until the * last source completes. Any error interrupts the sequence immediately and is * forwarded downstream. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concatinner.png" alt=""> * <p> * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation. * * @param sources The {@link Publisher} of {@link Publisher} to concatenate * @param prefetch the inner source request size * @param <T> The type of values in both source and output sequences * * @return a new {@link Flux} concatenating all inner sources sequences */
public static <T> Flux<T> concat(Publisher<? extends Publisher<? extends T>> sources, int prefetch) { return onAssembly(new FluxConcatMap<>(from(sources), identityFunction(), Queues.get(prefetch), prefetch, FluxConcatMap.ErrorMode.IMMEDIATE)); }
Concatenate all sources provided as a vararg, forwarding elements emitted by the sources downstream.

Concatenation is achieved by sequentially subscribing to the first source then waiting for it to complete before subscribing to the next, and so on until the last source completes. Any error interrupts the sequence immediately and is forwarded downstream.

Params:
Type parameters:
  • <T> – The type of values in both source and output sequences
Returns:a new Flux concatenating all source sequences
/** * Concatenate all sources provided as a vararg, forwarding elements emitted by the * sources downstream. * <p> * Concatenation is achieved by sequentially subscribing to the first source then * waiting for it to complete before subscribing to the next, and so on until the * last source completes. Any error interrupts the sequence immediately and is * forwarded downstream. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concat.png" alt=""> * <p> * @param sources The {@link Publisher} of {@link Publisher} to concat * @param <T> The type of values in both source and output sequences * * @return a new {@link Flux} concatenating all source sequences */
@SafeVarargs public static <T> Flux<T> concat(Publisher<? extends T>... sources) { return onAssembly(new FluxConcatArray<>(false, sources)); }
Concatenate all sources emitted as an onNext signal from a parent Publisher, forwarding elements emitted by the sources downstream.

Concatenation is achieved by sequentially subscribing to the first source then waiting for it to complete before subscribing to the next, and so on until the last source completes. Errors do not interrupt the main sequence but are propagated after the rest of the sources have had a chance to be concatenated.

Params:
Type parameters:
  • <T> – The type of values in both source and output sequences
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation.
Returns:a new Flux concatenating all inner sources sequences, delaying errors
/** * Concatenate all sources emitted as an onNext signal from a parent {@link Publisher}, * forwarding elements emitted by the sources downstream. * <p> * Concatenation is achieved by sequentially subscribing to the first source then * waiting for it to complete before subscribing to the next, and so on until the * last source completes. Errors do not interrupt the main sequence but are propagated * after the rest of the sources have had a chance to be concatenated. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concatinner.png" alt=""> * <p> * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation. * * @param sources The {@link Publisher} of {@link Publisher} to concatenate * @param <T> The type of values in both source and output sequences * * @return a new {@link Flux} concatenating all inner sources sequences, delaying errors */
public static <T> Flux<T> concatDelayError(Publisher<? extends Publisher<? extends T>> sources) { return concatDelayError(sources, Queues.XS_BUFFER_SIZE); }
Concatenate all sources emitted as an onNext signal from a parent Publisher, forwarding elements emitted by the sources downstream.

Concatenation is achieved by sequentially subscribing to the first source then waiting for it to complete before subscribing to the next, and so on until the last source completes. Errors do not interrupt the main sequence but are propagated after the rest of the sources have had a chance to be concatenated.

Params:
  • sources – The Publisher of Publisher to concatenate
  • prefetch – the inner source request size
Type parameters:
  • <T> – The type of values in both source and output sequences
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation.
Returns:a new Flux concatenating all inner sources sequences until complete or error
/** * Concatenate all sources emitted as an onNext signal from a parent {@link Publisher}, * forwarding elements emitted by the sources downstream. * <p> * Concatenation is achieved by sequentially subscribing to the first source then * waiting for it to complete before subscribing to the next, and so on until the * last source completes. Errors do not interrupt the main sequence but are propagated * after the rest of the sources have had a chance to be concatenated. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concatinner.png" alt=""> * <p> * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation. * * @param sources The {@link Publisher} of {@link Publisher} to concatenate * @param prefetch the inner source request size * @param <T> The type of values in both source and output sequences * * @return a new {@link Flux} concatenating all inner sources sequences until complete or error */
public static <T> Flux<T> concatDelayError(Publisher<? extends Publisher<? extends T>> sources, int prefetch) { return onAssembly(new FluxConcatMap<>(from(sources), identityFunction(), Queues.get(prefetch), prefetch, FluxConcatMap.ErrorMode.END)); }
Concatenate all sources emitted as an onNext signal from a parent Publisher, forwarding elements emitted by the sources downstream.

Concatenation is achieved by sequentially subscribing to the first source then waiting for it to complete before subscribing to the next, and so on until the last source completes.

Errors do not interrupt the main sequence but are propagated after the current concat backlog if delayUntilEnd is false or after all sources have had a chance to be concatenated if delayUntilEnd is true.

Params:
  • sources – The Publisher of Publisher to concatenate
  • delayUntilEnd – delay error until all sources have been consumed instead of after the current source
  • prefetch – the inner source request size
Type parameters:
  • <T> – The type of values in both source and output sequences
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation.
Returns:a new Flux concatenating all inner sources sequences until complete or error
/** * Concatenate all sources emitted as an onNext signal from a parent {@link Publisher}, * forwarding elements emitted by the sources downstream. * <p> * Concatenation is achieved by sequentially subscribing to the first source then * waiting for it to complete before subscribing to the next, and so on until the * last source completes. * <p> * Errors do not interrupt the main sequence but are propagated after the current * concat backlog if {@code delayUntilEnd} is {@literal false} or after all sources * have had a chance to be concatenated if {@code delayUntilEnd} is {@literal true}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concatinner.png" alt=""> * <p> * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation. * * @param sources The {@link Publisher} of {@link Publisher} to concatenate * @param delayUntilEnd delay error until all sources have been consumed instead of * after the current source * @param prefetch the inner source request size * @param <T> The type of values in both source and output sequences * * @return a new {@link Flux} concatenating all inner sources sequences until complete or error */
public static <T> Flux<T> concatDelayError(Publisher<? extends Publisher<? extends T>> sources, boolean delayUntilEnd, int prefetch) { return onAssembly(new FluxConcatMap<>(from(sources), identityFunction(), Queues.get(prefetch), prefetch, delayUntilEnd ? FluxConcatMap.ErrorMode.END : FluxConcatMap.ErrorMode.BOUNDARY)); }
Concatenate all sources provided as a vararg, forwarding elements emitted by the sources downstream.

Concatenation is achieved by sequentially subscribing to the first source then waiting for it to complete before subscribing to the next, and so on until the last source completes. Errors do not interrupt the main sequence but are propagated after the rest of the sources have had a chance to be concatenated.

Params:
Type parameters:
  • <T> – The type of values in both source and output sequences
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation.
Returns:a new Flux concatenating all source sequences
/** * Concatenate all sources provided as a vararg, forwarding elements emitted by the * sources downstream. * <p> * Concatenation is achieved by sequentially subscribing to the first source then * waiting for it to complete before subscribing to the next, and so on until the * last source completes. Errors do not interrupt the main sequence but are propagated * after the rest of the sources have had a chance to be concatenated. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concat.png" alt=""> * <p> * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation. * * @param sources The {@link Publisher} of {@link Publisher} to concat * @param <T> The type of values in both source and output sequences * * @return a new {@link Flux} concatenating all source sequences */
@SafeVarargs public static <T> Flux<T> concatDelayError(Publisher<? extends T>... sources) { return onAssembly(new FluxConcatArray<>(true, sources)); }
Programmatically create a Flux with the capability of emitting multiple elements in a synchronous or asynchronous manner through the FluxSink API. This includes emitting elements from multiple threads.

This Flux factory is useful if one wants to adapt some other multi-valued async API and not worry about cancellation and backpressure (which is handled by buffering all signals if the downstream can't keep up).

For example:


Flux.<String>create(emitter -> {
    ActionListener al = e -> {
        emitter.next(textField.getText());
    };
    // without cleanup support:
    button.addActionListener(al);
    // with cleanup support:
    button.addActionListener(al);
    emitter.onDispose(() -> {
        button.removeListener(al);
    });
});
Params:
  • emitter – Consume the FluxSink provided per-subscriber by Reactor to generate signals.
Type parameters:
  • <T> – The type of values in the sequence
See Also:
@reactor.discardThe FluxSink exposed by this operator buffers in case of overflow. The buffer is discarded when the main sequence is cancelled.
Returns:a Flux
/** * Programmatically create a {@link Flux} with the capability of emitting multiple * elements in a synchronous or asynchronous manner through the {@link FluxSink} API. * This includes emitting elements from multiple threads. * <p> * This Flux factory is useful if one wants to adapt some other multi-valued async API * and not worry about cancellation and backpressure (which is handled by buffering * all signals if the downstream can't keep up). * <p> * For example: * * <pre><code> * Flux.&lt;String&gt;create(emitter -&gt; { * * ActionListener al = e -&gt; { * emitter.next(textField.getText()); * }; * // without cleanup support: * * button.addActionListener(al); * * // with cleanup support: * * button.addActionListener(al); * emitter.onDispose(() -> { * button.removeListener(al); * }); * }); * </code></pre> * * @reactor.discard The {@link FluxSink} exposed by this operator buffers in case of * overflow. The buffer is discarded when the main sequence is cancelled. * * @param <T> The type of values in the sequence * @param emitter Consume the {@link FluxSink} provided per-subscriber by Reactor to generate signals. * @return a {@link Flux} * @see #push(Consumer) */
public static <T> Flux<T> create(Consumer<? super FluxSink<T>> emitter) { return create(emitter, OverflowStrategy.BUFFER); }
Programmatically create a Flux with the capability of emitting multiple elements in a synchronous or asynchronous manner through the FluxSink API. This includes emitting elements from multiple threads.

This Flux factory is useful if one wants to adapt some other multi-valued async API and not worry about cancellation and backpressure (which is handled by buffering all signals if the downstream can't keep up).

For example:


Flux.<String>create(emitter -> {
    ActionListener al = e -> {
        emitter.next(textField.getText());
    };
    // without cleanup support:
    button.addActionListener(al);
    // with cleanup support:
    button.addActionListener(al);
    emitter.onDispose(() -> {
        button.removeListener(al);
    });
}, FluxSink.OverflowStrategy.LATEST);
Params:
  • backpressure – the backpressure mode, see OverflowStrategy for the available backpressure modes
  • emitter – Consume the FluxSink provided per-subscriber by Reactor to generate signals.
Type parameters:
  • <T> – The type of values in the sequence
See Also:
@reactor.discardThe FluxSink exposed by this operator discards elements as relevant to the chosen OverflowStrategy. For example, the OverflowStrategy.DROP discards each items as they are being dropped, while OverflowStrategy.BUFFER will discard the buffer upon cancellation.
Returns:a Flux
/** * Programmatically create a {@link Flux} with the capability of emitting multiple * elements in a synchronous or asynchronous manner through the {@link FluxSink} API. * This includes emitting elements from multiple threads. * <p> * This Flux factory is useful if one wants to adapt some other multi-valued async API * and not worry about cancellation and backpressure (which is handled by buffering * all signals if the downstream can't keep up). * <p> * For example: * * <pre><code> * Flux.&lt;String&gt;create(emitter -&gt; { * * ActionListener al = e -&gt; { * emitter.next(textField.getText()); * }; * // without cleanup support: * * button.addActionListener(al); * * // with cleanup support: * * button.addActionListener(al); * emitter.onDispose(() -> { * button.removeListener(al); * }); * }, FluxSink.OverflowStrategy.LATEST); * </code></pre> * * @reactor.discard The {@link FluxSink} exposed by this operator discards elements * as relevant to the chosen {@link OverflowStrategy}. For example, the {@link OverflowStrategy#DROP} * discards each items as they are being dropped, while {@link OverflowStrategy#BUFFER} * will discard the buffer upon cancellation. * * @param <T> The type of values in the sequence * @param backpressure the backpressure mode, see {@link OverflowStrategy} for the * available backpressure modes * @param emitter Consume the {@link FluxSink} provided per-subscriber by Reactor to generate signals. * @return a {@link Flux} * @see #push(Consumer, reactor.core.publisher.FluxSink.OverflowStrategy) */
public static <T> Flux<T> create(Consumer<? super FluxSink<T>> emitter, OverflowStrategy backpressure) { return onAssembly(new FluxCreate<>(emitter, backpressure, FluxCreate.CreateMode.PUSH_PULL)); }
Programmatically create a Flux with the capability of emitting multiple elements from a single-threaded producer through the FluxSink API. For a multi-threaded capable alternative, see create(Consumer<? super FluxSink<Object>>).

This Flux factory is useful if one wants to adapt some other single-threaded multi-valued async API and not worry about cancellation and backpressure (which is handled by buffering all signals if the downstream can't keep up).

For example:


Flux.<String>push(emitter -> {
 ActionListener al = e -> {
	 emitter.next(textField.getText());
 };
 // without cleanup support:
 button.addActionListener(al);
 // with cleanup support:
 button.addActionListener(al);
 emitter.onDispose(() -> {
	 button.removeListener(al);
 });
}, FluxSink.OverflowStrategy.LATEST);
Params:
  • emitter – Consume the FluxSink provided per-subscriber by Reactor to generate signals.
Type parameters:
  • <T> – The type of values in the sequence
See Also:
@reactor.discardThe FluxSink exposed by this operator buffers in case of overflow. The buffer is discarded when the main sequence is cancelled.
Returns:a Flux
/** * Programmatically create a {@link Flux} with the capability of emitting multiple * elements from a single-threaded producer through the {@link FluxSink} API. For * a multi-threaded capable alternative, see {@link #create(Consumer)}. * <p> * This Flux factory is useful if one wants to adapt some other single-threaded * multi-valued async API and not worry about cancellation and backpressure (which is * handled by buffering all signals if the downstream can't keep up). * <p> * For example: * * <pre><code> * Flux.&lt;String&gt;push(emitter -&gt; { * * ActionListener al = e -&gt; { * emitter.next(textField.getText()); * }; * // without cleanup support: * * button.addActionListener(al); * * // with cleanup support: * * button.addActionListener(al); * emitter.onDispose(() -> { * button.removeListener(al); * }); * }, FluxSink.OverflowStrategy.LATEST); * </code></pre> * * @reactor.discard The {@link FluxSink} exposed by this operator buffers in case of * overflow. The buffer is discarded when the main sequence is cancelled. * * @param <T> The type of values in the sequence * @param emitter Consume the {@link FluxSink} provided per-subscriber by Reactor to generate signals. * @return a {@link Flux} * @see #create(Consumer) */
public static <T> Flux<T> push(Consumer<? super FluxSink<T>> emitter) { return onAssembly(new FluxCreate<>(emitter, OverflowStrategy.BUFFER, FluxCreate.CreateMode.PUSH_ONLY)); }
Programmatically create a Flux with the capability of emitting multiple elements from a single-threaded producer through the FluxSink API. For a multi-threaded capable alternative, see create(Consumer<? super FluxSink<Object>>, OverflowStrategy).

This Flux factory is useful if one wants to adapt some other single-threaded multi-valued async API and not worry about cancellation and backpressure (which is handled by buffering all signals if the downstream can't keep up).

For example:


Flux.<String>push(emitter -> {
 ActionListener al = e -> {
	 emitter.next(textField.getText());
 };
 // without cleanup support:
 button.addActionListener(al);
 // with cleanup support:
 button.addActionListener(al);
 emitter.onDispose(() -> {
	 button.removeListener(al);
 });
}, FluxSink.OverflowStrategy.LATEST);
Params:
  • backpressure – the backpressure mode, see OverflowStrategy for the available backpressure modes
  • emitter – Consume the FluxSink provided per-subscriber by Reactor to generate signals.
Type parameters:
  • <T> – The type of values in the sequence
See Also:
@reactor.discardThe FluxSink exposed by this operator discards elements as relevant to the chosen OverflowStrategy. For example, the OverflowStrategy.DROP discards each items as they are being dropped, while OverflowStrategy.BUFFER will discard the buffer upon cancellation.
Returns:a Flux
/** * Programmatically create a {@link Flux} with the capability of emitting multiple * elements from a single-threaded producer through the {@link FluxSink} API. For * a multi-threaded capable alternative, see {@link #create(Consumer, reactor.core.publisher.FluxSink.OverflowStrategy)}. * <p> * This Flux factory is useful if one wants to adapt some other single-threaded * multi-valued async API and not worry about cancellation and backpressure (which is * handled by buffering all signals if the downstream can't keep up). * <p> * For example: * * <pre><code> * Flux.&lt;String&gt;push(emitter -&gt; { * * ActionListener al = e -&gt; { * emitter.next(textField.getText()); * }; * // without cleanup support: * * button.addActionListener(al); * * // with cleanup support: * * button.addActionListener(al); * emitter.onDispose(() -> { * button.removeListener(al); * }); * }, FluxSink.OverflowStrategy.LATEST); * </code></pre> * * @reactor.discard The {@link FluxSink} exposed by this operator discards elements * as relevant to the chosen {@link OverflowStrategy}. For example, the {@link OverflowStrategy#DROP} * discards each items as they are being dropped, while {@link OverflowStrategy#BUFFER} * will discard the buffer upon cancellation. * * @param <T> The type of values in the sequence * @param backpressure the backpressure mode, see {@link OverflowStrategy} for the * available backpressure modes * @param emitter Consume the {@link FluxSink} provided per-subscriber by Reactor to generate signals. * @return a {@link Flux} * @see #create(Consumer, reactor.core.publisher.FluxSink.OverflowStrategy) */
public static <T> Flux<T> push(Consumer<? super FluxSink<T>> emitter, OverflowStrategy backpressure) { return onAssembly(new FluxCreate<>(emitter, backpressure, FluxCreate.CreateMode.PUSH_ONLY)); }
Lazily supply a Publisher every time a Subscription is made on the resulting Flux, so the actual source instantiation is deferred until each subscribe and the Supplier can create a subscriber-specific instance. If the supplier doesn't generate a new instance however, this operator will effectively behave like from(Publisher<? extends Object>).

Params:
Type parameters:
  • <T> – the type of values passing through the Flux
Returns:a deferred Flux
/** * Lazily supply a {@link Publisher} every time a {@link Subscription} is made on the * resulting {@link Flux}, so the actual source instantiation is deferred until each * subscribe and the {@link Supplier} can create a subscriber-specific instance. * If the supplier doesn't generate a new instance however, this operator will * effectively behave like {@link #from(Publisher)}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/defer.png" alt=""> * * @param supplier the {@link Publisher} {@link Supplier} to call on subscribe * @param <T> the type of values passing through the {@link Flux} * * @return a deferred {@link Flux} */
public static <T> Flux<T> defer(Supplier<? extends Publisher<T>> supplier) { return onAssembly(new FluxDefer<>(supplier)); }
Create a Flux that completes without emitting any item.

Type parameters:
Returns:an empty Flux
/** * Create a {@link Flux} that completes without emitting any item. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/empty.png" alt=""> * <p> * @param <T> the reified type of the target {@link Subscriber} * * @return an empty {@link Flux} */
public static <T> Flux<T> empty() { return FluxEmpty.instance(); }
Create a Flux that terminates with the specified error immediately after being subscribed to.

Params:
Type parameters:
Returns:a new failing Flux
/** * Create a {@link Flux} that terminates with the specified error immediately after * being subscribed to. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/error.png" alt=""> * <p> * @param error the error to signal to each {@link Subscriber} * @param <T> the reified type of the target {@link Subscriber} * * @return a new failing {@link Flux} */
public static <T> Flux<T> error(Throwable error) { return error(error, false); }
Create a Flux that terminates with an error immediately after being subscribed to. The Throwable is generated by a Supplier, invoked each time there is a subscription and allowing for lazy instantiation.

Params:
Type parameters:
Returns:a new failing Flux
/** * Create a {@link Flux} that terminates with an error immediately after being * subscribed to. The {@link Throwable} is generated by a {@link Supplier}, invoked * each time there is a subscription and allowing for lazy instantiation. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/error.png" alt=""> * <p> * @param errorSupplier the error signal {@link Supplier} to invoke for each {@link Subscriber} * @param <T> the reified type of the target {@link Subscriber} * * @return a new failing {@link Flux} */
public static <T> Flux<T> error(Supplier<Throwable> errorSupplier) { return onAssembly(new FluxErrorSupplied<>(errorSupplier)); }
Create a Flux that terminates with the specified error, either immediately after being subscribed to or after being first requested.

Params:
  • throwable – the error to signal to each Subscriber
  • whenRequested – if true, will onError on the first request instead of subscribe().
Type parameters:
Returns:a new failing Flux
/** * Create a {@link Flux} that terminates with the specified error, either immediately * after being subscribed to or after being first requested. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/errorrequest.png" alt=""> * * @param throwable the error to signal to each {@link Subscriber} * @param whenRequested if true, will onError on the first request instead of subscribe(). * @param <O> the reified type of the target {@link Subscriber} * * @return a new failing {@link Flux} */
public static <O> Flux<O> error(Throwable throwable, boolean whenRequested) { if (whenRequested) { return onAssembly(new FluxErrorOnRequest<>(throwable)); } else { return onAssembly(new FluxError<>(throwable)); } }
Pick the first Publisher to emit any signal (onNext/onError/onComplete) and replay all signals from that Publisher, effectively behaving like the fastest of these competing sources.

Params:
  • sources – The competing source publishers
Type parameters:
  • <I> – The type of values in both source and output sequences
Returns:a new Flux behaving like the fastest of its sources
/** * Pick the first {@link Publisher} to emit any signal (onNext/onError/onComplete) and * replay all signals from that {@link Publisher}, effectively behaving like the * fastest of these competing sources. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/firstemitting.png" alt=""> * <p> <p> * * @param sources The competing source publishers * @param <I> The type of values in both source and output sequences * * @return a new {@link Flux} behaving like the fastest of its sources */
@SafeVarargs public static <I> Flux<I> first(Publisher<? extends I>... sources) { return onAssembly(new FluxFirstEmitting<>(sources)); }
Pick the first Publisher to emit any signal (onNext/onError/onComplete) and replay all signals from that Publisher, effectively behaving like the fastest of these competing sources.

Params:
  • sources – The competing source publishers
Type parameters:
  • <I> – The type of values in both source and output sequences
Returns:a new Flux behaving like the fastest of its sources
/** * Pick the first {@link Publisher} to emit any signal (onNext/onError/onComplete) and * replay all signals from that {@link Publisher}, effectively behaving like the * fastest of these competing sources. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/firstemitting.png" alt=""> * <p> * * @param sources The competing source publishers * @param <I> The type of values in both source and output sequences * * @return a new {@link Flux} behaving like the fastest of its sources */
public static <I> Flux<I> first(Iterable<? extends Publisher<? extends I>> sources) { return onAssembly(new FluxFirstEmitting<>(sources)); }
Decorate the specified Publisher with the Flux API.

Params:
  • source – the source to decorate
Type parameters:
  • <T> – The type of values in both source and output sequences
Returns:a new Flux
/** * Decorate the specified {@link Publisher} with the {@link Flux} API. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/from.png" alt=""> * <p> * @param source the source to decorate * @param <T> The type of values in both source and output sequences * * @return a new {@link Flux} */
public static <T> Flux<T> from(Publisher<? extends T> source) { if (source instanceof Flux) { @SuppressWarnings("unchecked") Flux<T> casted = (Flux<T>) source; return casted; } if (source instanceof Fuseable.ScalarCallable) { try { @SuppressWarnings("unchecked") T t = ((Fuseable.ScalarCallable<T>) source).call(); if (t != null) { return just(t); } return empty(); } catch (Exception e) { return error(e); } } return wrap(source); }
Create a Flux that emits the items contained in the provided array.

Params:
  • array – the array to read data from
Type parameters:
  • <T> – The type of values in the source array and resulting Flux
Returns:a new Flux
/** * Create a {@link Flux} that emits the items contained in the provided array. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/fromarray.png" alt=""> * <p> * @param array the array to read data from * @param <T> The type of values in the source array and resulting Flux * * @return a new {@link Flux} */
public static <T> Flux<T> fromArray(T[] array) { if (array.length == 0) { return empty(); } if (array.length == 1) { return just(array[0]); } return onAssembly(new FluxArray<>(array)); }
Create a Flux that emits the items contained in the provided Iterable. A new iterator will be created for each subscriber.

Params:
Type parameters:
  • <T> – The type of values in the source Iterable and resulting Flux
Returns:a new Flux
/** * Create a {@link Flux} that emits the items contained in the provided {@link Iterable}. * A new iterator will be created for each subscriber. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/fromiterable.png" alt=""> * <p> * * @param it the {@link Iterable} to read data from * @param <T> The type of values in the source {@link Iterable} and resulting Flux * * @return a new {@link Flux} */
public static <T> Flux<T> fromIterable(Iterable<? extends T> it) { return onAssembly(new FluxIterable<>(it)); }
Create a Flux that emits the items contained in the provided Stream. Keep in mind that a Stream cannot be re-used, which can be problematic in case of multiple subscriptions or re-subscription (like with repeat() or retry()). The Stream is closed automatically by the operator on cancellation, error or completion.

Params:
  • s – the Stream to read data from
Type parameters:
  • <T> – The type of values in the source Stream and resulting Flux
Returns:a new Flux
/** * Create a {@link Flux} that emits the items contained in the provided {@link Stream}. * Keep in mind that a {@link Stream} cannot be re-used, which can be problematic in * case of multiple subscriptions or re-subscription (like with {@link #repeat()} or * {@link #retry()}). The {@link Stream} is {@link Stream#close() closed} automatically * by the operator on cancellation, error or completion. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/fromstream.png" alt=""> * <p> * @param s the {@link Stream} to read data from * @param <T> The type of values in the source {@link Stream} and resulting Flux * * @return a new {@link Flux} */
public static <T> Flux<T> fromStream(Stream<? extends T> s) { Objects.requireNonNull(s, "Stream s must be provided"); return onAssembly(new FluxStream<>(() -> s)); }
Create a Flux that emits the items contained in a Stream created by the provided Supplier for each subscription. The Stream is closed automatically by the operator on cancellation, error or completion.

Params:
  • streamSupplier – the Supplier that generates the Stream from which to read data
Type parameters:
  • <T> – The type of values in the source Stream and resulting Flux
Returns:a new Flux
/** * Create a {@link Flux} that emits the items contained in a {@link Stream} created by * the provided {@link Supplier} for each subscription. The {@link Stream} is * {@link Stream#close() closed} automatically by the operator on cancellation, error * or completion. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/fromstream.png" alt=""> * <p> * @param streamSupplier the {@link Supplier} that generates the {@link Stream} from * which to read data * @param <T> The type of values in the source {@link Stream} and resulting Flux * * @return a new {@link Flux} */
public static <T> Flux<T> fromStream(Supplier<Stream<? extends T>> streamSupplier) { return onAssembly(new FluxStream<>(streamSupplier)); }
Programmatically create a Flux by generating signals one-by-one via a consumer callback.

Params:
  • generator – Consume the SynchronousSink provided per-subscriber by Reactor to generate a single signal on each pass.
Type parameters:
  • <T> – the value type emitted
Returns:a Flux
/** * Programmatically create a {@link Flux} by generating signals one-by-one via a * consumer callback. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/generate.png" alt=""> * <p> * * @param <T> the value type emitted * @param generator Consume the {@link SynchronousSink} provided per-subscriber by Reactor * to generate a <strong>single</strong> signal on each pass. * * @return a {@link Flux} */
public static <T> Flux<T> generate(Consumer<SynchronousSink<T>> generator) { Objects.requireNonNull(generator, "generator"); return onAssembly(new FluxGenerate<>(generator)); }
Programmatically create a Flux by generating signals one-by-one via a consumer callback and some state. The stateSupplier may return null.

Params:
  • stateSupplier – called for each incoming Subscriber to provide the initial state for the generator bifunction
  • generator – Consume the SynchronousSink provided per-subscriber by Reactor as well as the current state to generate a single signal on each pass and return a (new) state.
Type parameters:
  • <T> – the value type emitted
  • <S> – the per-subscriber custom state type
Returns:a Flux
/** * Programmatically create a {@link Flux} by generating signals one-by-one via a * consumer callback and some state. The {@code stateSupplier} may return {@literal null}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/generate.png" alt=""> * <p> * * @param <T> the value type emitted * @param <S> the per-subscriber custom state type * @param stateSupplier called for each incoming Subscriber to provide the initial state for the generator bifunction * @param generator Consume the {@link SynchronousSink} provided per-subscriber by Reactor * as well as the current state to generate a <strong>single</strong> signal on each pass * and return a (new) state. * @return a {@link Flux} */
public static <T, S> Flux<T> generate(Callable<S> stateSupplier, BiFunction<S, SynchronousSink<T>, S> generator) { return onAssembly(new FluxGenerate<>(stateSupplier, generator)); }
Programmatically create a Flux by generating signals one-by-one via a consumer callback and some state, with a final cleanup callback. The stateSupplier may return null but your cleanup stateConsumer will need to handle the null case.

Params:
  • stateSupplier – called for each incoming Subscriber to provide the initial state for the generator bifunction
  • generator – Consume the SynchronousSink provided per-subscriber by Reactor as well as the current state to generate a single signal on each pass and return a (new) state.
  • stateConsumer – called after the generator has terminated or the downstream cancelled, receiving the last state to be handled (i.e., release resources or do other cleanup).
Type parameters:
  • <T> – the value type emitted
  • <S> – the per-subscriber custom state type
Returns:a Flux
/** * Programmatically create a {@link Flux} by generating signals one-by-one via a * consumer callback and some state, with a final cleanup callback. The * {@code stateSupplier} may return {@literal null} but your cleanup {@code stateConsumer} * will need to handle the null case. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/generate.png" alt=""> * <p> * * @param <T> the value type emitted * @param <S> the per-subscriber custom state type * @param stateSupplier called for each incoming Subscriber to provide the initial state for the generator bifunction * @param generator Consume the {@link SynchronousSink} provided per-subscriber by Reactor * as well as the current state to generate a <strong>single</strong> signal on each pass * and return a (new) state. * @param stateConsumer called after the generator has terminated or the downstream cancelled, receiving the last * state to be handled (i.e., release resources or do other cleanup). * * @return a {@link Flux} */
public static <T, S> Flux<T> generate(Callable<S> stateSupplier, BiFunction<S, SynchronousSink<T>, S> generator, Consumer<? super S> stateConsumer) { return onAssembly(new FluxGenerate<>(stateSupplier, generator, stateConsumer)); }
Create a Flux that emits long values starting with 0 and incrementing at specified time intervals on the global timer. If demand is not produced in time, an onError will be signalled with an overflow IllegalStateException detailing the tick that couldn't be emitted. In normal conditions, the Flux will never complete.

Runs on the Schedulers.parallel() Scheduler.

Params:
  • period – the period Duration between each increment
Returns:a new Flux emitting increasing numbers at regular intervals
/** * Create a {@link Flux} that emits long values starting with 0 and incrementing at * specified time intervals on the global timer. If demand is not produced in time, * an onError will be signalled with an {@link Exceptions#isOverflow(Throwable) overflow} * {@code IllegalStateException} detailing the tick that couldn't be emitted. * In normal conditions, the {@link Flux} will never complete. * <p> * Runs on the {@link Schedulers#parallel()} Scheduler. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/interval.png" alt=""> * <p> * @param period the period {@link Duration} between each increment * @return a new {@link Flux} emitting increasing numbers at regular intervals */
public static Flux<Long> interval(Duration period) { return interval(period, Schedulers.parallel()); }
Create a Flux that emits long values starting with 0 and incrementing at specified time intervals, after an initial delay, on the global timer. If demand is not produced in time, an onError will be signalled with an overflow IllegalStateException detailing the tick that couldn't be emitted. In normal conditions, the Flux will never complete.

Runs on the Schedulers.parallel() Scheduler.

Params:
  • delay – the Duration to wait before emitting 0l
  • period – the period Duration before each following increment
Returns:a new Flux emitting increasing numbers at regular intervals
/** * Create a {@link Flux} that emits long values starting with 0 and incrementing at * specified time intervals, after an initial delay, on the global timer. If demand is * not produced in time, an onError will be signalled with an * {@link Exceptions#isOverflow(Throwable) overflow} {@code IllegalStateException} * detailing the tick that couldn't be emitted. In normal conditions, the {@link Flux} * will never complete. * <p> * Runs on the {@link Schedulers#parallel()} Scheduler. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/intervald.png" alt=""> * * @param delay the {@link Duration} to wait before emitting 0l * @param period the period {@link Duration} before each following increment * * @return a new {@link Flux} emitting increasing numbers at regular intervals */
public static Flux<Long> interval(Duration delay, Duration period) { return interval(delay, period, Schedulers.parallel()); }
Create a Flux that emits long values starting with 0 and incrementing at specified time intervals, on the specified Scheduler. If demand is not produced in time, an onError will be signalled with an overflow IllegalStateException detailing the tick that couldn't be emitted. In normal conditions, the Flux will never complete.

Params:
  • period – the period Duration between each increment
  • timer – a time-capable Scheduler instance to run on
Returns:a new Flux emitting increasing numbers at regular intervals
/** * Create a {@link Flux} that emits long values starting with 0 and incrementing at * specified time intervals, on the specified {@link Scheduler}. If demand is not * produced in time, an onError will be signalled with an {@link Exceptions#isOverflow(Throwable) overflow} * {@code IllegalStateException} detailing the tick that couldn't be emitted. * In normal conditions, the {@link Flux} will never complete. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/interval.png" alt=""> * <p> * @param period the period {@link Duration} between each increment * @param timer a time-capable {@link Scheduler} instance to run on * * @return a new {@link Flux} emitting increasing numbers at regular intervals */
public static Flux<Long> interval(Duration period, Scheduler timer) { return onAssembly(new FluxInterval(period.toMillis(), period.toMillis(), TimeUnit.MILLISECONDS, timer)); }
Create a Flux that emits long values starting with 0 and incrementing at specified time intervals, after an initial delay, on the specified Scheduler. If demand is not produced in time, an onError will be signalled with an overflow IllegalStateException detailing the tick that couldn't be emitted. In normal conditions, the Flux will never complete.

Params:
  • delay – the Duration to wait before emitting 0l
  • period – the period Duration before each following increment
  • timer – a time-capable Scheduler instance to run on
Returns:a new Flux emitting increasing numbers at regular intervals
/** * Create a {@link Flux} that emits long values starting with 0 and incrementing at * specified time intervals, after an initial delay, on the specified {@link Scheduler}. * If demand is not produced in time, an onError will be signalled with an * {@link Exceptions#isOverflow(Throwable) overflow} {@code IllegalStateException} * detailing the tick that couldn't be emitted. In normal conditions, the {@link Flux} * will never complete. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/intervald.png" alt=""> * * @param delay the {@link Duration} to wait before emitting 0l * @param period the period {@link Duration} before each following increment * @param timer a time-capable {@link Scheduler} instance to run on * * @return a new {@link Flux} emitting increasing numbers at regular intervals */
public static Flux<Long> interval(Duration delay, Duration period, Scheduler timer) { return onAssembly(new FluxInterval(delay.toMillis(), period.toMillis(), TimeUnit.MILLISECONDS, timer)); }
Create a Flux that emits the provided elements and then completes.

Params:
  • data – the elements to emit, as a vararg
Type parameters:
  • <T> – the emitted data type
Returns:a new Flux
/** * Create a {@link Flux} that emits the provided elements and then completes. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/justn.png" alt=""> * <p> * @param data the elements to emit, as a vararg * @param <T> the emitted data type * * @return a new {@link Flux} */
@SafeVarargs public static <T> Flux<T> just(T... data) { return fromArray(data); }
Create a new Flux that will only emit a single element then onComplete.

Params:
  • data – the single element to emit
Type parameters:
  • <T> – the emitted data type
Returns:a new Flux
/** * Create a new {@link Flux} that will only emit a single element then onComplete. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/just.png" alt=""> * <p> * @param data the single element to emit * @param <T> the emitted data type * * @return a new {@link Flux} */
public static <T> Flux<T> just(T data) { return onAssembly(new FluxJust<>(data)); }
Merge data from Publisher sequences emitted by the passed Publisher into an interleaved merged sequence. Unlike concat, inner sources are subscribed to eagerly.

Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to another source.

Params:
Type parameters:
  • <T> – the merged type
Returns:a merged Flux
/** * Merge data from {@link Publisher} sequences emitted by the passed {@link Publisher} * into an interleaved merged sequence. Unlike {@link #concat(Publisher) concat}, inner * sources are subscribed to eagerly. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/mergeinner.png" alt=""> * <p> * Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with * an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source * in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to * another source. * * @param source a {@link Publisher} of {@link Publisher} sources to merge * @param <T> the merged type * * @return a merged {@link Flux} */
public static <T> Flux<T> merge(Publisher<? extends Publisher<? extends T>> source) { return merge(source, Queues.SMALL_BUFFER_SIZE, Queues.XS_BUFFER_SIZE); }
Merge data from Publisher sequences emitted by the passed Publisher into an interleaved merged sequence. Unlike concat, inner sources are subscribed to eagerly (but at most concurrency sources are subscribed to at the same time).

Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to another source.

Params:
  • source – a Publisher of Publisher sources to merge
  • concurrency – the request produced to the main source thus limiting concurrent merge backlog
Type parameters:
  • <T> – the merged type
Returns:a merged Flux
/** * Merge data from {@link Publisher} sequences emitted by the passed {@link Publisher} * into an interleaved merged sequence. Unlike {@link #concat(Publisher) concat}, inner * sources are subscribed to eagerly (but at most {@code concurrency} sources are * subscribed to at the same time). * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/mergeinner.png" alt=""> * <p> * Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with * an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source * in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to * another source. * * @param source a {@link Publisher} of {@link Publisher} sources to merge * @param concurrency the request produced to the main source thus limiting concurrent merge backlog * @param <T> the merged type * * @return a merged {@link Flux} */
public static <T> Flux<T> merge(Publisher<? extends Publisher<? extends T>> source, int concurrency) { return merge(source, concurrency, Queues.XS_BUFFER_SIZE); }
Merge data from Publisher sequences emitted by the passed Publisher into an interleaved merged sequence. Unlike concat, inner sources are subscribed to eagerly (but at most concurrency sources are subscribed to at the same time).

Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to another source.

Params:
  • source – a Publisher of Publisher sources to merge
  • concurrency – the request produced to the main source thus limiting concurrent merge backlog
  • prefetch – the inner source request size
Type parameters:
  • <T> – the merged type
Returns:a merged Flux
/** * Merge data from {@link Publisher} sequences emitted by the passed {@link Publisher} * into an interleaved merged sequence. Unlike {@link #concat(Publisher) concat}, inner * sources are subscribed to eagerly (but at most {@code concurrency} sources are * subscribed to at the same time). * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/mergeinner.png" alt=""> * <p> * Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with * an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source * in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to * another source. * * @param source a {@link Publisher} of {@link Publisher} sources to merge * @param concurrency the request produced to the main source thus limiting concurrent merge backlog * @param prefetch the inner source request size * @param <T> the merged type * * @return a merged {@link Flux} */
public static <T> Flux<T> merge(Publisher<? extends Publisher<? extends T>> source, int concurrency, int prefetch) { return onAssembly(new FluxFlatMap<>( from(source), identityFunction(), false, concurrency, Queues.get(concurrency), prefetch, Queues.get(prefetch))); }
Merge data from Publisher sequences contained in an Iterable into an interleaved merged sequence. Unlike concat, inner sources are subscribed to eagerly. A new Iterator will be created for each subscriber.

Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to another source.

Params:
  • sources – the Iterable of sources to merge (will be lazily iterated on subscribe)
Type parameters:
  • <I> – The source type of the data sequence
Returns:a merged Flux
/** * Merge data from {@link Publisher} sequences contained in an {@link Iterable} * into an interleaved merged sequence. Unlike {@link #concat(Publisher) concat}, inner * sources are subscribed to eagerly. * A new {@link Iterator} will be created for each subscriber. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/merge.png" alt=""> * <p> * Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with * an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source * in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to * another source. * * @param sources the {@link Iterable} of sources to merge (will be lazily iterated on subscribe) * @param <I> The source type of the data sequence * * @return a merged {@link Flux} */
public static <I> Flux<I> merge(Iterable<? extends Publisher<? extends I>> sources) { return merge(fromIterable(sources)); }
Merge data from Publisher sequences contained in an array / vararg into an interleaved merged sequence. Unlike concat, sources are subscribed to eagerly.

Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to another source.

Params:
  • sources – the array of Publisher sources to merge
Type parameters:
  • <I> – The source type of the data sequence
Returns:a merged Flux
/** * Merge data from {@link Publisher} sequences contained in an array / vararg * into an interleaved merged sequence. Unlike {@link #concat(Publisher) concat}, * sources are subscribed to eagerly. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/merge.png" alt=""> * <p> * Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with * an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source * in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to * another source. * * @param sources the array of {@link Publisher} sources to merge * @param <I> The source type of the data sequence * * @return a merged {@link Flux} */
@SafeVarargs public static <I> Flux<I> merge(Publisher<? extends I>... sources) { return merge(Queues.XS_BUFFER_SIZE, sources); }
Merge data from Publisher sequences contained in an array / vararg into an interleaved merged sequence. Unlike concat, sources are subscribed to eagerly.

Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to another source.

Params:
  • sources – the array of Publisher sources to merge
  • prefetch – the inner source request size
Type parameters:
  • <I> – The source type of the data sequence
Returns:a fresh Reactive Flux publisher ready to be subscribed
/** * Merge data from {@link Publisher} sequences contained in an array / vararg * into an interleaved merged sequence. Unlike {@link #concat(Publisher) concat}, * sources are subscribed to eagerly. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/merge.png" alt=""> * <p> * Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with * an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source * in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to * another source. * * @param sources the array of {@link Publisher} sources to merge * @param prefetch the inner source request size * @param <I> The source type of the data sequence * * @return a fresh Reactive {@link Flux} publisher ready to be subscribed */
@SafeVarargs public static <I> Flux<I> merge(int prefetch, Publisher<? extends I>... sources) { return merge(prefetch, false, sources); }
Merge data from Publisher sequences contained in an array / vararg into an interleaved merged sequence. Unlike concat, sources are subscribed to eagerly. This variant will delay any error until after the rest of the merge backlog has been processed.

Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to another source.

Params:
  • sources – the array of Publisher sources to merge
  • prefetch – the inner source request size
Type parameters:
  • <I> – The source type of the data sequence
Returns:a fresh Reactive Flux publisher ready to be subscribed
/** * Merge data from {@link Publisher} sequences contained in an array / vararg * into an interleaved merged sequence. Unlike {@link #concat(Publisher) concat}, * sources are subscribed to eagerly. * This variant will delay any error until after the rest of the merge backlog has been processed. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/merge.png" alt=""> * <p> * Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with * an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source * in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to * another source. * * @param sources the array of {@link Publisher} sources to merge * @param prefetch the inner source request size * @param <I> The source type of the data sequence * * @return a fresh Reactive {@link Flux} publisher ready to be subscribed */
@SafeVarargs public static <I> Flux<I> mergeDelayError(int prefetch, Publisher<? extends I>... sources) { return merge(prefetch, true, sources); }
Merge data from provided Publisher sequences into an ordered merged sequence, by picking the smallest values from each source (as defined by their natural order). This is not a sort(), as it doesn't consider the whole of each sequences.

Instead, this operator considers only one value from each source and picks the smallest of all these values, then replenishes the slot for that picked source.

Params:
Type parameters:
Returns:a merged Flux that , subscribing early but keeping the original ordering
/** * Merge data from provided {@link Publisher} sequences into an ordered merged sequence, * by picking the smallest values from each source (as defined by their natural order). * This is not a {@link #sort()}, as it doesn't consider the whole of each sequences. * <p> * Instead, this operator considers only one value from each source and picks the * smallest of all these values, then replenishes the slot for that picked source. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.2.0.M2/src/docs/marble/mergeordered.png" alt=""> * <p> * * @param sources {@link Publisher} sources of {@link Comparable} to merge * @param <I> a {@link Comparable} merged type that has a {@link Comparator#naturalOrder() natural order} * @return a merged {@link Flux} that , subscribing early but keeping the original ordering */
@SafeVarargs public static <I extends Comparable<? super I>> Flux<I> mergeOrdered(Publisher<? extends I>... sources) { return mergeOrdered(Queues.SMALL_BUFFER_SIZE, Comparator.naturalOrder(), sources); }
Merge data from provided Publisher sequences into an ordered merged sequence, by picking the smallest values from each source (as defined by the provided Comparator). This is not a sort(Comparator), as it doesn't consider the whole of each sequences.

Instead, this operator considers only one value from each source and picks the smallest of all these values, then replenishes the slot for that picked source.

Params:
  • comparator – the Comparator to use to find the smallest value
  • sources – Publisher sources to merge
Type parameters:
  • <T> – the merged type
Returns:a merged Flux that , subscribing early but keeping the original ordering
/** * Merge data from provided {@link Publisher} sequences into an ordered merged sequence, * by picking the smallest values from each source (as defined by the provided * {@link Comparator}). This is not a {@link #sort(Comparator)}, as it doesn't consider * the whole of each sequences. * <p> * Instead, this operator considers only one value from each source and picks the * smallest of all these values, then replenishes the slot for that picked source. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.2.0.M2/src/docs/marble/mergeordered.png" alt=""> * <p> * * @param comparator the {@link Comparator} to use to find the smallest value * @param sources {@link Publisher} sources to merge * @param <T> the merged type * @return a merged {@link Flux} that , subscribing early but keeping the original ordering */
@SafeVarargs public static <T> Flux<T> mergeOrdered(Comparator<? super T> comparator, Publisher<? extends T>... sources) { return mergeOrdered(Queues.SMALL_BUFFER_SIZE, comparator, sources); }
Merge data from provided Publisher sequences into an ordered merged sequence, by picking the smallest values from each source (as defined by the provided Comparator). This is not a sort(Comparator), as it doesn't consider the whole of each sequences.

Instead, this operator considers only one value from each source and picks the smallest of all these values, then replenishes the slot for that picked source.

Params:
  • prefetch – the number of elements to prefetch from each source (avoiding too many small requests to the source when picking)
  • comparator – the Comparator to use to find the smallest value
  • sources – Publisher sources to merge
Type parameters:
  • <T> – the merged type
Returns:a merged Flux that , subscribing early but keeping the original ordering
/** * Merge data from provided {@link Publisher} sequences into an ordered merged sequence, * by picking the smallest values from each source (as defined by the provided * {@link Comparator}). This is not a {@link #sort(Comparator)}, as it doesn't consider * the whole of each sequences. * <p> * Instead, this operator considers only one value from each source and picks the * smallest of all these values, then replenishes the slot for that picked source. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.2.0.M2/src/docs/marble/mergeordered.png" alt=""> * <p> * * @param prefetch the number of elements to prefetch from each source (avoiding too * many small requests to the source when picking) * @param comparator the {@link Comparator} to use to find the smallest value * @param sources {@link Publisher} sources to merge * @param <T> the merged type * @return a merged {@link Flux} that , subscribing early but keeping the original ordering */
@SafeVarargs public static <T> Flux<T> mergeOrdered(int prefetch, Comparator<? super T> comparator, Publisher<? extends T>... sources) { if (sources.length == 0) { return empty(); } if (sources.length == 1) { return from(sources[0]); } return onAssembly(new FluxMergeOrdered<>(prefetch, Queues.get(prefetch), comparator, sources)); }
Merge data from Publisher sequences emitted by the passed Publisher into an ordered merged sequence. Unlike concat, the inner publishers are subscribed to eagerly. Unlike merge, their emitted values are merged into the final sequence in subscription order.

Params:
Type parameters:
  • <T> – the merged type
Returns:a merged Flux, subscribing early but keeping the original ordering
/** * Merge data from {@link Publisher} sequences emitted by the passed {@link Publisher} * into an ordered merged sequence. Unlike concat, the inner publishers are subscribed to * eagerly. Unlike merge, their emitted values are merged into the final sequence in * subscription order. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/mergesequential.png" alt=""> * <p> * @param sources a {@link Publisher} of {@link Publisher} sources to merge * @param <T> the merged type * * @return a merged {@link Flux}, subscribing early but keeping the original ordering */
public static <T> Flux<T> mergeSequential(Publisher<? extends Publisher<? extends T>> sources) { return mergeSequential(sources, false, Queues.SMALL_BUFFER_SIZE, Queues.XS_BUFFER_SIZE); }
Merge data from Publisher sequences emitted by the passed Publisher into an ordered merged sequence. Unlike concat, the inner publishers are subscribed to eagerly (but at most maxConcurrency sources at a time). Unlike merge, their emitted values are merged into the final sequence in subscription order.

Params:
  • sources – a Publisher of Publisher sources to merge
  • prefetch – the inner source request size
  • maxConcurrency – the request produced to the main source thus limiting concurrent merge backlog
Type parameters:
  • <T> – the merged type
Returns:a merged Flux, subscribing early but keeping the original ordering
/** * Merge data from {@link Publisher} sequences emitted by the passed {@link Publisher} * into an ordered merged sequence. Unlike concat, the inner publishers are subscribed to * eagerly (but at most {@code maxConcurrency} sources at a time). Unlike merge, their * emitted values are merged into the final sequence in subscription order. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/mergesequential.png" alt=""> * <p> * @param sources a {@link Publisher} of {@link Publisher} sources to merge * @param prefetch the inner source request size * @param maxConcurrency the request produced to the main source thus limiting concurrent merge backlog * @param <T> the merged type * * @return a merged {@link Flux}, subscribing early but keeping the original ordering */
public static <T> Flux<T> mergeSequential(Publisher<? extends Publisher<? extends T>> sources, int maxConcurrency, int prefetch) { return mergeSequential(sources, false, maxConcurrency, prefetch); }
Merge data from Publisher sequences emitted by the passed Publisher into an ordered merged sequence. Unlike concat, the inner publishers are subscribed to eagerly (but at most maxConcurrency sources at a time). Unlike merge, their emitted values are merged into the final sequence in subscription order. This variant will delay any error until after the rest of the mergeSequential backlog has been processed.

Params:
  • sources – a Publisher of Publisher sources to merge
  • prefetch – the inner source request size
  • maxConcurrency – the request produced to the main source thus limiting concurrent merge backlog
Type parameters:
  • <T> – the merged type
Returns:a merged Flux, subscribing early but keeping the original ordering
/** * Merge data from {@link Publisher} sequences emitted by the passed {@link Publisher} * into an ordered merged sequence. Unlike concat, the inner publishers are subscribed to * eagerly (but at most {@code maxConcurrency} sources at a time). Unlike merge, their * emitted values are merged into the final sequence in subscription order. * This variant will delay any error until after the rest of the mergeSequential backlog has been processed. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/mergesequential.png" alt=""> * <p> * @param sources a {@link Publisher} of {@link Publisher} sources to merge * @param prefetch the inner source request size * @param maxConcurrency the request produced to the main source thus limiting concurrent merge backlog * @param <T> the merged type * * @return a merged {@link Flux}, subscribing early but keeping the original ordering */
public static <T> Flux<T> mergeSequentialDelayError(Publisher<? extends Publisher<? extends T>> sources, int maxConcurrency, int prefetch) { return mergeSequential(sources, true, maxConcurrency, prefetch); }
Merge data from Publisher sequences provided in an array/vararg into an ordered merged sequence. Unlike concat, sources are subscribed to eagerly. Unlike merge, their emitted values are merged into the final sequence in subscription order.

Params:
  • sources – a number of Publisher sequences to merge
Type parameters:
  • <I> – the merged type
Returns:a merged Flux, subscribing early but keeping the original ordering
/** * Merge data from {@link Publisher} sequences provided in an array/vararg * into an ordered merged sequence. Unlike concat, sources are subscribed to * eagerly. Unlike merge, their emitted values are merged into the final sequence in subscription order. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/mergesequential.png" alt=""> * <p> * @param sources a number of {@link Publisher} sequences to merge * @param <I> the merged type * * @return a merged {@link Flux}, subscribing early but keeping the original ordering */
@SafeVarargs public static <I> Flux<I> mergeSequential(Publisher<? extends I>... sources) { return mergeSequential(Queues.XS_BUFFER_SIZE, false, sources); }
Merge data from Publisher sequences provided in an array/vararg into an ordered merged sequence. Unlike concat, sources are subscribed to eagerly. Unlike merge, their emitted values are merged into the final sequence in subscription order.

Params:
  • prefetch – the inner source request size
  • sources – a number of Publisher sequences to merge
Type parameters:
  • <I> – the merged type
Returns:a merged Flux, subscribing early but keeping the original ordering
/** * Merge data from {@link Publisher} sequences provided in an array/vararg * into an ordered merged sequence. Unlike concat, sources are subscribed to * eagerly. Unlike merge, their emitted values are merged into the final sequence in subscription order. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/mergesequential.png" alt=""> * <p> * @param prefetch the inner source request size * @param sources a number of {@link Publisher} sequences to merge * @param <I> the merged type * * @return a merged {@link Flux}, subscribing early but keeping the original ordering */
@SafeVarargs public static <I> Flux<I> mergeSequential(int prefetch, Publisher<? extends I>... sources) { return mergeSequential(prefetch, false, sources); }
Merge data from Publisher sequences provided in an array/vararg into an ordered merged sequence. Unlike concat, sources are subscribed to eagerly. Unlike merge, their emitted values are merged into the final sequence in subscription order. This variant will delay any error until after the rest of the mergeSequential backlog has been processed.

Params:
  • prefetch – the inner source request size
  • sources – a number of Publisher sequences to merge
Type parameters:
  • <I> – the merged type
Returns:a merged Flux, subscribing early but keeping the original ordering
/** * Merge data from {@link Publisher} sequences provided in an array/vararg * into an ordered merged sequence. Unlike concat, sources are subscribed to * eagerly. Unlike merge, their emitted values are merged into the final sequence in subscription order. * This variant will delay any error until after the rest of the mergeSequential backlog * has been processed. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/mergesequential.png" alt=""> * <p> * @param prefetch the inner source request size * @param sources a number of {@link Publisher} sequences to merge * @param <I> the merged type * * @return a merged {@link Flux}, subscribing early but keeping the original ordering */
@SafeVarargs public static <I> Flux<I> mergeSequentialDelayError(int prefetch, Publisher<? extends I>... sources) { return mergeSequential(prefetch, true, sources); }
Merge data from Publisher sequences provided in an Iterable into an ordered merged sequence. Unlike concat, sources are subscribed to eagerly. Unlike merge, their emitted values are merged into the final sequence in subscription order.

Params:
Type parameters:
  • <I> – the merged type
Returns:a merged Flux, subscribing early but keeping the original ordering
/** * Merge data from {@link Publisher} sequences provided in an {@link Iterable} * into an ordered merged sequence. Unlike concat, sources are subscribed to * eagerly. Unlike merge, their emitted values are merged into the final sequence in subscription order. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/mergesequential.png" alt=""> * <p> * @param sources an {@link Iterable} of {@link Publisher} sequences to merge * @param <I> the merged type * * @return a merged {@link Flux}, subscribing early but keeping the original ordering */
public static <I> Flux<I> mergeSequential(Iterable<? extends Publisher<? extends I>> sources) { return mergeSequential(sources, false, Queues.SMALL_BUFFER_SIZE, Queues.XS_BUFFER_SIZE); }
Merge data from Publisher sequences provided in an Iterable into an ordered merged sequence. Unlike concat, sources are subscribed to eagerly (but at most maxConcurrency sources at a time). Unlike merge, their emitted values are merged into the final sequence in subscription order.

Params:
  • sources – an Iterable of Publisher sequences to merge
  • maxConcurrency – the request produced to the main source thus limiting concurrent merge backlog
  • prefetch – the inner source request size
Type parameters:
  • <I> – the merged type
Returns:a merged Flux, subscribing early but keeping the original ordering
/** * Merge data from {@link Publisher} sequences provided in an {@link Iterable} * into an ordered merged sequence. Unlike concat, sources are subscribed to * eagerly (but at most {@code maxConcurrency} sources at a time). Unlike merge, their * emitted values are merged into the final sequence in subscription order. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/mergesequential.png" alt=""> * <p> * @param sources an {@link Iterable} of {@link Publisher} sequences to merge * @param maxConcurrency the request produced to the main source thus limiting concurrent merge backlog * @param prefetch the inner source request size * @param <I> the merged type * * @return a merged {@link Flux}, subscribing early but keeping the original ordering */
public static <I> Flux<I> mergeSequential(Iterable<? extends Publisher<? extends I>> sources, int maxConcurrency, int prefetch) { return mergeSequential(sources, false, maxConcurrency, prefetch); }
Merge data from Publisher sequences provided in an Iterable into an ordered merged sequence. Unlike concat, sources are subscribed to eagerly (but at most maxConcurrency sources at a time). Unlike merge, their emitted values are merged into the final sequence in subscription order. This variant will delay any error until after the rest of the mergeSequential backlog has been processed.

Params:
  • sources – an Iterable of Publisher sequences to merge
  • maxConcurrency – the request produced to the main source thus limiting concurrent merge backlog
  • prefetch – the inner source request size
Type parameters:
  • <I> – the merged type
Returns:a merged Flux, subscribing early but keeping the original ordering
/** * Merge data from {@link Publisher} sequences provided in an {@link Iterable} * into an ordered merged sequence. Unlike concat, sources are subscribed to * eagerly (but at most {@code maxConcurrency} sources at a time). Unlike merge, their * emitted values are merged into the final sequence in subscription order. * This variant will delay any error until after the rest of the mergeSequential backlog * has been processed. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/mergesequential.png" alt=""> * <p> * @param sources an {@link Iterable} of {@link Publisher} sequences to merge * @param maxConcurrency the request produced to the main source thus limiting concurrent merge backlog * @param prefetch the inner source request size * @param <I> the merged type * * @return a merged {@link Flux}, subscribing early but keeping the original ordering */
public static <I> Flux<I> mergeSequentialDelayError(Iterable<? extends Publisher<? extends I>> sources, int maxConcurrency, int prefetch) { return mergeSequential(sources, true, maxConcurrency, prefetch); }
Create a Flux that will never signal any data, error or completion signal.

Type parameters:
Returns:a never completing Flux
/** * Create a {@link Flux} that will never signal any data, error or completion signal. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/never.png" alt=""> * <p> * @param <T> the {@link Subscriber} type target * * @return a never completing {@link Flux} */
public static <T> Flux<T> never() { return FluxNever.instance(); }
Build a Flux that will only emit a sequence of count incrementing integers, starting from start. That is, emit integers between start (included) and start + count (excluded) then complete.

Params:
  • start – the first integer to be emit
  • count – the total number of incrementing values to emit, including the first value
Returns:a ranged Flux
/** * Build a {@link Flux} that will only emit a sequence of {@code count} incrementing integers, * starting from {@code start}. That is, emit integers between {@code start} (included) * and {@code start + count} (excluded) then complete. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/range.png" alt=""> * * @param start the first integer to be emit * @param count the total number of incrementing values to emit, including the first value * @return a ranged {@link Flux} */
public static Flux<Integer> range(int start, int count) { if (count == 1) { return just(start); } if (count == 0) { return empty(); } return onAssembly(new FluxRange(start, count)); }
Creates a Flux that mirrors the most recently emitted Publisher, forwarding its data until a new Publisher comes in in the source.

The resulting Flux will complete once there are no new Publisher in the source (source has completed) and the last mirrored Publisher has also completed.

Params:
Type parameters:
  • <T> – the produced type
Returns:a FluxProcessor accepting publishers and producing T
/** * Creates a {@link Flux} that mirrors the most recently emitted {@link Publisher}, * forwarding its data until a new {@link Publisher} comes in in the source. * <p> * The resulting {@link Flux} will complete once there are no new {@link Publisher} in * the source (source has completed) and the last mirrored {@link Publisher} has also * completed. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/switchonnext.png" * alt=""> * * @param mergedPublishers The {@link Publisher} of {@link Publisher} to switch on and mirror. * @param <T> the produced type * * @return a {@link FluxProcessor} accepting publishers and producing T */
public static <T> Flux<T> switchOnNext(Publisher<? extends Publisher<? extends T>> mergedPublishers) { return switchOnNext(mergedPublishers, Queues.XS_BUFFER_SIZE); }
Creates a Flux that mirrors the most recently emitted Publisher, forwarding its data until a new Publisher comes in in the source.

The resulting Flux will complete once there are no new Publisher in the source (source has completed) and the last mirrored Publisher has also completed.

Params:
  • mergedPublishers – The Publisher of Publisher to switch on and mirror.
  • prefetch – the inner source request size
Type parameters:
  • <T> – the produced type
Returns:a FluxProcessor accepting publishers and producing T
/** * Creates a {@link Flux} that mirrors the most recently emitted {@link Publisher}, * forwarding its data until a new {@link Publisher} comes in in the source. * <p> * The resulting {@link Flux} will complete once there are no new {@link Publisher} in * the source (source has completed) and the last mirrored {@link Publisher} has also * completed. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/switchonnext.png" * alt=""> * * @param mergedPublishers The {@link Publisher} of {@link Publisher} to switch on and mirror. * @param prefetch the inner source request size * @param <T> the produced type * * @return a {@link FluxProcessor} accepting publishers and producing T */
public static <T> Flux<T> switchOnNext(Publisher<? extends Publisher<? extends T>> mergedPublishers, int prefetch) { return onAssembly(new FluxSwitchMap<>(from(mergedPublishers), identityFunction(), Queues.unbounded(prefetch), prefetch)); }
Uses a resource, generated by a supplier for each individual Subscriber, while streaming the values from a Publisher derived from the same resource and makes sure the resource is released if the sequence terminates or the Subscriber cancels.

Eager resource cleanup happens just before the source termination and exceptions raised by the cleanup Consumer may override the terminal even.

For an asynchronous version of the cleanup, with distinct path for onComplete, onError and cancel terminations, see usingWhen(Publisher<Object>, Function<? super Object,? extends Publisher<? extends Object>>, Function<? super Object,? extends Publisher<?>>, Function<? super Object,? extends Publisher<?>>, Function<? super Object,? extends Publisher<?>>).

Params:
  • resourceSupplier – a Callable that is called on subscribe to generate the resource
  • sourceSupplier – a factory to derive a Publisher from the supplied resource
  • resourceCleanup – a resource cleanup callback invoked on completion
Type parameters:
  • <T> – emitted type
  • <D> – resource type
See Also:
Returns:a new Flux built around a disposable resource
/** * Uses a resource, generated by a supplier for each individual Subscriber, while streaming the values from a * Publisher derived from the same resource and makes sure the resource is released if the sequence terminates or * the Subscriber cancels. * <p> * Eager resource cleanup happens just before the source termination and exceptions raised by the cleanup Consumer * may override the terminal even. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/using.png" * alt=""> * <p> * For an asynchronous version of the cleanup, with distinct path for onComplete, onError * and cancel terminations, see {@link #usingWhen(Publisher, Function, Function, Function, Function)}. * * @param resourceSupplier a {@link Callable} that is called on subscribe to generate the resource * @param sourceSupplier a factory to derive a {@link Publisher} from the supplied resource * @param resourceCleanup a resource cleanup callback invoked on completion * @param <T> emitted type * @param <D> resource type * * @return a new {@link Flux} built around a disposable resource * @see #usingWhen(Publisher, Function, Function, Function, Function) * @see #usingWhen(Publisher, Function, Function) */
public static <T, D> Flux<T> using(Callable<? extends D> resourceSupplier, Function<? super D, ? extends Publisher<? extends T>> sourceSupplier, Consumer<? super D> resourceCleanup) { return using(resourceSupplier, sourceSupplier, resourceCleanup, true); }
Uses a resource, generated by a supplier for each individual Subscriber, while streaming the values from a Publisher derived from the same resource and makes sure the resource is released if the sequence terminates or the Subscriber cancels.

  • Eager resource cleanup happens just before the source termination and exceptions raised by the cleanup Consumer may override the terminal even.
  • Non-eager cleanup will drop any exception.

For an asynchronous version of the cleanup, with distinct path for onComplete, onError and cancel terminations, see usingWhen(Publisher<Object>, Function<? super Object,? extends Publisher<? extends Object>>, Function<? super Object,? extends Publisher<?>>, Function<? super Object,? extends Publisher<?>>, Function<? super Object,? extends Publisher<?>>).

Params:
  • resourceSupplier – a Callable that is called on subscribe to generate the resource
  • sourceSupplier – a factory to derive a Publisher from the supplied resource
  • resourceCleanup – a resource cleanup callback invoked on completion
  • eager – true to clean before terminating downstream subscribers
Type parameters:
  • <T> – emitted type
  • <D> – resource type
See Also:
Returns:a new Flux built around a disposable resource
/** * Uses a resource, generated by a supplier for each individual Subscriber, while streaming the values from a * Publisher derived from the same resource and makes sure the resource is released if the sequence terminates or * the Subscriber cancels. * <p> * <ul> <li>Eager resource cleanup happens just before the source termination and exceptions raised by the cleanup * Consumer may override the terminal even.</li> <li>Non-eager cleanup will drop any exception.</li> </ul> * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/using.png" * alt=""> * <p> * For an asynchronous version of the cleanup, with distinct path for onComplete, onError * and cancel terminations, see {@link #usingWhen(Publisher, Function, Function, Function, Function)}. * * @param resourceSupplier a {@link Callable} that is called on subscribe to generate the resource * @param sourceSupplier a factory to derive a {@link Publisher} from the supplied resource * @param resourceCleanup a resource cleanup callback invoked on completion * @param eager true to clean before terminating downstream subscribers * @param <T> emitted type * @param <D> resource type * * @return a new {@link Flux} built around a disposable resource * @see #usingWhen(Publisher, Function, Function, Function, Function) * @see #usingWhen(Publisher, Function, Function) */
public static <T, D> Flux<T> using(Callable<? extends D> resourceSupplier, Function<? super D, ? extends Publisher<? extends T>> sourceSupplier, Consumer<? super D> resourceCleanup, boolean eager) { return onAssembly(new FluxUsing<>(resourceSupplier, sourceSupplier, resourceCleanup, eager)); }
Uses a resource, generated by a Publisher for each individual Subscriber, while streaming the values from a Publisher derived from the same resource. Whenever the resulting sequence terminates, the relevant Function generates a "cleanup" Publisher that is invoked but doesn't change the content of the main sequence. Instead it just defers the termination (unless it errors, in which case the error suppresses the original termination signal).

Note that if the resource supplying Publisher emits more than one resource, the subsequent resources are dropped (Operators.onNextDropped(Object, Context)). If the publisher errors AFTER having emitted one resource, the error is also silently dropped (Operators.onErrorDropped(Throwable, Context)). An empty completion or error without at least one onNext signal triggers a short-circuit of the main sequence with the same terminal signal (no resource is established, no cleanup is invoked).

Params:
  • resourceSupplier – a Publisher that "generates" the resource, subscribed for each subscription to the main sequence
  • resourceClosure – a factory to derive a Publisher from the supplied resource
  • asyncComplete – an asynchronous resource cleanup invoked if the resource closure terminates with onComplete or is cancelled
  • asyncError – an asynchronous resource cleanup invoked if the resource closure terminates with onError
Type parameters:
  • <T> – the type of elements emitted by the resource closure, and thus the main sequence
  • <D> – the type of the resource object
See Also:
Returns:a new Flux built around a "transactional" resource, with several termination path triggering asynchronous cleanup sequences
/** * Uses a resource, generated by a {@link Publisher} for each individual {@link Subscriber}, * while streaming the values from a {@link Publisher} derived from the same resource. * Whenever the resulting sequence terminates, the relevant {@link Function} generates * a "cleanup" {@link Publisher} that is invoked but doesn't change the content of the * main sequence. Instead it just defers the termination (unless it errors, in which case * the error suppresses the original termination signal). * <p> * Note that if the resource supplying {@link Publisher} emits more than one resource, the * subsequent resources are dropped ({@link Operators#onNextDropped(Object, Context)}). If * the publisher errors AFTER having emitted one resource, the error is also silently dropped * ({@link Operators#onErrorDropped(Throwable, Context)}). * An empty completion or error without at least one onNext signal triggers a short-circuit * of the main sequence with the same terminal signal (no resource is established, no * cleanup is invoked). * * @param resourceSupplier a {@link Publisher} that "generates" the resource, * subscribed for each subscription to the main sequence * @param resourceClosure a factory to derive a {@link Publisher} from the supplied resource * @param asyncComplete an asynchronous resource cleanup invoked if the resource closure terminates with onComplete or is cancelled * @param asyncError an asynchronous resource cleanup invoked if the resource closure terminates with onError * @param <T> the type of elements emitted by the resource closure, and thus the main sequence * @param <D> the type of the resource object * @return a new {@link Flux} built around a "transactional" resource, with several * termination path triggering asynchronous cleanup sequences * @see #usingWhen(Publisher, Function, Function, Function, Function) */
public static <T, D> Flux<T> usingWhen(Publisher<D> resourceSupplier, Function<? super D, ? extends Publisher<? extends T>> resourceClosure, Function<? super D, ? extends Publisher<?>> asyncComplete, Function<? super D, ? extends Publisher<?>> asyncError) { //null asyncCancel translates to using the `asyncComplete` function in the operator return onAssembly(new FluxUsingWhen<>(resourceSupplier, resourceClosure, asyncComplete, asyncError, null)); }
Uses a resource, generated by a Publisher for each individual Subscriber, while streaming the values from a Publisher derived from the same resource. Whenever the resulting sequence terminates, the relevant Function generates a "cleanup" Publisher that is invoked but doesn't change the content of the main sequence. Instead it just defers the termination (unless it errors, in which case the error suppresses the original termination signal).

Note that if the resource supplying Publisher emits more than one resource, the subsequent resources are dropped (Operators.onNextDropped(Object, Context)). If the publisher errors AFTER having emitted one resource, the error is also silently dropped (Operators.onErrorDropped(Throwable, Context)). An empty completion or error without at least one onNext signal triggers a short-circuit of the main sequence with the same terminal signal (no resource is established, no cleanup is invoked).

Params:
  • resourceSupplier – a Publisher that "generates" the resource, subscribed for each subscription to the main sequence
  • resourceClosure – a factory to derive a Publisher from the supplied resource
  • asyncComplete – an asynchronous resource cleanup invoked if the resource closure terminates with onComplete
  • asyncError – an asynchronous resource cleanup invoked if the resource closure terminates with onError
  • asyncCancel – an asynchronous resource cleanup invoked if the resource closure is cancelled. When null, the asyncComplete path is used instead.
Type parameters:
  • <T> – the type of elements emitted by the resource closure, and thus the main sequence
  • <D> – the type of the resource object
See Also:
Returns:a new Flux built around a "transactional" resource, with several termination path triggering asynchronous cleanup sequences
/** * Uses a resource, generated by a {@link Publisher} for each individual {@link Subscriber}, * while streaming the values from a {@link Publisher} derived from the same resource. * Whenever the resulting sequence terminates, the relevant {@link Function} generates * a "cleanup" {@link Publisher} that is invoked but doesn't change the content of the * main sequence. Instead it just defers the termination (unless it errors, in which case * the error suppresses the original termination signal). * <p> * Note that if the resource supplying {@link Publisher} emits more than one resource, the * subsequent resources are dropped ({@link Operators#onNextDropped(Object, Context)}). If * the publisher errors AFTER having emitted one resource, the error is also silently dropped * ({@link Operators#onErrorDropped(Throwable, Context)}). * An empty completion or error without at least one onNext signal triggers a short-circuit * of the main sequence with the same terminal signal (no resource is established, no * cleanup is invoked). * * @param resourceSupplier a {@link Publisher} that "generates" the resource, * subscribed for each subscription to the main sequence * @param resourceClosure a factory to derive a {@link Publisher} from the supplied resource * @param asyncComplete an asynchronous resource cleanup invoked if the resource closure terminates with onComplete * @param asyncError an asynchronous resource cleanup invoked if the resource closure terminates with onError * @param asyncCancel an asynchronous resource cleanup invoked if the resource closure is cancelled. * When {@code null}, the {@code asyncComplete} path is used instead. * @param <T> the type of elements emitted by the resource closure, and thus the main sequence * @param <D> the type of the resource object * @return a new {@link Flux} built around a "transactional" resource, with several * termination path triggering asynchronous cleanup sequences * @see #usingWhen(Publisher, Function, Function, Function) */
public static <T, D> Flux<T> usingWhen(Publisher<D> resourceSupplier, Function<? super D, ? extends Publisher<? extends T>> resourceClosure, Function<? super D, ? extends Publisher<?>> asyncComplete, Function<? super D, ? extends Publisher<?>> asyncError, //the operator itself accepts null for asyncCancel, but we won't in the public API Function<? super D, ? extends Publisher<?>> asyncCancel) { return onAssembly(new FluxUsingWhen<>(resourceSupplier, resourceClosure, asyncComplete, asyncError, asyncCancel)); }
Uses a resource, generated by a Publisher for each individual Subscriber, while streaming the values from a Publisher derived from the same resource. Whenever the resulting sequence terminates, a provided Function generates a "cleanup" Publisher that is invoked but doesn't change the content of the main sequence. Instead it just defers the termination (unless it errors, in which case the error suppresses the original termination signal).

Note that if the resource supplying Publisher emits more than one resource, the subsequent resources are dropped (Operators.onNextDropped(Object, Context)). If the publisher errors AFTER having emitted one resource, the error is also silently dropped (Operators.onErrorDropped(Throwable, Context)). An empty completion or error without at least one onNext signal triggers a short-circuit of the main sequence with the same terminal signal (no resource is established, no cleanup is invoked).

Params:
  • resourceSupplier – a Publisher that "generates" the resource, subscribed for each subscription to the main sequence
  • resourceClosure – a factory to derive a Publisher from the supplied resource
  • asyncCleanup – an asynchronous resource cleanup invoked when the resource closure terminates (with onComplete, onError or cancel)
Type parameters:
  • <T> – the type of elements emitted by the resource closure, and thus the main sequence
  • <D> – the type of the resource object
See Also:
Returns:a new Flux built around a "transactional" resource, with asynchronous cleanup on all terminations (onComplete, onError, cancel)
/** * Uses a resource, generated by a {@link Publisher} for each individual {@link Subscriber}, * while streaming the values from a {@link Publisher} derived from the same resource. * Whenever the resulting sequence terminates, a provided {@link Function} generates * a "cleanup" {@link Publisher} that is invoked but doesn't change the content of the * main sequence. Instead it just defers the termination (unless it errors, in which case * the error suppresses the original termination signal). * <p> * Note that if the resource supplying {@link Publisher} emits more than one resource, the * subsequent resources are dropped ({@link Operators#onNextDropped(Object, Context)}). If * the publisher errors AFTER having emitted one resource, the error is also silently dropped * ({@link Operators#onErrorDropped(Throwable, Context)}). * An empty completion or error without at least one onNext signal triggers a short-circuit * of the main sequence with the same terminal signal (no resource is established, no * cleanup is invoked). * * @param resourceSupplier a {@link Publisher} that "generates" the resource, * subscribed for each subscription to the main sequence * @param resourceClosure a factory to derive a {@link Publisher} from the supplied resource * @param asyncCleanup an asynchronous resource cleanup invoked when the resource * closure terminates (with onComplete, onError or cancel) * @param <T> the type of elements emitted by the resource closure, and thus the main sequence * @param <D> the type of the resource object * @return a new {@link Flux} built around a "transactional" resource, with asynchronous * cleanup on all terminations (onComplete, onError, cancel) * @see #usingWhen(Publisher, Function, Function, Function, Function) */
public static <T, D> Flux<T> usingWhen(Publisher<D> resourceSupplier, Function<? super D, ? extends Publisher<? extends T>> resourceClosure, Function<? super D, ? extends Publisher<?>> asyncCleanup) { return usingWhen(resourceSupplier, resourceClosure, asyncCleanup, asyncCleanup); }
Zip two sources together, that is to say wait for all the sources to emit one element and combine these elements once into an output value (constructed by the provided combinator). The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios.

Params:
  • source1 – The first Publisher source to zip.
  • source2 – The second Publisher source to zip.
  • combinator – The aggregate function that will receive a unique value from each upstream and return the value to signal downstream
Type parameters:
  • <T1> – type of the value from source1
  • <T2> – type of the value from source2
  • <O> – The produced output after transformation by the combinator
Returns:a zipped Flux
/** * Zip two sources together, that is to say wait for all the sources to emit one * element and combine these elements once into an output value (constructed by the provided * combinator). The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zip.png" alt=""> * <p> * * @param source1 The first {@link Publisher} source to zip. * @param source2 The second {@link Publisher} source to zip. * @param combinator The aggregate function that will receive a unique value from each upstream and return the * value to signal downstream * @param <T1> type of the value from source1 * @param <T2> type of the value from source2 * @param <O> The produced output after transformation by the combinator * * @return a zipped {@link Flux} */
public static <T1, T2, O> Flux<O> zip(Publisher<? extends T1> source1, Publisher<? extends T2> source2, final BiFunction<? super T1, ? super T2, ? extends O> combinator) { return onAssembly(new FluxZip<T1, O>(source1, source2, combinator, Queues.xs(), Queues.XS_BUFFER_SIZE)); }
Zip two sources together, that is to say wait for all the sources to emit one element and combine these elements once into a Tuple2. The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios.

Params:
  • source1 – The first Publisher source to zip.
  • source2 – The second Publisher source to zip.
Type parameters:
  • <T1> – type of the value from source1
  • <T2> – type of the value from source2
Returns:a zipped Flux
/** * Zip two sources together, that is to say wait for all the sources to emit one * element and combine these elements once into a {@link Tuple2}. * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zipt.png" alt=""> * <p> * @param source1 The first {@link Publisher} source to zip. * @param source2 The second {@link Publisher} source to zip. * @param <T1> type of the value from source1 * @param <T2> type of the value from source2 * * @return a zipped {@link Flux} */
public static <T1, T2> Flux<Tuple2<T1, T2>> zip(Publisher<? extends T1> source1, Publisher<? extends T2> source2) { return zip(source1, source2, tuple2Function()); }
Zip three sources together, that is to say wait for all the sources to emit one element and combine these elements once into a Tuple3. The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios.

Params:
  • source1 – The first upstream Publisher to subscribe to.
  • source2 – The second upstream Publisher to subscribe to.
  • source3 – The third upstream Publisher to subscribe to.
Type parameters:
  • <T1> – type of the value from source1
  • <T2> – type of the value from source2
  • <T3> – type of the value from source3
Returns:a zipped Flux
/** * Zip three sources together, that is to say wait for all the sources to emit one * element and combine these elements once into a {@link Tuple3}. * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zipt.png" alt=""> * <p> * @param source1 The first upstream {@link Publisher} to subscribe to. * @param source2 The second upstream {@link Publisher} to subscribe to. * @param source3 The third upstream {@link Publisher} to subscribe to. * @param <T1> type of the value from source1 * @param <T2> type of the value from source2 * @param <T3> type of the value from source3 * * @return a zipped {@link Flux} */
public static <T1, T2, T3> Flux<Tuple3<T1, T2, T3>> zip(Publisher<? extends T1> source1, Publisher<? extends T2> source2, Publisher<? extends T3> source3) { return zip(Tuples.fn3(), source1, source2, source3); }
Zip four sources together, that is to say wait for all the sources to emit one element and combine these elements once into a Tuple4. The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios.

Params:
  • source1 – The first upstream Publisher to subscribe to.
  • source2 – The second upstream Publisher to subscribe to.
  • source3 – The third upstream Publisher to subscribe to.
  • source4 – The fourth upstream Publisher to subscribe to.
Type parameters:
  • <T1> – type of the value from source1
  • <T2> – type of the value from source2
  • <T3> – type of the value from source3
  • <T4> – type of the value from source4
Returns:a zipped Flux
/** * Zip four sources together, that is to say wait for all the sources to emit one * element and combine these elements once into a {@link Tuple4}. * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zipt.png" alt=""> * <p> * @param source1 The first upstream {@link Publisher} to subscribe to. * @param source2 The second upstream {@link Publisher} to subscribe to. * @param source3 The third upstream {@link Publisher} to subscribe to. * @param source4 The fourth upstream {@link Publisher} to subscribe to. * @param <T1> type of the value from source1 * @param <T2> type of the value from source2 * @param <T3> type of the value from source3 * @param <T4> type of the value from source4 * * @return a zipped {@link Flux} */
public static <T1, T2, T3, T4> Flux<Tuple4<T1, T2, T3, T4>> zip(Publisher<? extends T1> source1, Publisher<? extends T2> source2, Publisher<? extends T3> source3, Publisher<? extends T4> source4) { return zip(Tuples.fn4(), source1, source2, source3, source4); }
Zip five sources together, that is to say wait for all the sources to emit one element and combine these elements once into a Tuple5. The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios.

Params:
  • source1 – The first upstream Publisher to subscribe to.
  • source2 – The second upstream Publisher to subscribe to.
  • source3 – The third upstream Publisher to subscribe to.
  • source4 – The fourth upstream Publisher to subscribe to.
  • source5 – The fifth upstream Publisher to subscribe to.
Type parameters:
  • <T1> – type of the value from source1
  • <T2> – type of the value from source2
  • <T3> – type of the value from source3
  • <T4> – type of the value from source4
  • <T5> – type of the value from source5
Returns:a zipped Flux
/** * Zip five sources together, that is to say wait for all the sources to emit one * element and combine these elements once into a {@link Tuple5}. * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zipt.png" alt=""> * <p> * @param source1 The first upstream {@link Publisher} to subscribe to. * @param source2 The second upstream {@link Publisher} to subscribe to. * @param source3 The third upstream {@link Publisher} to subscribe to. * @param source4 The fourth upstream {@link Publisher} to subscribe to. * @param source5 The fifth upstream {@link Publisher} to subscribe to. * @param <T1> type of the value from source1 * @param <T2> type of the value from source2 * @param <T3> type of the value from source3 * @param <T4> type of the value from source4 * @param <T5> type of the value from source5 * * @return a zipped {@link Flux} */
public static <T1, T2, T3, T4, T5> Flux<Tuple5<T1, T2, T3, T4, T5>> zip(Publisher<? extends T1> source1, Publisher<? extends T2> source2, Publisher<? extends T3> source3, Publisher<? extends T4> source4, Publisher<? extends T5> source5) { return zip(Tuples.fn5(), source1, source2, source3, source4, source5); }
Zip six sources together, that is to say wait for all the sources to emit one element and combine these elements once into a Tuple6. The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios.

Params:
  • source1 – The first upstream Publisher to subscribe to.
  • source2 – The second upstream Publisher to subscribe to.
  • source3 – The third upstream Publisher to subscribe to.
  • source4 – The fourth upstream Publisher to subscribe to.
  • source5 – The fifth upstream Publisher to subscribe to.
  • source6 – The sixth upstream Publisher to subscribe to.
Type parameters:
  • <T1> – type of the value from source1
  • <T2> – type of the value from source2
  • <T3> – type of the value from source3
  • <T4> – type of the value from source4
  • <T5> – type of the value from source5
  • <T6> – type of the value from source6
Returns:a zipped Flux
/** * Zip six sources together, that is to say wait for all the sources to emit one * element and combine these elements once into a {@link Tuple6}. * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zipt.png" alt=""> * <p> * @param source1 The first upstream {@link Publisher} to subscribe to. * @param source2 The second upstream {@link Publisher} to subscribe to. * @param source3 The third upstream {@link Publisher} to subscribe to. * @param source4 The fourth upstream {@link Publisher} to subscribe to. * @param source5 The fifth upstream {@link Publisher} to subscribe to. * @param source6 The sixth upstream {@link Publisher} to subscribe to. * @param <T1> type of the value from source1 * @param <T2> type of the value from source2 * @param <T3> type of the value from source3 * @param <T4> type of the value from source4 * @param <T5> type of the value from source5 * @param <T6> type of the value from source6 * * @return a zipped {@link Flux} */
public static <T1, T2, T3, T4, T5, T6> Flux<Tuple6<T1, T2, T3, T4, T5, T6>> zip(Publisher<? extends T1> source1, Publisher<? extends T2> source2, Publisher<? extends T3> source3, Publisher<? extends T4> source4, Publisher<? extends T5> source5, Publisher<? extends T6> source6) { return zip(Tuples.fn6(), source1, source2, source3, source4, source5, source6); }
Zip seven sources together, that is to say wait for all the sources to emit one element and combine these elements once into a Tuple7. The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios.

Params:
  • source1 – The first upstream Publisher to subscribe to.
  • source2 – The second upstream Publisher to subscribe to.
  • source3 – The third upstream Publisher to subscribe to.
  • source4 – The fourth upstream Publisher to subscribe to.
  • source5 – The fifth upstream Publisher to subscribe to.
  • source6 – The sixth upstream Publisher to subscribe to.
  • source7 – The seventh upstream Publisher to subscribe to.
Type parameters:
  • <T1> – type of the value from source1
  • <T2> – type of the value from source2
  • <T3> – type of the value from source3
  • <T4> – type of the value from source4
  • <T5> – type of the value from source5
  • <T6> – type of the value from source6
  • <T7> – type of the value from source7
Returns:a zipped Flux
/** * Zip seven sources together, that is to say wait for all the sources to emit one * element and combine these elements once into a {@link Tuple7}. * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zipt.png" alt=""> * <p> * @param source1 The first upstream {@link Publisher} to subscribe to. * @param source2 The second upstream {@link Publisher} to subscribe to. * @param source3 The third upstream {@link Publisher} to subscribe to. * @param source4 The fourth upstream {@link Publisher} to subscribe to. * @param source5 The fifth upstream {@link Publisher} to subscribe to. * @param source6 The sixth upstream {@link Publisher} to subscribe to. * @param source7 The seventh upstream {@link Publisher} to subscribe to. * @param <T1> type of the value from source1 * @param <T2> type of the value from source2 * @param <T3> type of the value from source3 * @param <T4> type of the value from source4 * @param <T5> type of the value from source5 * @param <T6> type of the value from source6 * @param <T7> type of the value from source7 * * @return a zipped {@link Flux} */
public static <T1, T2, T3, T4, T5, T6, T7> Flux<Tuple7<T1, T2, T3, T4, T5, T6, T7>> zip(Publisher<? extends T1> source1, Publisher<? extends T2> source2, Publisher<? extends T3> source3, Publisher<? extends T4> source4, Publisher<? extends T5> source5, Publisher<? extends T6> source6, Publisher<? extends T7> source7) { return zip(Tuples.fn7(), source1, source2, source3, source4, source5, source6, source7); }
Zip eight sources together, that is to say wait for all the sources to emit one element and combine these elements once into a Tuple8. The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios.

Params:
  • source1 – The first upstream Publisher to subscribe to.
  • source2 – The second upstream Publisher to subscribe to.
  • source3 – The third upstream Publisher to subscribe to.
  • source4 – The fourth upstream Publisher to subscribe to.
  • source5 – The fifth upstream Publisher to subscribe to.
  • source6 – The sixth upstream Publisher to subscribe to.
  • source7 – The seventh upstream Publisher to subscribe to.
  • source8 – The eight upstream Publisher to subscribe to.
Type parameters:
  • <T1> – type of the value from source1
  • <T2> – type of the value from source2
  • <T3> – type of the value from source3
  • <T4> – type of the value from source4
  • <T5> – type of the value from source5
  • <T6> – type of the value from source6
  • <T7> – type of the value from source7
  • <T8> – type of the value from source8
Returns:a zipped Flux
/** * Zip eight sources together, that is to say wait for all the sources to emit one * element and combine these elements once into a {@link Tuple8}. * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zipt.png" alt=""> * <p> * @param source1 The first upstream {@link Publisher} to subscribe to. * @param source2 The second upstream {@link Publisher} to subscribe to. * @param source3 The third upstream {@link Publisher} to subscribe to. * @param source4 The fourth upstream {@link Publisher} to subscribe to. * @param source5 The fifth upstream {@link Publisher} to subscribe to. * @param source6 The sixth upstream {@link Publisher} to subscribe to. * @param source7 The seventh upstream {@link Publisher} to subscribe to. * @param source8 The eight upstream {@link Publisher} to subscribe to. * @param <T1> type of the value from source1 * @param <T2> type of the value from source2 * @param <T3> type of the value from source3 * @param <T4> type of the value from source4 * @param <T5> type of the value from source5 * @param <T6> type of the value from source6 * @param <T7> type of the value from source7 * @param <T8> type of the value from source8 * * @return a zipped {@link Flux} */
public static <T1, T2, T3, T4, T5, T6, T7, T8> Flux<Tuple8<T1, T2, T3, T4, T5, T6, T7, T8>> zip(Publisher<? extends T1> source1, Publisher<? extends T2> source2, Publisher<? extends T3> source3, Publisher<? extends T4> source4, Publisher<? extends T5> source5, Publisher<? extends T6> source6, Publisher<? extends T7> source7, Publisher<? extends T8> source8) { return zip(Tuples.fn8(), source1, source2, source3, source4, source5, source6, source7, source8); }
Zip multiple sources together, that is to say wait for all the sources to emit one element and combine these elements once into an output value (constructed by the provided combinator). The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. The Iterable.iterator() will be called on each Publisher.subscribe(Subscriber).

Params:
  • sources – the Iterable providing sources to zip
  • combinator – The aggregate function that will receive a unique value from each upstream and return the value to signal downstream
Type parameters:
  • <O> – the combined produced type
Returns:a zipped Flux
/** * Zip multiple sources together, that is to say wait for all the sources to emit one * element and combine these elements once into an output value (constructed by the provided * combinator). * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * * The {@link Iterable#iterator()} will be called on each {@link Publisher#subscribe(Subscriber)}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zip.png" alt=""> * * @param sources the {@link Iterable} providing sources to zip * @param combinator The aggregate function that will receive a unique value from each upstream and return the value * to signal downstream * @param <O> the combined produced type * * @return a zipped {@link Flux} */
public static <O> Flux<O> zip(Iterable<? extends Publisher<?>> sources, final Function<? super Object[], ? extends O> combinator) { return zip(sources, Queues.XS_BUFFER_SIZE, combinator); }
Zip multiple sources together, that is to say wait for all the sources to emit one element and combine these elements once into an output value (constructed by the provided combinator). The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. The Iterable.iterator() will be called on each Publisher.subscribe(Subscriber).

Params:
  • sources – the Iterable providing sources to zip
  • prefetch – the inner source request size
  • combinator – The aggregate function that will receive a unique value from each upstream and return the value to signal downstream
Type parameters:
  • <O> – the combined produced type
Returns:a zipped Flux
/** * Zip multiple sources together, that is to say wait for all the sources to emit one * element and combine these elements once into an output value (constructed by the provided * combinator). * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * * The {@link Iterable#iterator()} will be called on each {@link Publisher#subscribe(Subscriber)}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zipp.png" alt=""> * * @param sources the {@link Iterable} providing sources to zip * @param prefetch the inner source request size * @param combinator The aggregate function that will receive a unique value from each upstream and return the value * to signal downstream * @param <O> the combined produced type * * @return a zipped {@link Flux} */
public static <O> Flux<O> zip(Iterable<? extends Publisher<?>> sources, int prefetch, final Function<? super Object[], ? extends O> combinator) { return onAssembly(new FluxZip<Object, O>(sources, combinator, Queues.get(prefetch), prefetch)); }
Zip multiple sources together, that is to say wait for all the sources to emit one element and combine these elements once into an output value (constructed by the provided combinator). The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios.

Params:
  • combinator – The aggregate function that will receive a unique value from each upstream and return the value to signal downstream
  • sources – the array providing sources to zip
Type parameters:
  • <I> – the type of the input sources
  • <O> – the combined produced type
Returns:a zipped Flux
/** * Zip multiple sources together, that is to say wait for all the sources to emit one * element and combine these elements once into an output value (constructed by the provided * combinator). * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zip.png" alt=""> * <p> * @param combinator The aggregate function that will receive a unique value from each upstream and return the * value to signal downstream * @param sources the array providing sources to zip * @param <I> the type of the input sources * @param <O> the combined produced type * * @return a zipped {@link Flux} */
@SafeVarargs public static <I, O> Flux<O> zip( final Function<? super Object[], ? extends O> combinator, Publisher<? extends I>... sources) { return zip(combinator, Queues.XS_BUFFER_SIZE, sources); }
Zip multiple sources together, that is to say wait for all the sources to emit one element and combine these elements once into an output value (constructed by the provided combinator). The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios.

Params:
  • combinator – The aggregate function that will receive a unique value from each upstream and return the value to signal downstream
  • prefetch – individual source request size
  • sources – the array providing sources to zip
Type parameters:
  • <I> – the type of the input sources
  • <O> – the combined produced type
Returns:a zipped Flux
/** * Zip multiple sources together, that is to say wait for all the sources to emit one * element and combine these elements once into an output value (constructed by the provided * combinator). * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zipp.png" alt=""> * <p> * @param combinator The aggregate function that will receive a unique value from each upstream and return the * value to signal downstream * @param prefetch individual source request size * @param sources the array providing sources to zip * @param <I> the type of the input sources * @param <O> the combined produced type * * @return a zipped {@link Flux} */
@SafeVarargs public static <I, O> Flux<O> zip(final Function<? super Object[], ? extends O> combinator, int prefetch, Publisher<? extends I>... sources) { if (sources.length == 0) { return empty(); } if (sources.length == 1) { Publisher<? extends I> source = sources[0]; if (source instanceof Fuseable) { return onAssembly(new FluxMapFuseable<>(from(source), v -> combinator.apply(new Object[]{v}))); } return onAssembly(new FluxMap<>(from(source), v -> combinator.apply(new Object[]{v}))); } return onAssembly(new FluxZip<>(sources, combinator, Queues.get(prefetch), prefetch)); }
Zip multiple sources together, that is to say wait for all the sources to emit one element and combine these elements once into an output value (constructed by the provided combinator). The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios.

Note that the Publisher sources from the outer Publisher will accumulate into an exhaustive list before starting zip operation.

Params:
  • sources – The Publisher of Publisher sources to zip. A finite publisher is required.
  • combinator – The aggregate function that will receive a unique value from each upstream and return the value to signal downstream
Type parameters:
  • <TUPLE> – the raw tuple type
  • <V> – The produced output after transformation by the given combinator
Returns:a Flux based on the produced value
/** * Zip multiple sources together, that is to say wait for all the sources to emit one * element and combine these elements once into an output value (constructed by the provided * combinator). * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * <p> * Note that the {@link Publisher} sources from the outer {@link Publisher} will * accumulate into an exhaustive list before starting zip operation. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zip.png" * alt=""> * * @param sources The {@link Publisher} of {@link Publisher} sources to zip. A finite publisher is required. * @param combinator The aggregate function that will receive a unique value from each upstream and return the value * to signal downstream * @param <TUPLE> the raw tuple type * @param <V> The produced output after transformation by the given combinator * * @return a {@link Flux} based on the produced value */
@SuppressWarnings({ "rawtypes", "unchecked" }) public static <TUPLE extends Tuple2, V> Flux<V> zip(Publisher<? extends Publisher<?>> sources, final Function<? super TUPLE, ? extends V> combinator) { return onAssembly(new FluxBuffer<>(from(sources), Integer.MAX_VALUE, listSupplier()) .flatMap(new Function<List<? extends Publisher<?>>, Publisher<V>>() { @Override public Publisher<V> apply(List<? extends Publisher<?>> publishers) { return zip(Tuples.fnAny((Function<Tuple2, V>) combinator), publishers.toArray(new Publisher[publishers .size()])); } })); }
Emit a single boolean true if all values of this sequence match the Predicate.

The implementation uses short-circuit logic and completes with false if the predicate doesn't match a value.

Params:
  • predicate – the Predicate that needs to apply to all emitted items
Returns:a new Mono with true if all values satisfies a predicate and false otherwise
/** * * Emit a single boolean true if all values of this sequence match * the {@link Predicate}. * <p> * The implementation uses short-circuit logic and completes with false if * the predicate doesn't match a value. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/all.png" alt=""> * * @param predicate the {@link Predicate} that needs to apply to all emitted items * * @return a new {@link Mono} with <code>true</code> if all values satisfies a predicate and <code>false</code> * otherwise */
public final Mono<Boolean> all(Predicate<? super T> predicate) { return Mono.onAssembly(new MonoAll<>(this, predicate)); }
Emit a single boolean true if any of the values of this Flux sequence match the predicate.

The implementation uses short-circuit logic and completes with false if any value doesn't match the predicate.

Params:
  • predicate – the Predicate that needs to apply to at least one emitted item
Returns:a new Mono with true if any value satisfies a predicate and false otherwise
/** * Emit a single boolean true if any of the values of this {@link Flux} sequence match * the predicate. * <p> * The implementation uses short-circuit logic and completes with false if any value * doesn't match the predicate. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/any.png" alt=""> * * @param predicate the {@link Predicate} that needs to apply to at least one emitted item * * @return a new {@link Mono} with <code>true</code> if any value satisfies a predicate and <code>false</code> * otherwise */
public final Mono<Boolean> any(Predicate<? super T> predicate) { return Mono.onAssembly(new MonoAny<>(this, predicate)); }
Transform this Flux into a target type.
flux.as(Mono::from).subscribe()  
Params:
  • transformer – the Function to immediately map this Flux into a target type instance.
Type parameters:
  • <P> – the returned instance type
See Also:
Returns:the Flux transformed to an instance of P
/** * Transform this {@link Flux} into a target type. * <blockquote><pre> * {@code flux.as(Mono::from).subscribe() } * </pre></blockquote> * * @param transformer the {@link Function} to immediately map this {@link Flux} * into a target type instance. * @param <P> the returned instance type * * @return the {@link Flux} transformed to an instance of P * @see #compose for a bounded conversion to {@link Publisher} */
public final <P> P as(Function<? super Flux<T>, P> transformer) { return transformer.apply(this); }
Subscribe to this Flux and block indefinitely until the upstream signals its first value or completes. Returns that value, or null if the Flux completes empty. In case the Flux errors, the original exception is thrown (wrapped in a RuntimeException if it was a checked exception).

Note that each blockFirst() will trigger a new subscription: in other words, the result might miss signal from hot publishers.

Returns:the first value or null
/** * Subscribe to this {@link Flux} and <strong>block indefinitely</strong> * until the upstream signals its first value or completes. Returns that value, * or null if the Flux completes empty. In case the Flux errors, the original * exception is thrown (wrapped in a {@link RuntimeException} if it was a checked * exception). * <p> * Note that each blockFirst() will trigger a new subscription: in other words, * the result might miss signal from hot publishers. * * @return the first value or null */
@Nullable public final T blockFirst() { BlockingFirstSubscriber<T> subscriber = new BlockingFirstSubscriber<>(); onLastAssembly(this).subscribe(Operators.toCoreSubscriber(subscriber)); return subscriber.blockingGet(); }
Subscribe to this Flux and block until the upstream signals its first value, completes or a timeout expires. Returns that value, or null if the Flux completes empty. In case the Flux errors, the original exception is thrown (wrapped in a RuntimeException if it was a checked exception). If the provided timeout expires,a RuntimeException is thrown.

Note that each blockFirst() will trigger a new subscription: in other words, the result might miss signal from hot publishers.

Params:
Returns:the first value or null
/** * Subscribe to this {@link Flux} and <strong>block</strong> until the upstream * signals its first value, completes or a timeout expires. Returns that value, * or null if the Flux completes empty. In case the Flux errors, the original * exception is thrown (wrapped in a {@link RuntimeException} if it was a checked * exception). If the provided timeout expires,a {@link RuntimeException} is thrown. * <p> * Note that each blockFirst() will trigger a new subscription: in other words, * the result might miss signal from hot publishers. * * @param timeout maximum time period to wait for before raising a {@link RuntimeException} * @return the first value or null */
@Nullable public final T blockFirst(Duration timeout) { BlockingFirstSubscriber<T> subscriber = new BlockingFirstSubscriber<>(); onLastAssembly(this).subscribe(Operators.toCoreSubscriber(subscriber)); return subscriber.blockingGet(timeout.toMillis(), TimeUnit.MILLISECONDS); }
Subscribe to this Flux and block indefinitely until the upstream signals its last value or completes. Returns that value, or null if the Flux completes empty. In case the Flux errors, the original exception is thrown (wrapped in a RuntimeException if it was a checked exception).

Note that each blockLast() will trigger a new subscription: in other words, the result might miss signal from hot publishers.

Returns:the last value or null
/** * Subscribe to this {@link Flux} and <strong>block indefinitely</strong> * until the upstream signals its last value or completes. Returns that value, * or null if the Flux completes empty. In case the Flux errors, the original * exception is thrown (wrapped in a {@link RuntimeException} if it was a checked * exception). * <p> * Note that each blockLast() will trigger a new subscription: in other words, * the result might miss signal from hot publishers. * * @return the last value or null */
@Nullable public final T blockLast() { BlockingLastSubscriber<T> subscriber = new BlockingLastSubscriber<>(); onLastAssembly(this).subscribe(Operators.toCoreSubscriber(subscriber)); return subscriber.blockingGet(); }
Subscribe to this Flux and block until the upstream signals its last value, completes or a timeout expires. Returns that value, or null if the Flux completes empty. In case the Flux errors, the original exception is thrown (wrapped in a RuntimeException if it was a checked exception). If the provided timeout expires,a RuntimeException is thrown.

Note that each blockLast() will trigger a new subscription: in other words, the result might miss signal from hot publishers.

Params:
Returns:the last value or null
/** * Subscribe to this {@link Flux} and <strong>block</strong> until the upstream * signals its last value, completes or a timeout expires. Returns that value, * or null if the Flux completes empty. In case the Flux errors, the original * exception is thrown (wrapped in a {@link RuntimeException} if it was a checked * exception). If the provided timeout expires,a {@link RuntimeException} is thrown. * <p> * Note that each blockLast() will trigger a new subscription: in other words, * the result might miss signal from hot publishers. * * @param timeout maximum time period to wait for before raising a {@link RuntimeException} * @return the last value or null */
@Nullable public final T blockLast(Duration timeout) { BlockingLastSubscriber<T> subscriber = new BlockingLastSubscriber<>(); onLastAssembly(this).subscribe(Operators.toCoreSubscriber(subscriber)); return subscriber.blockingGet(timeout.toMillis(), TimeUnit.MILLISECONDS); }
Collect all incoming values into a single List buffer that will be emitted by the returned Flux once this Flux completes.

See Also:
@reactor.discardThis operator discards the buffer upon cancellation or error triggered by a data signal.
Returns:a buffered Flux of at most one List
/** * Collect all incoming values into a single {@link List} buffer that will be emitted * by the returned {@link Flux} once this Flux completes. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffer.png" * alt=""> * * @reactor.discard This operator discards the buffer upon cancellation or error triggered by a data signal. * * @return a buffered {@link Flux} of at most one {@link List} * @see #collectList() for an alternative collecting algorithm returning {@link Mono} */
public final Flux<List<T>> buffer() { return buffer(Integer.MAX_VALUE); }
Collect incoming values into multiple List buffers that will be emitted by the returned Flux each time the given max size is reached or once this Flux completes.

Params:
  • maxSize – the maximum collected size
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal.
Returns:a microbatched Flux of List
/** * Collect incoming values into multiple {@link List} buffers that will be emitted * by the returned {@link Flux} each time the given max size is reached or once this * Flux completes. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffersize.png" * alt=""> * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal. * * @param maxSize the maximum collected size * * @return a microbatched {@link Flux} of {@link List} */
public final Flux<List<T>> buffer(int maxSize) { return buffer(maxSize, listSupplier()); }
Collect incoming values into multiple user-defined Collection buffers that will be emitted by the returned Flux each time the given max size is reached or once this Flux completes.

Params:
  • maxSize – the maximum collected size
  • bufferSupplier – a Supplier of the concrete Collection to use for each buffer
Type parameters:
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal, as well as latest unbuffered element if the bufferSupplier fails.
Returns:a microbatched Flux of Collection
/** * Collect incoming values into multiple user-defined {@link Collection} buffers that * will be emitted by the returned {@link Flux} each time the given max size is reached * or once this Flux completes. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffersize.png" * alt=""> * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal, * as well as latest unbuffered element if the bufferSupplier fails. * * @param maxSize the maximum collected size * @param bufferSupplier a {@link Supplier} of the concrete {@link Collection} to use for each buffer * @param <C> the {@link Collection} buffer type * * @return a microbatched {@link Flux} of {@link Collection} */
public final <C extends Collection<? super T>> Flux<C> buffer(int maxSize, Supplier<C> bufferSupplier) { return onAssembly(new FluxBuffer<>(this, maxSize, bufferSupplier)); }
Collect incoming values into multiple List buffers that will be emitted by the returned Flux each time the given max size is reached or once this Flux completes. Buffers can be created with gaps, as a new buffer will be created every time skip values have been emitted by the source.

When maxSize < skip : dropping buffers

When maxSize > skip : overlapping buffers

When maxSize == skip : exact buffers

Params:
  • skip – the number of items to count before creating a new buffer
  • maxSize – the max collected size
@reactor.discardThis operator discards elements in between buffers (in the case of dropping buffers). It also discards the currently open buffer upon cancellation or error triggered by a data signal. Note however that overlapping buffer variant DOES NOT discard, as this might result in an element being discarded from an early buffer while it is still valid in a more recent buffer.
Returns:a microbatched Flux of possibly overlapped or gapped List
/** * Collect incoming values into multiple {@link List} buffers that will be emitted * by the returned {@link Flux} each time the given max size is reached or once this * Flux completes. Buffers can be created with gaps, as a new buffer will be created * every time {@code skip} values have been emitted by the source. * <p> * When maxSize < skip : dropping buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffersizeskip.png" * alt=""> * <p> * When maxSize > skip : overlapping buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffersizeskipover.png" * alt=""> * <p> * When maxSize == skip : exact buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffersize.png" * alt=""> * * @reactor.discard This operator discards elements in between buffers (in the case of * dropping buffers). It also discards the currently open buffer upon cancellation or error triggered by a data signal. * Note however that overlapping buffer variant DOES NOT discard, as this might result in an element * being discarded from an early buffer while it is still valid in a more recent buffer. * * @param skip the number of items to count before creating a new buffer * @param maxSize the max collected size * * @return a microbatched {@link Flux} of possibly overlapped or gapped {@link List} */
public final Flux<List<T>> buffer(int maxSize, int skip) { return buffer(maxSize, skip, listSupplier()); }
Collect incoming values into multiple user-defined Collection buffers that will be emitted by the returned Flux each time the given max size is reached or once this Flux completes. Buffers can be created with gaps, as a new buffer will be created every time skip values have been emitted by the source

When maxSize < skip : dropping buffers

When maxSize > skip : overlapping buffers

When maxSize == skip : exact buffers

Params:
  • skip – the number of items to count before creating a new buffer
  • maxSize – the max collected size
  • bufferSupplier – a Supplier of the concrete Collection to use for each buffer
Type parameters:
@reactor.discardThis operator discards elements in between buffers (in the case of dropping buffers). It also discards the currently open buffer upon cancellation or error triggered by a data signal. Note however that overlapping buffer variant DOES NOT discard, as this might result in an element being discarded from an early buffer while it is still valid in a more recent buffer.
Returns:a microbatched Flux of possibly overlapped or gapped Collection
/** * Collect incoming values into multiple user-defined {@link Collection} buffers that * will be emitted by the returned {@link Flux} each time the given max size is reached * or once this Flux completes. Buffers can be created with gaps, as a new buffer will * be created every time {@code skip} values have been emitted by the source * <p> * When maxSize < skip : dropping buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffersizeskip.png" * alt=""> * <p> * When maxSize > skip : overlapping buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffersizeskipover.png" * alt=""> * <p> * When maxSize == skip : exact buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffersize.png" * alt=""> * * @reactor.discard This operator discards elements in between buffers (in the case of * dropping buffers). It also discards the currently open buffer upon cancellation or error triggered by a data signal. * Note however that overlapping buffer variant DOES NOT discard, as this might result in an element * being discarded from an early buffer while it is still valid in a more recent buffer. * * @param skip the number of items to count before creating a new buffer * @param maxSize the max collected size * @param bufferSupplier a {@link Supplier} of the concrete {@link Collection} to use for each buffer * @param <C> the {@link Collection} buffer type * * @return a microbatched {@link Flux} of possibly overlapped or gapped * {@link Collection} */
public final <C extends Collection<? super T>> Flux<C> buffer(int maxSize, int skip, Supplier<C> bufferSupplier) { return onAssembly(new FluxBuffer<>(this, maxSize, skip, bufferSupplier)); }
Collect incoming values into multiple List buffers, as delimited by the signals of a companion Publisher this operator will subscribe to.

Params:
  • other – the companion Publisher whose signals trigger new buffers
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal.
Returns:a microbatched Flux of List delimited by signals from a Publisher
/** * Collect incoming values into multiple {@link List} buffers, as delimited by the * signals of a companion {@link Publisher} this operator will subscribe to. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/bufferboundary.png" * alt=""> * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal. * * @param other the companion {@link Publisher} whose signals trigger new buffers * * @return a microbatched {@link Flux} of {@link List} delimited by signals from a {@link Publisher} */
public final Flux<List<T>> buffer(Publisher<?> other) { return buffer(other, listSupplier()); }
Collect incoming values into multiple user-defined Collection buffers, as delimited by the signals of a companion Publisher this operator will subscribe to.

Params:
  • other – the companion Publisher whose signals trigger new buffers
  • bufferSupplier – a Supplier of the concrete Collection to use for each buffer
Type parameters:
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal, and the last received element when the bufferSupplier fails.
Returns:a microbatched Flux of Collection delimited by signals from a Publisher
/** * Collect incoming values into multiple user-defined {@link Collection} buffers, as * delimited by the signals of a companion {@link Publisher} this operator will * subscribe to. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/bufferboundary.png" * alt=""> * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal, * and the last received element when the bufferSupplier fails. * * @param other the companion {@link Publisher} whose signals trigger new buffers * @param bufferSupplier a {@link Supplier} of the concrete {@link Collection} to use for each buffer * @param <C> the {@link Collection} buffer type * * @return a microbatched {@link Flux} of {@link Collection} delimited by signals from a {@link Publisher} */
public final <C extends Collection<? super T>> Flux<C> buffer(Publisher<?> other, Supplier<C> bufferSupplier) { return onAssembly(new FluxBufferBoundary<>(this, other, bufferSupplier)); }
Collect incoming values into multiple List buffers that will be emitted by the returned Flux every bufferingTimespan.

Params:
  • bufferingTimespan – the duration from buffer creation until a buffer is closed and emitted
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal.
Returns:a microbatched Flux of List delimited by the given time span
/** * Collect incoming values into multiple {@link List} buffers that will be emitted by * the returned {@link Flux} every {@code bufferingTimespan}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffertimespan.png" * alt=""> * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal. * * @param bufferingTimespan the duration from buffer creation until a buffer is closed and emitted * * @return a microbatched {@link Flux} of {@link List} delimited by the given time span */
public final Flux<List<T>> buffer(Duration bufferingTimespan) { return buffer(bufferingTimespan, Schedulers.parallel()); }
Collect incoming values into multiple List buffers created at a given openBufferEvery period. Each buffer will last until the bufferingTimespan has elapsed, thus emitting the bucket in the resulting Flux.

When bufferingTimespan < openBufferEvery : dropping buffers

When bufferingTimespan > openBufferEvery : overlapping buffers

When bufferingTimespan == openBufferEvery : exact buffers

Params:
  • bufferingTimespan – the duration from buffer creation until a buffer is closed and emitted
  • openBufferEvery – the interval at which to create a new buffer
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal. It DOES NOT provide strong guarantees in the case of overlapping buffers, as elements might get discarded too early (from the first of two overlapping buffers for instance).
Returns:a microbatched Flux of List delimited by the given period openBufferEvery and sized by bufferingTimespan
/** * Collect incoming values into multiple {@link List} buffers created at a given * {@code openBufferEvery} period. Each buffer will last until the {@code bufferingTimespan} has elapsed, * thus emitting the bucket in the resulting {@link Flux}. * <p> * When bufferingTimespan < openBufferEvery : dropping buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffertimeshift.png" * alt=""> * <p> * When bufferingTimespan > openBufferEvery : overlapping buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffertimeshiftover.png" * alt=""> * <p> * When bufferingTimespan == openBufferEvery : exact buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffertimespan.png" * alt=""> * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal. * It DOES NOT provide strong guarantees in the case of overlapping buffers, as elements * might get discarded too early (from the first of two overlapping buffers for instance). * * @param bufferingTimespan the duration from buffer creation until a buffer is closed and emitted * @param openBufferEvery the interval at which to create a new buffer * * @return a microbatched {@link Flux} of {@link List} delimited by the given period openBufferEvery and sized by bufferingTimespan */
public final Flux<List<T>> buffer(Duration bufferingTimespan, Duration openBufferEvery) { return buffer(bufferingTimespan, openBufferEvery, Schedulers.parallel()); }
Collect incoming values into multiple List buffers that will be emitted by the returned Flux every bufferingTimespan, as measured on the provided Scheduler.

Params:
  • bufferingTimespan – the duration from buffer creation until a buffer is closed and emitted
  • timer – a time-capable Scheduler instance to run on
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal.
Returns:a microbatched Flux of List delimited by the given period
/** * Collect incoming values into multiple {@link List} buffers that will be emitted by * the returned {@link Flux} every {@code bufferingTimespan}, as measured on the provided {@link Scheduler}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffertimespan.png" * alt=""> * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal. * * @param bufferingTimespan the duration from buffer creation until a buffer is closed and emitted * @param timer a time-capable {@link Scheduler} instance to run on * * @return a microbatched {@link Flux} of {@link List} delimited by the given period */
public final Flux<List<T>> buffer(Duration bufferingTimespan, Scheduler timer) { return buffer(interval(bufferingTimespan, timer)); }
Collect incoming values into multiple List buffers created at a given openBufferEvery period, as measured on the provided Scheduler. Each buffer will last until the bufferingTimespan has elapsed (also measured on the scheduler), thus emitting the bucket in the resulting Flux.

When bufferingTimespan < openBufferEvery : dropping buffers

When bufferingTimespan > openBufferEvery : overlapping buffers

When bufferingTimespan == openBufferEvery : exact buffers

Params:
  • bufferingTimespan – the duration from buffer creation until a buffer is closed and emitted
  • openBufferEvery – the interval at which to create a new buffer
  • timer – a time-capable Scheduler instance to run on
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal. It DOES NOT provide strong guarantees in the case of overlapping buffers, as elements might get discarded too early (from the first of two overlapping buffers for instance).
Returns:a microbatched Flux of List delimited by the given period openBufferEvery and sized by bufferingTimespan
/** * Collect incoming values into multiple {@link List} buffers created at a given * {@code openBufferEvery} period, as measured on the provided {@link Scheduler}. Each * buffer will last until the {@code bufferingTimespan} has elapsed (also measured on the scheduler), * thus emitting the bucket in the resulting {@link Flux}. * <p> * When bufferingTimespan < openBufferEvery : dropping buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffertimeshift.png" * alt=""> * <p> * When bufferingTimespan > openBufferEvery : overlapping buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffertimeshiftover.png" * alt=""> * <p> * When bufferingTimespan == openBufferEvery : exact buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffertimespan.png" * alt=""> * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal. * It DOES NOT provide strong guarantees in the case of overlapping buffers, as elements * might get discarded too early (from the first of two overlapping buffers for instance). * * @param bufferingTimespan the duration from buffer creation until a buffer is closed and emitted * @param openBufferEvery the interval at which to create a new buffer * @param timer a time-capable {@link Scheduler} instance to run on * * @return a microbatched {@link Flux} of {@link List} delimited by the given period openBufferEvery and sized by bufferingTimespan */
public final Flux<List<T>> buffer(Duration bufferingTimespan, Duration openBufferEvery, Scheduler timer) { if (bufferingTimespan.equals(openBufferEvery)) { return buffer(bufferingTimespan, timer); } return bufferWhen(interval(Duration.ZERO, openBufferEvery, timer), aLong -> Mono .delay(bufferingTimespan, timer)); }
Collect incoming values into multiple List buffers that will be emitted by the returned Flux each time the buffer reaches a maximum size OR the maxTime Duration elapses.

Params:
  • maxSize – the max collected size
  • maxTime – the timeout enforcing the release of a partial buffer
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal.
Returns:a microbatched Flux of List delimited by given size or a given period timeout
/** * Collect incoming values into multiple {@link List} buffers that will be emitted * by the returned {@link Flux} each time the buffer reaches a maximum size OR the * maxTime {@link Duration} elapses. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffertimespansize.png" * alt=""> * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal. * * @param maxSize the max collected size * @param maxTime the timeout enforcing the release of a partial buffer * * @return a microbatched {@link Flux} of {@link List} delimited by given size or a given period timeout */
public final Flux<List<T>> bufferTimeout(int maxSize, Duration maxTime) { return bufferTimeout(maxSize, maxTime, listSupplier()); }
Collect incoming values into multiple user-defined Collection buffers that will be emitted by the returned Flux each time the buffer reaches a maximum size OR the maxTime Duration elapses.

Params:
  • maxSize – the max collected size
  • maxTime – the timeout enforcing the release of a partial buffer
  • bufferSupplier – a Supplier of the concrete Collection to use for each buffer
Type parameters:
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal.
Returns:a microbatched Flux of Collection delimited by given size or a given period timeout
/** * Collect incoming values into multiple user-defined {@link Collection} buffers that * will be emitted by the returned {@link Flux} each time the buffer reaches a maximum * size OR the maxTime {@link Duration} elapses. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffertimespansize.png" * alt=""> * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal. * * @param maxSize the max collected size * @param maxTime the timeout enforcing the release of a partial buffer * @param bufferSupplier a {@link Supplier} of the concrete {@link Collection} to use for each buffer * @param <C> the {@link Collection} buffer type * @return a microbatched {@link Flux} of {@link Collection} delimited by given size or a given period timeout */
public final <C extends Collection<? super T>> Flux<C> bufferTimeout(int maxSize, Duration maxTime, Supplier<C> bufferSupplier) { return bufferTimeout(maxSize, maxTime, Schedulers.parallel(), bufferSupplier); }
Collect incoming values into multiple List buffers that will be emitted by the returned Flux each time the buffer reaches a maximum size OR the maxTime Duration elapses, as measured on the provided Scheduler.

Params:
  • maxSize – the max collected size
  • maxTime – the timeout enforcing the release of a partial buffer
  • timer – a time-capable Scheduler instance to run on
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal.
Returns:a microbatched Flux of List delimited by given size or a given period timeout
/** * Collect incoming values into multiple {@link List} buffers that will be emitted * by the returned {@link Flux} each time the buffer reaches a maximum size OR the * maxTime {@link Duration} elapses, as measured on the provided {@link Scheduler}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffertimespansize.png" * alt=""> * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal. * * @param maxSize the max collected size * @param maxTime the timeout enforcing the release of a partial buffer * @param timer a time-capable {@link Scheduler} instance to run on * * @return a microbatched {@link Flux} of {@link List} delimited by given size or a given period timeout */
public final Flux<List<T>> bufferTimeout(int maxSize, Duration maxTime, Scheduler timer) { return bufferTimeout(maxSize, maxTime, timer, listSupplier()); }
Collect incoming values into multiple user-defined Collection buffers that will be emitted by the returned Flux each time the buffer reaches a maximum size OR the maxTime Duration elapses, as measured on the provided Scheduler.

Params:
  • maxSize – the max collected size
  • maxTime – the timeout enforcing the release of a partial buffer
  • timer – a time-capable Scheduler instance to run on
  • bufferSupplier – a Supplier of the concrete Collection to use for each buffer
Type parameters:
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal.
Returns:a microbatched Flux of Collection delimited by given size or a given period timeout
/** * Collect incoming values into multiple user-defined {@link Collection} buffers that * will be emitted by the returned {@link Flux} each time the buffer reaches a maximum * size OR the maxTime {@link Duration} elapses, as measured on the provided {@link Scheduler}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffertimespansize.png" * alt=""> * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal. * * @param maxSize the max collected size * @param maxTime the timeout enforcing the release of a partial buffer * @param timer a time-capable {@link Scheduler} instance to run on * @param bufferSupplier a {@link Supplier} of the concrete {@link Collection} to use for each buffer * @param <C> the {@link Collection} buffer type * @return a microbatched {@link Flux} of {@link Collection} delimited by given size or a given period timeout */
public final <C extends Collection<? super T>> Flux<C> bufferTimeout(int maxSize, Duration maxTime, Scheduler timer, Supplier<C> bufferSupplier) { return onAssembly(new FluxBufferTimeout<>(this, maxSize, maxTime.toMillis(), timer, bufferSupplier)); }
Collect incoming values into multiple List buffers that will be emitted by the resulting Flux each time the given predicate returns true. Note that the element that triggers the predicate to return true (and thus closes a buffer) is included as last element in the emitted buffer.

On completion, if the latest buffer is non-empty and has not been closed it is emitted. However, such a "partial" buffer isn't emitted in case of onError termination.

Params:
  • predicate – a predicate that triggers the next buffer when it becomes true.
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal.
Returns:a microbatched Flux of List
/** * Collect incoming values into multiple {@link List} buffers that will be emitted by * the resulting {@link Flux} each time the given predicate returns true. Note that * the element that triggers the predicate to return true (and thus closes a buffer) * is included as last element in the emitted buffer. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffersize.png" * alt=""> * <p> * On completion, if the latest buffer is non-empty and has not been closed it is * emitted. However, such a "partial" buffer isn't emitted in case of onError * termination. * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal. * * @param predicate a predicate that triggers the next buffer when it becomes true. * @return a microbatched {@link Flux} of {@link List} */
public final Flux<List<T>> bufferUntil(Predicate<? super T> predicate) { return onAssembly(new FluxBufferPredicate<>(this, predicate, listSupplier(), FluxBufferPredicate.Mode.UNTIL)); }
Collect incoming values into multiple List buffers that will be emitted by the resulting Flux each time the given predicate returns true. Note that the buffer into which the element that triggers the predicate to return true (and thus closes a buffer) is included depends on the cutBefore parameter: set it to true to include the boundary element in the newly opened buffer, false to include it in the closed buffer (as in bufferUntil(Predicate)).

On completion, if the latest buffer is non-empty and has not been closed it is emitted. However, such a "partial" buffer isn't emitted in case of onError termination.

Params:
  • predicate – a predicate that triggers the next buffer when it becomes true.
  • cutBefore – set to true to include the triggering element in the new buffer rather than the old.
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal.
Returns:a microbatched Flux of List
/** * Collect incoming values into multiple {@link List} buffers that will be emitted by * the resulting {@link Flux} each time the given predicate returns true. Note that * the buffer into which the element that triggers the predicate to return true * (and thus closes a buffer) is included depends on the {@code cutBefore} parameter: * set it to true to include the boundary element in the newly opened buffer, false to * include it in the closed buffer (as in {@link #bufferUntil(Predicate)}). * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffersize.png" * alt=""> * <p> * On completion, if the latest buffer is non-empty and has not been closed it is * emitted. However, such a "partial" buffer isn't emitted in case of onError * termination. * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal. * * @param predicate a predicate that triggers the next buffer when it becomes true. * @param cutBefore set to true to include the triggering element in the new buffer rather than the old. * @return a microbatched {@link Flux} of {@link List} */
public final Flux<List<T>> bufferUntil(Predicate<? super T> predicate, boolean cutBefore) { return onAssembly(new FluxBufferPredicate<>(this, predicate, listSupplier(), cutBefore ? FluxBufferPredicate.Mode.UNTIL_CUT_BEFORE : FluxBufferPredicate.Mode.UNTIL)); }
Collect incoming values into multiple List buffers that will be emitted by the resulting Flux. Each buffer continues aggregating values while the given predicate returns true, and a new buffer is created as soon as the predicate returns false... Note that the element that triggers the predicate to return false (and thus closes a buffer) is NOT included in any emitted buffer.

On completion, if the latest buffer is non-empty and has not been closed it is emitted. However, such a "partial" buffer isn't emitted in case of onError termination.

Params:
  • predicate – a predicate that triggers the next buffer when it becomes false.
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal, as well as the buffer-triggering element.
Returns:a microbatched Flux of List
/** * Collect incoming values into multiple {@link List} buffers that will be emitted by * the resulting {@link Flux}. Each buffer continues aggregating values while the * given predicate returns true, and a new buffer is created as soon as the * predicate returns false... Note that the element that triggers the predicate * to return false (and thus closes a buffer) is NOT included in any emitted buffer. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/buffersize.png" * alt=""> * <p> * On completion, if the latest buffer is non-empty and has not been closed it is * emitted. However, such a "partial" buffer isn't emitted in case of onError * termination. * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal, * as well as the buffer-triggering element. * * @param predicate a predicate that triggers the next buffer when it becomes false. * @return a microbatched {@link Flux} of {@link List} */
public final Flux<List<T>> bufferWhile(Predicate<? super T> predicate) { return onAssembly(new FluxBufferPredicate<>(this, predicate, listSupplier(), FluxBufferPredicate.Mode.WHILE)); }
Collect incoming values into multiple List buffers started each time an opening companion Publisher emits. Each buffer will last until the corresponding closing companion Publisher emits, thus releasing the buffer to the resulting Flux.

When Open signal is strictly not overlapping Close signal : dropping buffers

When Open signal is strictly more frequent than Close signal : overlapping buffers

When Open signal is exactly coordinated with Close signal : exact buffers

Params:
  • bucketOpening – a companion Publisher to subscribe for buffer creation signals.
  • closeSelector – a factory that, given a buffer opening signal, returns a companion Publisher to subscribe to for buffer closure and emission signals.
Type parameters:
  • <U> – the element type of the buffer-opening sequence
  • <V> – the element type of the buffer-closing sequence
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal. It DOES NOT provide strong guarantees in the case of overlapping buffers, as elements might get discarded too early (from the first of two overlapping buffers for instance).
Returns:a microbatched Flux of List delimited by an opening Publisher and a relative closing Publisher
/** * Collect incoming values into multiple {@link List} buffers started each time an opening * companion {@link Publisher} emits. Each buffer will last until the corresponding * closing companion {@link Publisher} emits, thus releasing the buffer to the resulting {@link Flux}. * <p> * When Open signal is strictly not overlapping Close signal : dropping buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/bufferopenclose.png" * alt=""> * <p> * When Open signal is strictly more frequent than Close signal : overlapping buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/bufferopencloseover.png" * alt=""> * <p> * When Open signal is exactly coordinated with Close signal : exact buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/bufferboundary.png" * alt=""> * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal. * It DOES NOT provide strong guarantees in the case of overlapping buffers, as elements * might get discarded too early (from the first of two overlapping buffers for instance). * * @param bucketOpening a companion {@link Publisher} to subscribe for buffer creation signals. * @param closeSelector a factory that, given a buffer opening signal, returns a companion * {@link Publisher} to subscribe to for buffer closure and emission signals. * @param <U> the element type of the buffer-opening sequence * @param <V> the element type of the buffer-closing sequence * * @return a microbatched {@link Flux} of {@link List} delimited by an opening {@link Publisher} and a relative * closing {@link Publisher} */
public final <U, V> Flux<List<T>> bufferWhen(Publisher<U> bucketOpening, Function<? super U, ? extends Publisher<V>> closeSelector) { return bufferWhen(bucketOpening, closeSelector, listSupplier()); }
Collect incoming values into multiple user-defined Collection buffers started each time an opening companion Publisher emits. Each buffer will last until the corresponding closing companion Publisher emits, thus releasing the buffer to the resulting Flux.

When Open signal is strictly not overlapping Close signal : dropping buffers

When Open signal is strictly more frequent than Close signal : overlapping buffers

When Open signal is exactly coordinated with Close signal : exact buffers

Params:
  • bucketOpening – a companion Publisher to subscribe for buffer creation signals.
  • closeSelector – a factory that, given a buffer opening signal, returns a companion Publisher to subscribe to for buffer closure and emission signals.
  • bufferSupplier – a Supplier of the concrete Collection to use for each buffer
Type parameters:
  • <U> – the element type of the buffer-opening sequence
  • <V> – the element type of the buffer-closing sequence
  • <C> – the Collection buffer type
@reactor.discardThis operator discards the currently open buffer upon cancellation or error triggered by a data signal. It DOES NOT provide strong guarantees in the case of overlapping buffers, as elements might get discarded too early (from the first of two overlapping buffers for instance).
Returns:a microbatched Flux of Collection delimited by an opening Publisher and a relative closing Publisher
/** * Collect incoming values into multiple user-defined {@link Collection} buffers started each time an opening * companion {@link Publisher} emits. Each buffer will last until the corresponding * closing companion {@link Publisher} emits, thus releasing the buffer to the resulting {@link Flux}. * <p> * When Open signal is strictly not overlapping Close signal : dropping buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/bufferopenclose.png" * alt=""> * <p> * When Open signal is strictly more frequent than Close signal : overlapping buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/bufferopencloseover.png" * alt=""> * <p> * When Open signal is exactly coordinated with Close signal : exact buffers * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/bufferboundary.png" * alt=""> * * @reactor.discard This operator discards the currently open buffer upon cancellation or error triggered by a data signal. * It DOES NOT provide strong guarantees in the case of overlapping buffers, as elements * might get discarded too early (from the first of two overlapping buffers for instance). * * @param bucketOpening a companion {@link Publisher} to subscribe for buffer creation signals. * @param closeSelector a factory that, given a buffer opening signal, returns a companion * {@link Publisher} to subscribe to for buffer closure and emission signals. * @param bufferSupplier a {@link Supplier} of the concrete {@link Collection} to use for each buffer * @param <U> the element type of the buffer-opening sequence * @param <V> the element type of the buffer-closing sequence * @param <C> the {@link Collection} buffer type * * @return a microbatched {@link Flux} of {@link Collection} delimited by an opening {@link Publisher} and a relative * closing {@link Publisher} */
public final <U, V, C extends Collection<? super T>> Flux<C> bufferWhen(Publisher<U> bucketOpening, Function<? super U, ? extends Publisher<V>> closeSelector, Supplier<C> bufferSupplier) { return onAssembly(new FluxBufferWhen<>(this, bucketOpening, closeSelector, bufferSupplier, Queues.unbounded(Queues.XS_BUFFER_SIZE))); }
Turn this Flux into a hot source and cache last emitted signals for further Subscriber. Will retain an unbounded volume of onNext signals. Completion and Error will also be replayed.

Returns:a replaying Flux
/** * Turn this {@link Flux} into a hot source and cache last emitted signals for further {@link Subscriber}. Will * retain an unbounded volume of onNext signals. Completion and Error will also be * replayed. * <p> * <img width="500" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/cache.png" * alt=""> * * @return a replaying {@link Flux} */
public final Flux<T> cache() { return cache(Integer.MAX_VALUE); }
Turn this Flux into a hot source and cache last emitted signals for further Subscriber. Will retain up to the given history size onNext signals. Completion and Error will also be replayed.

Note that cache(0) will only cache the terminal signal without expiration.

Params:
  • history – number of elements retained in cache
Returns:a replaying Flux
/** * Turn this {@link Flux} into a hot source and cache last emitted signals for further {@link Subscriber}. * Will retain up to the given history size onNext signals. Completion and Error will also be * replayed. * <p> * Note that {@code cache(0)} will only cache the terminal signal without * expiration. * <p> * <img width="500" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/cache.png" alt=""> * * @param history number of elements retained in cache * * @return a replaying {@link Flux} * */
public final Flux<T> cache(int history) { return replay(history).autoConnect(); }
Turn this Flux into a hot source and cache last emitted signals for further Subscriber. Will retain an unbounded history but apply a per-item expiry timeout

Completion and Error will also be replayed until ttl triggers in which case the next Subscriber will start over a new subscription.

Params:
  • ttl – Time-to-live for each cached item and post termination.
Returns:a replaying Flux
/** * Turn this {@link Flux} into a hot source and cache last emitted signals for further * {@link Subscriber}. Will retain an unbounded history but apply a per-item expiry timeout * <p> * Completion and Error will also be replayed until {@code ttl} triggers in which case * the next {@link Subscriber} will start over a new subscription. * <p> * <img width="500" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/cache.png" * alt=""> * * @param ttl Time-to-live for each cached item and post termination. * * @return a replaying {@link Flux} */
public final Flux<T> cache(Duration ttl) { return cache(ttl, Schedulers.parallel()); }
Turn this Flux into a hot source and cache last emitted signals for further Subscriber. Will retain an unbounded history but apply a per-item expiry timeout

Completion and Error will also be replayed until ttl triggers in which case the next Subscriber will start over a new subscription.

Params:
  • ttl – Time-to-live for each cached item and post termination.
  • timer – the Scheduler on which to measure the duration.
Returns:a replaying Flux
/** * Turn this {@link Flux} into a hot source and cache last emitted signals for further * {@link Subscriber}. Will retain an unbounded history but apply a per-item expiry timeout * <p> * Completion and Error will also be replayed until {@code ttl} triggers in which case * the next {@link Subscriber} will start over a new subscription. * <p> * <img width="500" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/cache.png" * alt=""> * * @param ttl Time-to-live for each cached item and post termination. * @param timer the {@link Scheduler} on which to measure the duration. * * @return a replaying {@link Flux} */
public final Flux<T> cache(Duration ttl, Scheduler timer) { return cache(Integer.MAX_VALUE, ttl, timer); }
Turn this Flux into a hot source and cache last emitted signals for further Subscriber. Will retain up to the given history size and apply a per-item expiry timeout.

Completion and Error will also be replayed until ttl triggers in which case the next Subscriber will start over a new subscription.

Params:
  • history – number of elements retained in cache
  • ttl – Time-to-live for each cached item and post termination.
Returns:a replaying Flux
/** * Turn this {@link Flux} into a hot source and cache last emitted signals for further * {@link Subscriber}. Will retain up to the given history size and apply a per-item expiry * timeout. * <p> * Completion and Error will also be replayed until {@code ttl} triggers in which case * the next {@link Subscriber} will start over a new subscription. * <p> * <img width="500" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/cache.png" * alt=""> * * @param history number of elements retained in cache * @param ttl Time-to-live for each cached item and post termination. * * @return a replaying {@link Flux} */
public final Flux<T> cache(int history, Duration ttl) { return cache(history, ttl, Schedulers.parallel()); }
Turn this Flux into a hot source and cache last emitted signals for further Subscriber. Will retain up to the given history size and apply a per-item expiry timeout.

Completion and Error will also be replayed until ttl triggers in which case the next Subscriber will start over a new subscription.

Params:
  • history – number of elements retained in cache
  • ttl – Time-to-live for each cached item and post termination.
  • timer – the Scheduler on which to measure the duration.
Returns:a replaying Flux
/** * Turn this {@link Flux} into a hot source and cache last emitted signals for further * {@link Subscriber}. Will retain up to the given history size and apply a per-item expiry * timeout. * <p> * Completion and Error will also be replayed until {@code ttl} triggers in which case * the next {@link Subscriber} will start over a new subscription. * <p> * <img width="500" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/cache.png" * alt=""> * * @param history number of elements retained in cache * @param ttl Time-to-live for each cached item and post termination. * @param timer the {@link Scheduler} on which to measure the duration. * * @return a replaying {@link Flux} */
public final Flux<T> cache(int history, Duration ttl, Scheduler timer) { return replay(history, ttl, timer).autoConnect(); }
Cast the current Flux produced type into a target produced type.

Params:
  • clazz – the target class to cast to
Type parameters:
  • <E> – the Flux output type
Returns:a casted Flux
/** * Cast the current {@link Flux} produced type into a target produced type. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/cast.png" alt=""> * * @param <E> the {@link Flux} output type * @param clazz the target class to cast to * * @return a casted {@link Flux} */
public final <E> Flux<E> cast(Class<E> clazz) { Objects.requireNonNull(clazz, "clazz"); return map(clazz::cast); }
Prepare this Flux so that subscribers will cancel from it on a specified Scheduler.
Params:
  • scheduler – the Scheduler to signal cancel on
Returns:a scheduled cancel Flux
/** * Prepare this {@link Flux} so that subscribers will cancel from it on a * specified * {@link Scheduler}. * * @param scheduler the {@link Scheduler} to signal cancel on * * @return a scheduled cancel {@link Flux} */
public final Flux<T> cancelOn(Scheduler scheduler) { return onAssembly(new FluxCancelOn<>(this, scheduler)); }
Activate assembly tracing for this particular Flux, in case of an error upstream of the checkpoint. Tracing incurs the cost of an exception stack trace creation.

It should be placed towards the end of the reactive chain, as errors triggered downstream of it cannot be observed and augmented with assembly trace.

Returns:the assembly tracing Flux.
/** * Activate assembly tracing for this particular {@link Flux}, in case of an error * upstream of the checkpoint. Tracing incurs the cost of an exception stack trace * creation. * <p> * It should be placed towards the end of the reactive chain, as errors * triggered downstream of it cannot be observed and augmented with assembly trace. * * @return the assembly tracing {@link Flux}. */
public final Flux<T> checkpoint() { return checkpoint(null, true); }
Activate assembly marker for this particular Flux by giving it a description that will be reflected in the assembly traceback in case of an error upstream of the checkpoint. Note that unlike checkpoint(), this doesn't create a filled stack trace, avoiding the main cost of the operator. However, as a trade-off the description must be unique enough for the user to find out where this Flux was assembled. If you only want a generic description, and still rely on the stack trace to find the assembly site, use the checkpoint(String, boolean) variant.

It should be placed towards the end of the reactive chain, as errors triggered downstream of it cannot be observed and augmented with assembly trace.

Params:
  • description – a unique enough description to include in the light assembly traceback.
Returns:the assembly marked Flux
/** * Activate assembly marker for this particular {@link Flux} by giving it a description that * will be reflected in the assembly traceback in case of an error upstream of the * checkpoint. Note that unlike {@link #checkpoint()}, this doesn't create a * filled stack trace, avoiding the main cost of the operator. * However, as a trade-off the description must be unique enough for the user to find * out where this Flux was assembled. If you only want a generic description, and * still rely on the stack trace to find the assembly site, use the * {@link #checkpoint(String, boolean)} variant. * <p> * It should be placed towards the end of the reactive chain, as errors * triggered downstream of it cannot be observed and augmented with assembly trace. * * @param description a unique enough description to include in the light assembly traceback. * @return the assembly marked {@link Flux} */
public final Flux<T> checkpoint(String description) { return checkpoint(Objects.requireNonNull(description), false); }
Activate assembly tracing or the lighter assembly marking depending on the forceStackTrace option.

By setting the forceStackTrace parameter to true, activate assembly tracing for this particular Flux and give it a description that will be reflected in the assembly traceback in case of an error upstream of the checkpoint. Note that unlike checkpoint(String), this will incur the cost of an exception stack trace creation. The description could for example be a meaningful name for the assembled flux or a wider correlation ID, since the stack trace will always provide enough information to locate where this Flux was assembled.

By setting forceStackTrace to false, behaves like checkpoint(String) and is subject to the same caveat in choosing the description.

It should be placed towards the end of the reactive chain, as errors triggered downstream of it cannot be observed and augmented with assembly marker.

Params:
  • description – a description (must be unique enough if forceStackTrace is set to false).
  • forceStackTrace – false to make a light checkpoint without a stacktrace, true to use a stack trace.
Returns:the assembly marked Flux.
/** * Activate assembly tracing or the lighter assembly marking depending on the * {@code forceStackTrace} option. * <p> * By setting the {@code forceStackTrace} parameter to {@literal true}, activate assembly * tracing for this particular {@link Flux} and give it a description that * will be reflected in the assembly traceback in case of an error upstream of the * checkpoint. Note that unlike {@link #checkpoint(String)}, this will incur * the cost of an exception stack trace creation. The description could for * example be a meaningful name for the assembled flux or a wider correlation ID, * since the stack trace will always provide enough information to locate where this * Flux was assembled. * <p> * By setting {@code forceStackTrace} to {@literal false}, behaves like * {@link #checkpoint(String)} and is subject to the same caveat in choosing the * description. * <p> * It should be placed towards the end of the reactive chain, as errors * triggered downstream of it cannot be observed and augmented with assembly marker. * * @param description a description (must be unique enough if forceStackTrace is set * to false). * @param forceStackTrace false to make a light checkpoint without a stacktrace, true * to use a stack trace. * @return the assembly marked {@link Flux}. */
public final Flux<T> checkpoint(@Nullable String description, boolean forceStackTrace) { return new FluxOnAssembly<>(this, description, !forceStackTrace); }
Collect all elements emitted by this Flux into a user-defined container, by applying a collector BiConsumer taking the container and each element. The collected result will be emitted when this sequence completes.

Params:
  • containerSupplier – the supplier of the container instance for each Subscriber
  • collector – a consumer of both the container instance and the value being currently collected
Type parameters:
  • <E> – the container type
Returns:a Mono of the collected container on complete
/** * Collect all elements emitted by this {@link Flux} into a user-defined container, * by applying a collector {@link BiConsumer} taking the container and each element. * The collected result will be emitted when this sequence completes. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/collect.png" alt=""> * * @param <E> the container type * @param containerSupplier the supplier of the container instance for each Subscriber * @param collector a consumer of both the container instance and the value being currently collected * * @return a {@link Mono} of the collected container on complete * */
public final <E> Mono<E> collect(Supplier<E> containerSupplier, BiConsumer<E, ? super T> collector) { return Mono.onAssembly(new MonoCollect<>(this, containerSupplier, collector)); }
Collect all elements emitted by this Flux into a container, by applying a Java 8 Stream API Collector The collected result will be emitted when this sequence completes.

Params:
Type parameters:
  • <A> – The mutable accumulation type
  • <R> – the container type
Returns:a Mono of the collected container on complete
/** * Collect all elements emitted by this {@link Flux} into a container, * by applying a Java 8 Stream API {@link Collector} * The collected result will be emitted when this sequence completes. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/collect.png" alt=""> * * @param collector the {@link Collector} * @param <A> The mutable accumulation type * @param <R> the container type * * @return a {@link Mono} of the collected container on complete * */
public final <R, A> Mono<R> collect(Collector<? super T, A, ? extends R> collector) { return Mono.onAssembly(new MonoStreamCollector<>(this, collector)); }
Collect all elements emitted by this Flux into a List that is emitted by the resulting Mono when this sequence completes.

Returns:a Mono of a List of all values from this Flux
/** * Collect all elements emitted by this {@link Flux} into a {@link List} that is * emitted by the resulting {@link Mono} when this sequence completes. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/collectlist.png" alt=""> * * @return a {@link Mono} of a {@link List} of all values from this {@link Flux} */
public final Mono<List<T>> collectList() { if (this instanceof Callable) { if (this instanceof Fuseable.ScalarCallable) { @SuppressWarnings("unchecked") Fuseable.ScalarCallable<T> scalarCallable = (Fuseable.ScalarCallable<T>) this; T v; try { v = scalarCallable.call(); } catch (Exception e) { return Mono.error(e); } if (v == null) { return Mono.onAssembly(new MonoSupplier<>(listSupplier())); } return Mono.just(v).map(u -> { List<T> list = Flux.<T>listSupplier().get(); list.add(u); return list; }); } @SuppressWarnings("unchecked") Callable<T> thiz = (Callable<T>)this; return Mono.onAssembly(new MonoCallable<>(thiz).map(u -> { List<T> list = Flux.<T>listSupplier().get(); list.add(u); return list; })); } return Mono.onAssembly(new MonoCollectList<>(this, listSupplier())); }
Collect all elements emitted by this Flux into a hashed Map that is emitted by the resulting Mono when this sequence completes. The key is extracted from each element by applying the keyExtractor Function. In case several elements map to the same key, the associated value will be the most recently emitted element.

Params:
  • keyExtractor – a Function to map elements to a key for the Map
Type parameters:
  • <K> – the type of the key extracted from each source element
Returns:a Mono of a Map of key-element pairs (only including latest element in case of key conflicts)
/** * Collect all elements emitted by this {@link Flux} into a hashed {@link Map} that is * emitted by the resulting {@link Mono} when this sequence completes. * The key is extracted from each element by applying the {@code keyExtractor} * {@link Function}. In case several elements map to the same key, the associated value * will be the most recently emitted element. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/collectmap.png" alt=""> * * @param keyExtractor a {@link Function} to map elements to a key for the {@link Map} * @param <K> the type of the key extracted from each source element * * @return a {@link Mono} of a {@link Map} of key-element pairs (only including latest * element in case of key conflicts) * */
public final <K> Mono<Map<K, T>> collectMap(Function<? super T, ? extends K> keyExtractor) { return collectMap(keyExtractor, identityFunction()); }
Collect all elements emitted by this Flux into a hashed Map that is emitted by the resulting Mono when this sequence completes. The key is extracted from each element by applying the keyExtractor Function, and the value is extracted by the valueExtractor Function. In case several elements map to the same key, the associated value will be derived from the most recently emitted element.

Params:
  • keyExtractor – a Function to map elements to a key for the Map
  • valueExtractor – a Function to map elements to a value for the Map
Type parameters:
  • <K> – the type of the key extracted from each source element
  • <V> – the type of the value extracted from each source element
Returns:a Mono of a Map of key-element pairs (only including latest element's value in case of key conflicts)
/** * Collect all elements emitted by this {@link Flux} into a hashed {@link Map} that is * emitted by the resulting {@link Mono} when this sequence completes. * The key is extracted from each element by applying the {@code keyExtractor} * {@link Function}, and the value is extracted by the {@code valueExtractor} Function. * In case several elements map to the same key, the associated value will be derived * from the most recently emitted element. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/collectmap.png" alt=""> * * @param keyExtractor a {@link Function} to map elements to a key for the {@link Map} * @param valueExtractor a {@link Function} to map elements to a value for the {@link Map} * * @param <K> the type of the key extracted from each source element * @param <V> the type of the value extracted from each source element * * @return a {@link Mono} of a {@link Map} of key-element pairs (only including latest * element's value in case of key conflicts) */
public final <K, V> Mono<Map<K, V>> collectMap(Function<? super T, ? extends K> keyExtractor, Function<? super T, ? extends V> valueExtractor) { return collectMap(keyExtractor, valueExtractor, () -> new HashMap<>()); }
Collect all elements emitted by this Flux into a user-defined Map that is emitted by the resulting Mono when this sequence completes. The key is extracted from each element by applying the keyExtractor Function, and the value is extracted by the valueExtractor Function. In case several elements map to the same key, the associated value will be derived from the most recently emitted element.

Params:
  • keyExtractor – a Function to map elements to a key for the Map
  • valueExtractor – a Function to map elements to a value for the Map
  • mapSupplier – a Map factory called for each Subscriber
Type parameters:
  • <K> – the type of the key extracted from each source element
  • <V> – the type of the value extracted from each source element
Returns:a Mono of a Map of key-value pairs (only including latest element's value in case of key conflicts)
/** * Collect all elements emitted by this {@link Flux} into a user-defined {@link Map} that is * emitted by the resulting {@link Mono} when this sequence completes. * The key is extracted from each element by applying the {@code keyExtractor} * {@link Function}, and the value is extracted by the {@code valueExtractor} Function. * In case several elements map to the same key, the associated value will be derived * from the most recently emitted element. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/collectmap.png" alt=""> * * @param keyExtractor a {@link Function} to map elements to a key for the {@link Map} * @param valueExtractor a {@link Function} to map elements to a value for the {@link Map} * @param mapSupplier a {@link Map} factory called for each {@link Subscriber} * * @param <K> the type of the key extracted from each source element * @param <V> the type of the value extracted from each source element * * @return a {@link Mono} of a {@link Map} of key-value pairs (only including latest * element's value in case of key conflicts) */
public final <K, V> Mono<Map<K, V>> collectMap( final Function<? super T, ? extends K> keyExtractor, final Function<? super T, ? extends V> valueExtractor, Supplier<Map<K, V>> mapSupplier) { Objects.requireNonNull(keyExtractor, "Key extractor is null"); Objects.requireNonNull(valueExtractor, "Value extractor is null"); Objects.requireNonNull(mapSupplier, "Map supplier is null"); return collect(mapSupplier, (m, d) -> m.put(keyExtractor.apply(d), valueExtractor.apply(d))); }
Collect all elements emitted by this Flux into a multimap that is emitted by the resulting Mono when this sequence completes. The key is extracted from each element by applying the keyExtractor Function, and every element mapping to the same key is stored in the List associated to said key.

Params:
  • keyExtractor – a Function to map elements to a key for the Map
Type parameters:
  • <K> – the type of the key extracted from each source element
Returns:a Mono of a Map of key-List(elements) pairs
/** * Collect all elements emitted by this {@link Flux} into a {@link Map multimap} that is * emitted by the resulting {@link Mono} when this sequence completes. * The key is extracted from each element by applying the {@code keyExtractor} * {@link Function}, and every element mapping to the same key is stored in the {@link List} * associated to said key. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/collectmultimap.png" alt=""> * * @param keyExtractor a {@link Function} to map elements to a key for the {@link Map} * * @param <K> the type of the key extracted from each source element * @return a {@link Mono} of a {@link Map} of key-List(elements) pairs */
public final <K> Mono<Map<K, Collection<T>>> collectMultimap(Function<? super T, ? extends K> keyExtractor) { return collectMultimap(keyExtractor, identityFunction()); }
Collect all elements emitted by this Flux into a multimap that is emitted by the resulting Mono when this sequence completes. The key is extracted from each element by applying the keyExtractor Function, and every element mapping to the same key is converted by the valueExtractor Function to a value stored in the List associated to said key.

Params:
  • keyExtractor – a Function to map elements to a key for the Map
  • valueExtractor – a Function to map elements to a value for the Map
Type parameters:
  • <K> – the type of the key extracted from each source element
  • <V> – the type of the value extracted from each source element
Returns:a Mono of a Map of key-List(values) pairs
/** * Collect all elements emitted by this {@link Flux} into a {@link Map multimap} that is * emitted by the resulting {@link Mono} when this sequence completes. * The key is extracted from each element by applying the {@code keyExtractor} * {@link Function}, and every element mapping to the same key is converted by the * {@code valueExtractor} Function to a value stored in the {@link List} associated to * said key. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/collectmultimap.png" alt=""> * * @param keyExtractor a {@link Function} to map elements to a key for the {@link Map} * @param valueExtractor a {@link Function} to map elements to a value for the {@link Map} * * @param <K> the type of the key extracted from each source element * @param <V> the type of the value extracted from each source element * * @return a {@link Mono} of a {@link Map} of key-List(values) pairs */
public final <K, V> Mono<Map<K, Collection<V>>> collectMultimap(Function<? super T, ? extends K> keyExtractor, Function<? super T, ? extends V> valueExtractor) { return collectMultimap(keyExtractor, valueExtractor, () -> new HashMap<>()); }
Collect all elements emitted by this Flux into a user-defined multimap that is emitted by the resulting Mono when this sequence completes. The key is extracted from each element by applying the keyExtractor Function, and every element mapping to the same key is converted by the valueExtractor Function to a value stored in the Collection associated to said key.

Params:
Type parameters:
  • <K> – the type of the key extracted from each source element
  • <V> – the type of the value extracted from each source element
Returns:a Mono of a Map of key-Collection(values) pairs
/** * Collect all elements emitted by this {@link Flux} into a user-defined {@link Map multimap} that is * emitted by the resulting {@link Mono} when this sequence completes. * The key is extracted from each element by applying the {@code keyExtractor} * {@link Function}, and every element mapping to the same key is converted by the * {@code valueExtractor} Function to a value stored in the {@link Collection} associated to * said key. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/collectmultimap.png" alt=""> * * @param keyExtractor a {@link Function} to map elements to a key for the {@link Map} * @param valueExtractor a {@link Function} to map elements to a value for the {@link Map} * @param mapSupplier a multimap ({@link Map} of {@link Collection}) factory called * for each {@link Subscriber} * * @param <K> the type of the key extracted from each source element * @param <V> the type of the value extracted from each source element * * @return a {@link Mono} of a {@link Map} of key-Collection(values) pairs * */
public final <K, V> Mono<Map<K, Collection<V>>> collectMultimap( final Function<? super T, ? extends K> keyExtractor, final Function<? super T, ? extends V> valueExtractor, Supplier<Map<K, Collection<V>>> mapSupplier) { Objects.requireNonNull(keyExtractor, "Key extractor is null"); Objects.requireNonNull(valueExtractor, "Value extractor is null"); Objects.requireNonNull(mapSupplier, "Map supplier is null"); return collect(mapSupplier, (m, d) -> { K key = keyExtractor.apply(d); Collection<V> values = m.computeIfAbsent(key, k -> new ArrayList<>()); values.add(valueExtractor.apply(d)); }); }
Collect all elements emitted by this Flux until this sequence completes, and then sort them in natural order into a List that is emitted by the resulting Mono.

Returns:a Mono of a sorted List of all values from this Flux, in natural order
/** * Collect all elements emitted by this {@link Flux} until this sequence completes, * and then sort them in natural order into a {@link List} that is emitted by the * resulting {@link Mono}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/collectsortedlist.png" alt=""> * * @return a {@link Mono} of a sorted {@link List} of all values from this {@link Flux}, in natural order */
public final Mono<List<T>> collectSortedList() { return collectSortedList(null); }
Collect all elements emitted by this Flux until this sequence completes, and then sort them using a Comparator into a List that is emitted by the resulting Mono.

Params:
  • comparator – a Comparator to sort the items of this sequences
Returns:a Mono of a sorted List of all values from this Flux
/** * Collect all elements emitted by this {@link Flux} until this sequence completes, * and then sort them using a {@link Comparator} into a {@link List} that is emitted * by the resulting {@link Mono}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/collectsortedlist.png" alt=""> * * @param comparator a {@link Comparator} to sort the items of this sequences * * @return a {@link Mono} of a sorted {@link List} of all values from this {@link Flux} */
@SuppressWarnings({ "unchecked", "rawtypes" }) public final Mono<List<T>> collectSortedList(@Nullable Comparator<? super T> comparator) { return collectList().map(list -> { // Note: this assumes the list emitted by buffer() is mutable if (comparator != null) { list.sort(comparator); } else { List<Comparable> l = (List<Comparable>)list; Collections.sort(l); } return list; }); }
Defer the transformation of this Flux in order to generate a target Flux type. A transformation will occur for each Subscriber. For instance:
flux.compose(Mono::from).subscribe()  
Params:
  • transformer – the Function to lazily map this Flux into a target Publisher instance for each new subscriber
Type parameters:
See Also:
Returns:a new Flux
/** * Defer the transformation of this {@link Flux} in order to generate a target {@link Flux} type. * A transformation will occur for each {@link Subscriber}. For instance: * <blockquote><pre> * {@code flux.compose(Mono::from).subscribe() } * </pre></blockquote> * * @param transformer the {@link Function} to lazily map this {@link Flux} into a target {@link Publisher} * instance for each new subscriber * @param <V> the item type in the returned {@link Publisher} * * @return a new {@link Flux} * @see #transform transform() for immmediate transformation of {@link Flux} * @see #as as() for a loose conversion to an arbitrary type */
public final <V> Flux<V> compose(Function<? super Flux<T>, ? extends Publisher<V>> transformer) { return defer(() -> transformer.apply(this)); }
Transform the elements emitted by this Flux asynchronously into Publishers, then flatten these inner publishers into a single Flux, sequentially and preserving order using concatenation.

There are three dimensions to this operator that can be compared with flatMap and flatMapSequential:

  • Generation of inners and subscription: this operator waits for one inner to complete before generating the next one and subscribing to it.
  • Ordering of the flattened values: this operator naturally preserves the same order as the source elements, concatenating the inners from each source element sequentially.
  • Interleaving: this operator does not let values from different inners interleave (concatenation).

Errors will immediately short circuit current concat backlog.

Params:
  • mapper – the function to transform this sequence of T into concatenated sequences of V
Type parameters:
  • <V> – the produced concatenated type
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation.
Returns:a concatenated Flux
/** * Transform the elements emitted by this {@link Flux} asynchronously into Publishers, * then flatten these inner publishers into a single {@link Flux}, sequentially and * preserving order using concatenation. * <p> * There are three dimensions to this operator that can be compared with * {@link #flatMap(Function) flatMap} and {@link #flatMapSequential(Function) flatMapSequential}: * <ul> * <li><b>Generation of inners and subscription</b>: this operator waits for one * inner to complete before generating the next one and subscribing to it.</li> * <li><b>Ordering of the flattened values</b>: this operator naturally preserves * the same order as the source elements, concatenating the inners from each source * element sequentially.</li> * <li><b>Interleaving</b>: this operator does not let values from different inners * interleave (concatenation).</li> * </ul> * * <p> * Errors will immediately short circuit current concat backlog. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concatmap.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation. * * @param mapper the function to transform this sequence of T into concatenated sequences of V * @param <V> the produced concatenated type * * @return a concatenated {@link Flux} */
public final <V> Flux<V> concatMap(Function<? super T, ? extends Publisher<? extends V>> mapper) { return concatMap(mapper, Queues.XS_BUFFER_SIZE); }
Transform the elements emitted by this Flux asynchronously into Publishers, then flatten these inner publishers into a single Flux, sequentially and preserving order using concatenation.

There are three dimensions to this operator that can be compared with flatMap and flatMapSequential:

  • Generation of inners and subscription: this operator waits for one inner to complete before generating the next one and subscribing to it.
  • Ordering of the flattened values: this operator naturally preserves the same order as the source elements, concatenating the inners from each source element sequentially.
  • Interleaving: this operator does not let values from different inners interleave (concatenation).

Errors will immediately short circuit current concat backlog. The prefetch argument allows to give an arbitrary prefetch size to the inner Publisher.

Params:
  • mapper – the function to transform this sequence of T into concatenated sequences of V
  • prefetch – the inner source produced demand
Type parameters:
  • <V> – the produced concatenated type
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation.
Returns:a concatenated Flux
/** * Transform the elements emitted by this {@link Flux} asynchronously into Publishers, * then flatten these inner publishers into a single {@link Flux}, sequentially and * preserving order using concatenation. * <p> * There are three dimensions to this operator that can be compared with * {@link #flatMap(Function) flatMap} and {@link #flatMapSequential(Function) flatMapSequential}: * <ul> * <li><b>Generation of inners and subscription</b>: this operator waits for one * inner to complete before generating the next one and subscribing to it.</li> * <li><b>Ordering of the flattened values</b>: this operator naturally preserves * the same order as the source elements, concatenating the inners from each source * element sequentially.</li> * <li><b>Interleaving</b>: this operator does not let values from different inners * interleave (concatenation).</li> * </ul> * * <p> * Errors will immediately short circuit current concat backlog. The prefetch argument * allows to give an arbitrary prefetch size to the inner {@link Publisher}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concatmap.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation. * * @param mapper the function to transform this sequence of T into concatenated sequences of V * @param prefetch the inner source produced demand * @param <V> the produced concatenated type * * @return a concatenated {@link Flux} */
public final <V> Flux<V> concatMap(Function<? super T, ? extends Publisher<? extends V>> mapper, int prefetch) { return onAssembly(new FluxConcatMap<>(this, mapper, Queues.get(prefetch), prefetch, FluxConcatMap.ErrorMode.IMMEDIATE)); }
Transform the elements emitted by this Flux asynchronously into Publishers, then flatten these inner publishers into a single Flux, sequentially and preserving order using concatenation.

There are three dimensions to this operator that can be compared with flatMap and flatMapSequential:

  • Generation of inners and subscription: this operator waits for one inner to complete before generating the next one and subscribing to it.
  • Ordering of the flattened values: this operator naturally preserves the same order as the source elements, concatenating the inners from each source element sequentially.
  • Interleaving: this operator does not let values from different inners interleave (concatenation).

Errors in the individual publishers will be delayed at the end of the whole concat sequence (possibly getting combined into a composite if several sources error.

Params:
  • mapper – the function to transform this sequence of T into concatenated sequences of V
Type parameters:
  • <V> – the produced concatenated type
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation.
Returns:a concatenated Flux
/** * Transform the elements emitted by this {@link Flux} asynchronously into Publishers, * then flatten these inner publishers into a single {@link Flux}, sequentially and * preserving order using concatenation. * <p> * There are three dimensions to this operator that can be compared with * {@link #flatMap(Function) flatMap} and {@link #flatMapSequential(Function) flatMapSequential}: * <ul> * <li><b>Generation of inners and subscription</b>: this operator waits for one * inner to complete before generating the next one and subscribing to it.</li> * <li><b>Ordering of the flattened values</b>: this operator naturally preserves * the same order as the source elements, concatenating the inners from each source * element sequentially.</li> * <li><b>Interleaving</b>: this operator does not let values from different inners * interleave (concatenation).</li> * </ul> * * <p> * Errors in the individual publishers will be delayed at the end of the whole concat * sequence (possibly getting combined into a {@link Exceptions#isMultiple(Throwable) composite} * if several sources error. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concatmap.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation. * * @param mapper the function to transform this sequence of T into concatenated sequences of V * @param <V> the produced concatenated type * * @return a concatenated {@link Flux} * */
public final <V> Flux<V> concatMapDelayError(Function<? super T, ? extends Publisher<? extends V>> mapper) { return concatMapDelayError(mapper, Queues.XS_BUFFER_SIZE); }
Transform the elements emitted by this Flux asynchronously into Publishers, then flatten these inner publishers into a single Flux, sequentially and preserving order using concatenation.

There are three dimensions to this operator that can be compared with flatMap and flatMapSequential:

  • Generation of inners and subscription: this operator waits for one inner to complete before generating the next one and subscribing to it.
  • Ordering of the flattened values: this operator naturally preserves the same order as the source elements, concatenating the inners from each source element sequentially.
  • Interleaving: this operator does not let values from different inners interleave (concatenation).

Errors in the individual publishers will be delayed at the end of the whole concat sequence (possibly getting combined into a composite if several sources error. The prefetch argument allows to give an arbitrary prefetch size to the inner Publisher.

Params:
  • mapper – the function to transform this sequence of T into concatenated sequences of V
  • prefetch – the inner source produced demand
Type parameters:
  • <V> – the produced concatenated type
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation.
Returns:a concatenated Flux
/** * Transform the elements emitted by this {@link Flux} asynchronously into Publishers, * then flatten these inner publishers into a single {@link Flux}, sequentially and * preserving order using concatenation. * <p> * There are three dimensions to this operator that can be compared with * {@link #flatMap(Function) flatMap} and {@link #flatMapSequential(Function) flatMapSequential}: * <ul> * <li><b>Generation of inners and subscription</b>: this operator waits for one * inner to complete before generating the next one and subscribing to it.</li> * <li><b>Ordering of the flattened values</b>: this operator naturally preserves * the same order as the source elements, concatenating the inners from each source * element sequentially.</li> * <li><b>Interleaving</b>: this operator does not let values from different inners * interleave (concatenation).</li> * </ul> * * <p> * Errors in the individual publishers will be delayed at the end of the whole concat * sequence (possibly getting combined into a {@link Exceptions#isMultiple(Throwable) composite} * if several sources error. * The prefetch argument allows to give an arbitrary prefetch size to the inner {@link Publisher}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concatmap.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation. * * @param mapper the function to transform this sequence of T into concatenated sequences of V * @param prefetch the inner source produced demand * @param <V> the produced concatenated type * * @return a concatenated {@link Flux} * */
public final <V> Flux<V> concatMapDelayError(Function<? super T, ? extends Publisher<? extends V>> mapper, int prefetch) { return onAssembly(new FluxConcatMap<>(this, mapper, Queues.get(prefetch), prefetch, FluxConcatMap.ErrorMode.END)); }
Transform the elements emitted by this Flux asynchronously into Publishers, then flatten these inner publishers into a single Flux, sequentially and preserving order using concatenation.

There are three dimensions to this operator that can be compared with flatMap and flatMapSequential:

  • Generation of inners and subscription: this operator waits for one inner to complete before generating the next one and subscribing to it.
  • Ordering of the flattened values: this operator naturally preserves the same order as the source elements, concatenating the inners from each source element sequentially.
  • Interleaving: this operator does not let values from different inners interleave (concatenation).

Errors in the individual publishers will be delayed after the current concat backlog if delayUntilEnd is false or after all sources if delayUntilEnd is true. The prefetch argument allows to give an arbitrary prefetch size to the inner Publisher.

Params:
  • mapper – the function to transform this sequence of T into concatenated sequences of V
  • delayUntilEnd – delay error until all sources have been consumed instead of after the current source
  • prefetch – the inner source produced demand
Type parameters:
  • <V> – the produced concatenated type
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation.
Returns:a concatenated Flux
/** * Transform the elements emitted by this {@link Flux} asynchronously into Publishers, * then flatten these inner publishers into a single {@link Flux}, sequentially and * preserving order using concatenation. * <p> * There are three dimensions to this operator that can be compared with * {@link #flatMap(Function) flatMap} and {@link #flatMapSequential(Function) flatMapSequential}: * <ul> * <li><b>Generation of inners and subscription</b>: this operator waits for one * inner to complete before generating the next one and subscribing to it.</li> * <li><b>Ordering of the flattened values</b>: this operator naturally preserves * the same order as the source elements, concatenating the inners from each source * element sequentially.</li> * <li><b>Interleaving</b>: this operator does not let values from different inners * interleave (concatenation).</li> * </ul> * * <p> * Errors in the individual publishers will be delayed after the current concat * backlog if delayUntilEnd is false or after all sources if delayUntilEnd is true. * The prefetch argument allows to give an arbitrary prefetch size to the inner {@link Publisher}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concatmap.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation. * * @param mapper the function to transform this sequence of T into concatenated sequences of V * @param delayUntilEnd delay error until all sources have been consumed instead of * after the current source * @param prefetch the inner source produced demand * @param <V> the produced concatenated type * * @return a concatenated {@link Flux} * */
public final <V> Flux<V> concatMapDelayError(Function<? super T, ? extends Publisher<? extends V>> mapper, boolean delayUntilEnd, int prefetch) { return onAssembly(new FluxConcatMap<>(this, mapper, Queues.get(prefetch), prefetch, delayUntilEnd ? FluxConcatMap.ErrorMode.END : FluxConcatMap.ErrorMode .BOUNDARY)); }
Transform the items emitted by this Flux into Iterable, then flatten the elements from those by concatenating them into a single Flux.

Note that unlike flatMap(Function) and concatMap(Function), with Iterable there is no notion of eager vs lazy inner subscription. The content of the Iterables are all played sequentially. Thus flatMapIterable and concatMapIterable are equivalent offered as a discoverability improvement for users that explore the API with the concat vs flatMap expectation.

Params:
Type parameters:
  • <R> – the merged output sequence type
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation.
Returns:a concatenation of the values from the Iterables obtained from each element in this Flux
/** * Transform the items emitted by this {@link Flux} into {@link Iterable}, then flatten the elements from those by * concatenating them into a single {@link Flux}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concatmap.png" alt=""> * <p> * Note that unlike {@link #flatMap(Function)} and {@link #concatMap(Function)}, with Iterable there is * no notion of eager vs lazy inner subscription. The content of the Iterables are all played sequentially. * Thus {@code flatMapIterable} and {@code concatMapIterable} are equivalent offered as a discoverability * improvement for users that explore the API with the concat vs flatMap expectation. * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation. * * @param mapper the {@link Function} to transform input sequence into N {@link Iterable} * @param <R> the merged output sequence type * * @return a concatenation of the values from the Iterables obtained from each element in this {@link Flux} */
public final <R> Flux<R> concatMapIterable(Function<? super T, ? extends Iterable<? extends R>> mapper) { return concatMapIterable(mapper, Queues.XS_BUFFER_SIZE); }
Transform the items emitted by this Flux into Iterable, then flatten the emissions from those by concatenating them into a single Flux. The prefetch argument allows to give an arbitrary prefetch size to the merged Iterable.

Note that unlike flatMap(Function) and concatMap(Function), with Iterable there is no notion of eager vs lazy inner subscription. The content of the Iterables are all played sequentially. Thus flatMapIterable and concatMapIterable are equivalent offered as a discoverability improvement for users that explore the API with the concat vs flatMap expectation.

Params:
  • mapper – the Function to transform input sequence into N Iterable
  • prefetch – the maximum in-flight elements from each inner Iterable sequence
Type parameters:
  • <R> – the merged output sequence type
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation.
Returns:a concatenation of the values from the Iterables obtained from each element in this Flux
/** * Transform the items emitted by this {@link Flux} into {@link Iterable}, then flatten the emissions from those by * concatenating them into a single {@link Flux}. The prefetch argument allows to give an arbitrary prefetch size to the merged {@link Iterable}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concatmap.png" alt=""> * <p> * Note that unlike {@link #flatMap(Function)} and {@link #concatMap(Function)}, with Iterable there is * no notion of eager vs lazy inner subscription. The content of the Iterables are all played sequentially. * Thus {@code flatMapIterable} and {@code concatMapIterable} are equivalent offered as a discoverability * improvement for users that explore the API with the concat vs flatMap expectation. * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation. * * @param mapper the {@link Function} to transform input sequence into N {@link Iterable} * @param prefetch the maximum in-flight elements from each inner {@link Iterable} sequence * @param <R> the merged output sequence type * * @return a concatenation of the values from the Iterables obtained from each element in this {@link Flux} */
public final <R> Flux<R> concatMapIterable(Function<? super T, ? extends Iterable<? extends R>> mapper, int prefetch) { return onAssembly(new FluxFlattenIterable<>(this, mapper, prefetch, Queues.get(prefetch))); }
Concatenate emissions of this Flux with the provided Publisher (no interleave).

Params:
Returns:a concatenated Flux
/** * Concatenate emissions of this {@link Flux} with the provided {@link Publisher} (no interleave). * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concat.png" * alt=""> * * @param other the {@link Publisher} sequence to concat after this {@link Flux} * * @return a concatenated {@link Flux} */
public final Flux<T> concatWith(Publisher<? extends T> other) { if (this instanceof FluxConcatArray) { @SuppressWarnings({ "unchecked" }) FluxConcatArray<T> fluxConcatArray = (FluxConcatArray<T>) this; return fluxConcatArray.concatAdditionalSourceLast(other); } return concat(this, other); }
Counts the number of values in this Flux. The count will be emitted when onComplete is observed.

Returns:a new Mono of Long count
/** * Counts the number of values in this {@link Flux}. * The count will be emitted when onComplete is observed. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/count.png" alt=""> * * @return a new {@link Mono} of {@link Long} count */
public final Mono<Long> count() { return Mono.onAssembly(new MonoCount<>(this)); }
Provide a default unique value if this sequence is completed without any data

Params:
  • defaultV – the alternate value if this sequence is empty
Returns:a new Flux
/** * Provide a default unique value if this sequence is completed without any data * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/defaultifempty.png" alt=""> * <p> * @param defaultV the alternate value if this sequence is empty * * @return a new {@link Flux} */
public final Flux<T> defaultIfEmpty(T defaultV) { return onAssembly(new FluxDefaultIfEmpty<>(this, defaultV)); }
Delay each of this Flux elements (Subscriber.onNext signals) by a given Duration. Signals are delayed and continue on the parallel default Scheduler, but empty sequences or immediate error signals are not delayed.

Params:
See Also:
Returns:a delayed Flux
/** * Delay each of this {@link Flux} elements ({@link Subscriber#onNext} signals) * by a given {@link Duration}. Signals are delayed and continue on the * {@link Schedulers#parallel() parallel} default Scheduler, but empty sequences or * immediate error signals are not delayed. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/delayonnext.png" alt=""> * * @param delay duration by which to delay each {@link Subscriber#onNext} signal * @return a delayed {@link Flux} * @see #delaySubscription(Duration) delaySubscription to introduce a delay at the beginning of the sequence only */
public final Flux<T> delayElements(Duration delay) { return delayElements(delay, Schedulers.parallel()); }
Delay each of this Flux elements (Subscriber.onNext signals) by a given Duration. Signals are delayed and continue on an user-specified Scheduler, but empty sequences or immediate error signals are not delayed.

Params:
Returns:a delayed Flux
/** * Delay each of this {@link Flux} elements ({@link Subscriber#onNext} signals) * by a given {@link Duration}. Signals are delayed and continue on an user-specified * {@link Scheduler}, but empty sequences or immediate error signals are not delayed. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/delayonnext.png" alt=""> * * @param delay period to delay each {@link Subscriber#onNext} signal * @param timer a time-capable {@link Scheduler} instance to delay each signal on * @return a delayed {@link Flux} */
public final Flux<T> delayElements(Duration delay, Scheduler timer) { return delayUntil(d -> Mono.delay(delay, timer)); }
Shift this Flux forward in time by a given Duration. Unlike with delayElements(Duration), elements are shifted forward in time as they are emitted, always resulting in the delay between two elements being the same as in the source (only the first element is visibly delayed from the previous event, that is the subscription). Signals are delayed and continue on the parallel Scheduler, but empty sequences or immediate error signals are not delayed.

With this operator, a source emitting at 10Hz with a delaySequence Duration of 1s will still emit at 10Hz, with an initial "hiccup" of 1s. On the other hand, delayElements(Duration) would end up emitting at 1Hz.

This is closer to delaySubscription(Duration), except the source is subscribed to immediately.

Params:
  • delay – Duration to shift the sequence by
Returns:an shifted Flux emitting at the same frequency as the source
/** * Shift this {@link Flux} forward in time by a given {@link Duration}. * Unlike with {@link #delayElements(Duration)}, elements are shifted forward in time * as they are emitted, always resulting in the delay between two elements being * the same as in the source (only the first element is visibly delayed from the * previous event, that is the subscription). * Signals are delayed and continue on the {@link Schedulers#parallel() parallel} * {@link Scheduler}, but empty sequences or immediate error signals are not delayed. * <p> * With this operator, a source emitting at 10Hz with a delaySequence {@link Duration} * of 1s will still emit at 10Hz, with an initial "hiccup" of 1s. * On the other hand, {@link #delayElements(Duration)} would end up emitting * at 1Hz. * <p> * This is closer to {@link #delaySubscription(Duration)}, except the source * is subscribed to immediately. * * @param delay {@link Duration} to shift the sequence by * @return an shifted {@link Flux} emitting at the same frequency as the source */
//FIXME marble diagram public final Flux<T> delaySequence(Duration delay) { return delaySequence(delay, Schedulers.parallel()); }
Shift this Flux forward in time by a given Duration. Unlike with delayElements(Duration, Scheduler), elements are shifted forward in time as they are emitted, always resulting in the delay between two elements being the same as in the source (only the first element is visibly delayed from the previous event, that is the subscription). Signals are delayed and continue on an user-specified Scheduler, but empty sequences or immediate error signals are not delayed.

With this operator, a source emitting at 10Hz with a delaySequence Duration of 1s will still emit at 10Hz, with an initial "hiccup" of 1s. On the other hand, delayElements(Duration, Scheduler) would end up emitting at 1Hz.

This is closer to delaySubscription(Duration, Scheduler), except the source is subscribed to immediately.

Params:
  • delay – Duration to shift the sequence by
  • timer – a time-capable Scheduler instance to delay signals on
Returns:an shifted Flux emitting at the same frequency as the source
/** * Shift this {@link Flux} forward in time by a given {@link Duration}. * Unlike with {@link #delayElements(Duration, Scheduler)}, elements are shifted forward in time * as they are emitted, always resulting in the delay between two elements being * the same as in the source (only the first element is visibly delayed from the * previous event, that is the subscription). * Signals are delayed and continue on an user-specified {@link Scheduler}, but empty * sequences or immediate error signals are not delayed. * <p> * With this operator, a source emitting at 10Hz with a delaySequence {@link Duration} * of 1s will still emit at 10Hz, with an initial "hiccup" of 1s. * On the other hand, {@link #delayElements(Duration, Scheduler)} would end up emitting * at 1Hz. * <p> * This is closer to {@link #delaySubscription(Duration, Scheduler)}, except the source * is subscribed to immediately. * * @param delay {@link Duration} to shift the sequence by * @param timer a time-capable {@link Scheduler} instance to delay signals on * @return an shifted {@link Flux} emitting at the same frequency as the source */
//FIXME marble diagram public final Flux<T> delaySequence(Duration delay, Scheduler timer) { return onAssembly(new FluxDelaySequence<>(this, delay, timer)); }
Subscribe to this Flux and generate a Publisher from each of this Flux elements, each acting as a trigger for relaying said element.

That is to say, the resulting Flux delays each of its emission until the associated trigger Publisher terminates.

In case of an error either in the source or in a trigger, that error is propagated immediately downstream. Note that unlike with the Mono variant there is no fusion of subsequent calls.

Params:
  • triggerProvider – a Function that maps each element into a Publisher whose termination will trigger relaying the value.
Returns:this Flux, but with elements delayed until their derived publisher terminates.
/** * Subscribe to this {@link Flux} and generate a {@link Publisher} from each of this * Flux elements, each acting as a trigger for relaying said element. * <p> * That is to say, the resulting {@link Flux} delays each of its emission until the * associated trigger Publisher terminates. * <p> * In case of an error either in the source or in a trigger, that error is propagated * immediately downstream. * Note that unlike with the {@link Mono#delayUntil(Function) Mono variant} there is * no fusion of subsequent calls. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/delayUntil.png" alt=""> * * @param triggerProvider a {@link Function} that maps each element into a * {@link Publisher} whose termination will trigger relaying the value. * * @return this Flux, but with elements delayed until their derived publisher terminates. */
public final Flux<T> delayUntil(Function<? super T, ? extends Publisher<?>> triggerProvider) { return concatMap(v -> Mono.just(v) .delayUntil(triggerProvider)); }
Delay the subscription to this Flux source until the given period elapses. The delay is introduced through the parallel default Scheduler.

Params:
  • delay – duration before subscribing this Flux
Returns:a delayed Flux
/** * Delay the {@link Flux#subscribe(Subscriber) subscription} to this {@link Flux} source until the given * period elapses. The delay is introduced through the {@link Schedulers#parallel() parallel} default Scheduler. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/delaysubscription.png" alt=""> * * @param delay duration before subscribing this {@link Flux} * * @return a delayed {@link Flux} * */
public final Flux<T> delaySubscription(Duration delay) { return delaySubscription(delay, Schedulers.parallel()); }
Delay the subscription to this Flux source until the given period elapses, as measured on the user-provided Scheduler.

Params:
Returns:a delayed Flux
/** * Delay the {@link Flux#subscribe(Subscriber) subscription} to this {@link Flux} source until the given * period elapses, as measured on the user-provided {@link Scheduler}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/delaysubscription.png" alt=""> * * @param delay {@link Duration} before subscribing this {@link Flux} * @param timer a time-capable {@link Scheduler} instance to run on * * @return a delayed {@link Flux} */
public final Flux<T> delaySubscription(Duration delay, Scheduler timer) { return delaySubscription(Mono.delay(delay, timer)); }
Delay the subscription to this Flux source until another Publisher signals a value or completes.

Params:
Type parameters:
  • <U> – the other source type
Returns:a delayed Flux
/** * Delay the {@link Flux#subscribe(Subscriber) subscription} to this {@link Flux} * source until another {@link Publisher} signals a value or completes. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/delaysubscriptionp.png" alt=""> * * @param subscriptionDelay a companion {@link Publisher} whose onNext/onComplete signal will trigger the {@link Flux#subscribe(Subscriber) subscription} * @param <U> the other source type * * @return a delayed {@link Flux} * */
public final <U> Flux<T> delaySubscription(Publisher<U> subscriptionDelay) { return onAssembly(new FluxDelaySubscription<>(this, subscriptionDelay)); }
An operator working only if this Flux emits onNext, onError or onComplete Signal instances, transforming these materialized signals into real signals on the Subscriber. The error Signal will trigger onError and complete Signal will trigger onComplete.

Type parameters:
  • <X> – the dematerialized type
See Also:
Returns:a dematerialized Flux
/** * An operator working only if this {@link Flux} emits onNext, onError or onComplete {@link Signal} * instances, transforming these {@link #materialize() materialized} signals into * real signals on the {@link Subscriber}. * The error {@link Signal} will trigger onError and complete {@link Signal} will trigger * onComplete. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/dematerialize.png" alt=""> * * @param <X> the dematerialized type * * @return a dematerialized {@link Flux} * @see #materialize() */
public final <X> Flux<X> dematerialize() { @SuppressWarnings("unchecked") Flux<Signal<X>> thiz = (Flux<Signal<X>>) this; return onAssembly(new FluxDematerialize<>(thiz)); }
For each Subscriber, track elements from this Flux that have been seen and filter out duplicates.

The values themselves are recorded into a HashSet for distinct detection. Use distinct(Object::hashcode) if you want a more lightweight approach that doesn't retain all the objects, but is more susceptible to falsely considering two elements as distinct due to a hashcode collision.

@reactor.discardThis operator discards elements that don't match the distinct predicate, but you should use the version with a cleanup if you need discarding of keys categorized by the operator as "seen". See distinct(Function, Supplier, BiPredicate, Consumer).
Returns:a filtering Flux only emitting distinct values
/** * For each {@link Subscriber}, track elements from this {@link Flux} that have been * seen and filter out duplicates. * <p> * The values themselves are recorded into a {@link HashSet} for distinct detection. * Use {@code distinct(Object::hashcode)} if you want a more lightweight approach that * doesn't retain all the objects, but is more susceptible to falsely considering two * elements as distinct due to a hashcode collision. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/distinct.png" alt=""> * * @reactor.discard This operator discards elements that don't match the distinct predicate, * but you should use the version with a cleanup if you need discarding of keys * categorized by the operator as "seen". See {@link #distinct(Function, Supplier, BiPredicate, Consumer)}. * * @return a filtering {@link Flux} only emitting distinct values */
public final Flux<T> distinct() { return distinct(identityFunction()); }
For each Subscriber, track elements from this Flux that have been seen and filter out duplicates, as compared by a key extracted through the user provided Function.

Params:
  • keySelector – function to compute comparison key for each element
Type parameters:
  • <V> – the type of the key extracted from each value in this sequence
@reactor.discardThis operator discards elements that don't match the distinct predicate, but you should use the version with a cleanup if you need discarding of keys categorized by the operator as "seen". See distinct(Function, Supplier, BiPredicate, Consumer).
Returns:a filtering Flux only emitting values with distinct keys
/** * For each {@link Subscriber}, track elements from this {@link Flux} that have been * seen and filter out duplicates, as compared by a key extracted through the user * provided {@link Function}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/distinctk.png" alt=""> * * @reactor.discard This operator discards elements that don't match the distinct predicate, * but you should use the version with a cleanup if you need discarding of keys * categorized by the operator as "seen". See {@link #distinct(Function, Supplier, BiPredicate, Consumer)}. * * @param keySelector function to compute comparison key for each element * @param <V> the type of the key extracted from each value in this sequence * * @return a filtering {@link Flux} only emitting values with distinct keys */
public final <V> Flux<T> distinct(Function<? super T, ? extends V> keySelector) { return distinct(keySelector, hashSetSupplier()); }
For each Subscriber, track elements from this Flux that have been seen and filter out duplicates, as compared by a key extracted through the user provided Function and by the add method of the Collection supplied (typically a Set).

Params:
  • keySelector – function to compute comparison key for each element
  • distinctCollectionSupplier – supplier of the Collection used for distinct check through add of the key.
Type parameters:
  • <V> – the type of the key extracted from each value in this sequence
  • <C> – the type of Collection used for distinct checking of keys
@reactor.discardThis operator discards elements that don't match the distinct predicate, but you should use the version with a cleanup if you need discarding of keys categorized by the operator as "seen". See distinct(Function, Supplier, BiPredicate, Consumer).
Returns:a filtering Flux only emitting values with distinct keys
/** * For each {@link Subscriber}, track elements from this {@link Flux} that have been * seen and filter out duplicates, as compared by a key extracted through the user * provided {@link Function} and by the {@link Collection#add(Object) add method} * of the {@link Collection} supplied (typically a {@link Set}). * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/distinctk.png" alt=""> * * @reactor.discard This operator discards elements that don't match the distinct predicate, * but you should use the version with a cleanup if you need discarding of keys * categorized by the operator as "seen". See {@link #distinct(Function, Supplier, BiPredicate, Consumer)}. * * @param keySelector function to compute comparison key for each element * @param distinctCollectionSupplier supplier of the {@link Collection} used for distinct * check through {@link Collection#add(Object) add} of the key. * * @param <V> the type of the key extracted from each value in this sequence * @param <C> the type of Collection used for distinct checking of keys * * @return a filtering {@link Flux} only emitting values with distinct keys */
public final <V, C extends Collection<? super V>> Flux<T> distinct( Function<? super T, ? extends V> keySelector, Supplier<C> distinctCollectionSupplier) { return this.distinct(keySelector, distinctCollectionSupplier, Collection::add, Collection::clear); }
For each Subscriber, track elements from this Flux that have been seen and filter out duplicates, as compared by applying a BiPredicate on an arbitrary user-supplied <C> store and a key extracted through the user provided Function. The BiPredicate should typically add the key to the arbitrary store for further comparison. A cleanup callback is also invoked on the store upon termination of the sequence.

Params:
  • keySelector – function to compute comparison key for each element
  • distinctStoreSupplier – supplier of the arbitrary store object used in distinct checks along the extracted key.
  • distinctPredicate – the BiPredicate to apply to the arbitrary store + extracted key to perform a distinct check. Since nothing is assumed of the store, this predicate should also add the key to the store as necessary.
  • cleanup – the cleanup callback to invoke on the store upon termination.
Type parameters:
  • <V> – the type of the key extracted from each value in this sequence
  • <C> – the type of store backing the BiPredicate
@reactor.discardThis operator discards elements that don't match the distinct predicate, but you should use the cleanup as well if you need discarding of keys categorized by the operator as "seen".
Returns:a filtering Flux only emitting values with distinct keys
/** * For each {@link Subscriber}, track elements from this {@link Flux} that have been * seen and filter out duplicates, as compared by applying a {@link BiPredicate} on * an arbitrary user-supplied {@code <C>} store and a key extracted through the user * provided {@link Function}. The BiPredicate should typically add the key to the * arbitrary store for further comparison. A cleanup callback is also invoked on the * store upon termination of the sequence. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/distinctk.png" alt=""> * * @reactor.discard This operator discards elements that don't match the distinct predicate, * but you should use the {@code cleanup} as well if you need discarding of keys * categorized by the operator as "seen". * * @param keySelector function to compute comparison key for each element * @param distinctStoreSupplier supplier of the arbitrary store object used in distinct * checks along the extracted key. * @param distinctPredicate the {@link BiPredicate} to apply to the arbitrary store + * extracted key to perform a distinct check. Since nothing is assumed of the store, * this predicate should also add the key to the store as necessary. * @param cleanup the cleanup callback to invoke on the store upon termination. * * @param <V> the type of the key extracted from each value in this sequence * @param <C> the type of store backing the {@link BiPredicate} * * @return a filtering {@link Flux} only emitting values with distinct keys */
public final <V, C> Flux<T> distinct( Function<? super T, ? extends V> keySelector, Supplier<C> distinctStoreSupplier, BiPredicate<C, V> distinctPredicate, Consumer<C> cleanup) { if (this instanceof Fuseable) { return onAssembly(new FluxDistinctFuseable<>(this, keySelector, distinctStoreSupplier, distinctPredicate, cleanup)); } return onAssembly(new FluxDistinct<>(this, keySelector, distinctStoreSupplier, distinctPredicate, cleanup)); }
Filter out subsequent repetitions of an element (that is, if they arrive right after one another).

The last distinct value seen is retained for further comparison, which is done on the values themselves using the equals method. Use distinctUntilChanged(Object::hashcode) if you want a more lightweight approach that doesn't retain all the objects, but is more susceptible to falsely considering two elements as distinct due to a hashcode collision.

@reactor.discardAlthough this operator discards elements that are considered as "already seen", it is not recommended for cases where discarding is needed as the operator doesn't discard the "key" (in this context, the distinct instance that was last seen).
Returns:a filtering Flux with only one occurrence in a row of each element (yet elements can repeat in the overall sequence)
/** * Filter out subsequent repetitions of an element (that is, if they arrive right after * one another). * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/distinctuntilchanged.png" alt=""> * <p> * The last distinct value seen is retained for further comparison, which is done * on the values themselves using {@link Object#equals(Object) the equals method}. * Use {@code distinctUntilChanged(Object::hashcode)} if you want a more lightweight approach that * doesn't retain all the objects, but is more susceptible to falsely considering two * elements as distinct due to a hashcode collision. * * @reactor.discard Although this operator discards elements that are considered as "already seen", * it is not recommended for cases where discarding is needed as the operator doesn't * discard the "key" (in this context, the distinct instance that was last seen). * * @return a filtering {@link Flux} with only one occurrence in a row of each element * (yet elements can repeat in the overall sequence) */
public final Flux<T> distinctUntilChanged() { return distinctUntilChanged(identityFunction()); }
Filter out subsequent repetitions of an element (that is, if they arrive right after one another), as compared by a key extracted through the user provided Function using equality.

Params:
  • keySelector – function to compute comparison key for each element
Type parameters:
  • <V> – the type of the key extracted from each value in this sequence
@reactor.discardThis operator discards elements that are considered as "already seen". The keys themselves are not discarded.
Returns:a filtering Flux with only one occurrence in a row of each element of the same key (yet element keys can repeat in the overall sequence)
/** * Filter out subsequent repetitions of an element (that is, if they arrive right after * one another), as compared by a key extracted through the user provided {@link Function} * using equality. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/distinctuntilchangedk.png" alt=""> * * @reactor.discard This operator discards elements that are considered as "already seen". * The keys themselves are not discarded. * * @param keySelector function to compute comparison key for each element * @param <V> the type of the key extracted from each value in this sequence * * @return a filtering {@link Flux} with only one occurrence in a row of each element of * the same key (yet element keys can repeat in the overall sequence) */
public final <V> Flux<T> distinctUntilChanged(Function<? super T, ? extends V> keySelector) { return distinctUntilChanged(keySelector, equalPredicate()); }
Filter out subsequent repetitions of an element (that is, if they arrive right after one another), as compared by a key extracted through the user provided Function and then comparing keys with the supplied BiPredicate.

Params:
  • keySelector – function to compute comparison key for each element
  • keyComparator – predicate used to compare keys.
Type parameters:
  • <V> – the type of the key extracted from each value in this sequence
@reactor.discardThis operator discards elements that are considered as "already seen" (for which the keyComparator returns true). The keys themselves are not discarded.
Returns:a filtering Flux with only one occurrence in a row of each element of the same key for which the predicate returns true (yet element keys can repeat in the overall sequence)
/** * Filter out subsequent repetitions of an element (that is, if they arrive right * after one another), as compared by a key extracted through the user provided {@link * Function} and then comparing keys with the supplied {@link BiPredicate}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/distinctuntilchangedk.png" * alt=""> * * @reactor.discard This operator discards elements that are considered as "already seen" * (for which the {@code keyComparator} returns {@literal true}). The keys themselves * are not discarded. * * @param keySelector function to compute comparison key for each element * @param keyComparator predicate used to compare keys. * @param <V> the type of the key extracted from each value in this sequence * * @return a filtering {@link Flux} with only one occurrence in a row of each element * of the same key for which the predicate returns true (yet element keys can repeat * in the overall sequence) */
public final <V> Flux<T> distinctUntilChanged(Function<? super T, ? extends V> keySelector, BiPredicate<? super V, ? super V> keyComparator) { return onAssembly(new FluxDistinctUntilChanged<>(this, keySelector, keyComparator)); }
Add behavior (side-effect) triggered after the Flux terminates, either by completing downstream successfully or with an error.

Params:
Returns:an observed Flux
/** * Add behavior (side-effect) triggered after the {@link Flux} terminates, either by completing downstream successfully or with an error. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/doafterterminate.png" alt=""> * <p> * @param afterTerminate the callback to call after {@link Subscriber#onComplete} or {@link Subscriber#onError} * * @return an observed {@link Flux} */
public final Flux<T> doAfterTerminate(Runnable afterTerminate) { Objects.requireNonNull(afterTerminate, "afterTerminate"); return doOnSignal(this, null, null, null, null, afterTerminate, null, null); }
Add behavior (side-effect) triggered when the Flux is cancelled.

Params:
Returns:an observed Flux
/** * Add behavior (side-effect) triggered when the {@link Flux} is cancelled. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/dooncancel.png" alt=""> * <p> * @param onCancel the callback to call on {@link Subscription#cancel} * * @return an observed {@link Flux} */
public final Flux<T> doOnCancel(Runnable onCancel) { Objects.requireNonNull(onCancel, "onCancel"); return doOnSignal(this, null, null, null, null, null, null, onCancel); }
Add behavior (side-effect) triggered when the Flux completes successfully.

Params:
Returns:an observed Flux
/** * Add behavior (side-effect) triggered when the {@link Flux} completes successfully. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/dooncomplete.png" alt=""> * <p> * @param onComplete the callback to call on {@link Subscriber#onComplete} * * @return an observed {@link Flux} */
public final Flux<T> doOnComplete(Runnable onComplete) { Objects.requireNonNull(onComplete, "onComplete"); return doOnSignal(this, null, null, null, onComplete, null, null, null); }
Modify the behavior of the whole chain of operators upstream of this one to conditionally clean up elements that get discarded by these operators.

The discardHook must be idempotent and safe to use on any instance of the desired type. Calls to this method are additive, and the order of invocation of the discardHook is the same as the order of declaration (calling .filter(...).doOnDiscard(first).doOnDiscard(second) will let the filter invoke first then second handlers).

Two main categories of discarding operators exist:

  • filtering operators, dropping some source elements as part of their designed behavior
  • operators that prefetch a few elements and keep them around pending a request, but get cancelled/in error
These operators are identified in the javadoc by the presence of an onDiscard Support section.
Params:
  • type – the Class of elements in the upstream chain of operators that this cleanup hook should take into account.
  • discardHook – a Consumer of elements in the upstream chain of operators that performs the cleanup.
Returns:a Flux that cleans up matching elements that get discarded upstream of it.
/** * Modify the behavior of the <i>whole chain</i> of operators upstream of this one to * conditionally clean up elements that get <i>discarded</i> by these operators. * <p> * The {@code discardHook} must be idempotent and safe to use on any instance of the desired * type. * Calls to this method are additive, and the order of invocation of the {@code discardHook} * is the same as the order of declaration (calling {@code .filter(...).doOnDiscard(first).doOnDiscard(second)} * will let the filter invoke {@code first} then {@code second} handlers). * <p> * Two main categories of discarding operators exist: * <ul> * <li>filtering operators, dropping some source elements as part of their designed behavior</li> * <li>operators that prefetch a few elements and keep them around pending a request, but get cancelled/in error</li> * </ul> * These operators are identified in the javadoc by the presence of an {@code onDiscard Support} section. * * @param type the {@link Class} of elements in the upstream chain of operators that * this cleanup hook should take into account. * @param discardHook a {@link Consumer} of elements in the upstream chain of operators * that performs the cleanup. * @return a {@link Flux} that cleans up matching elements that get discarded upstream of it. */
public final <R> Flux<T> doOnDiscard(final Class<R> type, final Consumer<? super R> discardHook) { return subscriberContext(Operators.discardLocalAdapter(type, discardHook)); }
Add behavior (side-effects) triggered when the Flux emits an item, fails with an error or completes successfully. All these events are represented as a Signal that is passed to the side-effect callback. Note that this is an advanced operator, typically used for monitoring of a Flux. These Signal have a Context associated to them.
Params:
See Also:
Returns:an observed Flux
/** * Add behavior (side-effects) triggered when the {@link Flux} emits an item, fails with an error * or completes successfully. All these events are represented as a {@link Signal} * that is passed to the side-effect callback. Note that this is an advanced operator, * typically used for monitoring of a Flux. These {@link Signal} have a {@link Context} * associated to them. * * @param signalConsumer the mandatory callback to call on * {@link Subscriber#onNext(Object)}, {@link Subscriber#onError(Throwable)} and * {@link Subscriber#onComplete()} * @return an observed {@link Flux} * @see #doOnNext(Consumer) * @see #doOnError(Consumer) * @see #doOnComplete(Runnable) * @see #materialize() * @see Signal */
public final Flux<T> doOnEach(Consumer<? super Signal<T>> signalConsumer) { if (this instanceof Fuseable) { return onAssembly(new FluxDoOnEachFuseable<>(this, signalConsumer)); } return onAssembly(new FluxDoOnEach<>(this, signalConsumer)); }
Add behavior (side-effect) triggered when the Flux completes with an error.

Params:
Returns:an observed Flux
/** * Add behavior (side-effect) triggered when the {@link Flux} completes with an error. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/doonerror.png" alt=""> * <p> * @param onError the callback to call on {@link Subscriber#onError} * * @return an observed {@link Flux} */
public final Flux<T> doOnError(Consumer<? super Throwable> onError) { Objects.requireNonNull(onError, "onError"); return doOnSignal(this, null, null, onError, null, null, null, null); }
Add behavior (side-effect) triggered when the Flux completes with an error matching the given exception type.

Params:
  • exceptionType – the type of exceptions to handle
  • onError – the error handler for each error
Type parameters:
  • <E> – type of the error to handle
Returns:an observed Flux
/** * Add behavior (side-effect) triggered when the {@link Flux} completes with an error matching the given exception type. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/doonerrorw.png" alt=""> * * @param exceptionType the type of exceptions to handle * @param onError the error handler for each error * @param <E> type of the error to handle * * @return an observed {@link Flux} * */
public final <E extends Throwable> Flux<T> doOnError(Class<E> exceptionType, final Consumer<? super E> onError) { Objects.requireNonNull(exceptionType, "type"); @SuppressWarnings("unchecked") Consumer<Throwable> handler = (Consumer<Throwable>)onError; return doOnError(exceptionType::isInstance, (handler)); }
Add behavior (side-effect) triggered when the Flux completes with an error matching the given exception.

Params:
  • predicate – the matcher for exceptions to handle
  • onError – the error handler for each error
Returns:an observed Flux
/** * Add behavior (side-effect) triggered when the {@link Flux} completes with an error matching the given exception. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/doonerrorw.png" alt=""> * * @param predicate the matcher for exceptions to handle * @param onError the error handler for each error * * @return an observed {@link Flux} * */
public final Flux<T> doOnError(Predicate<? super Throwable> predicate, final Consumer<? super Throwable> onError) { Objects.requireNonNull(predicate, "predicate"); return doOnError(t -> { if (predicate.test(t)) { onError.accept(t); } }); }
Add behavior (side-effect) triggered when the Flux emits an item.

Params:
@reactor.errorModeThis operator supports resuming on errors (including when fusion is enabled). Exceptions thrown by the consumer are passed to the onErrorContinue(BiConsumer) error consumer (the value consumer is not invoked, as the source element will be part of the sequence). The onNext signal is then propagated as normal.
Returns:an observed Flux
/** * Add behavior (side-effect) triggered when the {@link Flux} emits an item. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/doonnext.png" alt=""> * <p> * @param onNext the callback to call on {@link Subscriber#onNext} * * @reactor.errorMode This operator supports {@link #onErrorContinue(BiConsumer) resuming on errors} * (including when fusion is enabled). Exceptions thrown by the consumer are passed to * the {@link #onErrorContinue(BiConsumer)} error consumer (the value consumer * is not invoked, as the source element will be part of the sequence). The onNext * signal is then propagated as normal. * * @return an observed {@link Flux} */
public final Flux<T> doOnNext(Consumer<? super T> onNext) { Objects.requireNonNull(onNext, "onNext"); return doOnSignal(this, null, onNext, null, null, null, null, null); }
Add behavior (side-effect) triggering a LongConsumer when this Flux receives any request.

Note that non fatal error raised in the callback will not be propagated and will simply trigger Operators.onOperatorError(Throwable, Context).

Params:
  • consumer – the consumer to invoke on each request
Returns:an observed Flux
/** * Add behavior (side-effect) triggering a {@link LongConsumer} when this {@link Flux} * receives any request. * <p> * Note that non fatal error raised in the callback will not be propagated and * will simply trigger {@link Operators#onOperatorError(Throwable, Context)}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/doonrequest.png" alt=""> * * @param consumer the consumer to invoke on each request * * @return an observed {@link Flux} */
public final Flux<T> doOnRequest(LongConsumer consumer) { Objects.requireNonNull(consumer, "consumer"); return doOnSignal(this, null, null, null, null, null, consumer, null); }
Add behavior (side-effect) triggered when the Flux is subscribed.

This method is not intended for capturing the subscription and calling its methods, but for side effects like monitoring. For instance, the correct way to cancel a subscription is to call Disposable.dispose() on the Disposable returned by subscribe().

Params:
Returns:an observed Flux
/** * Add behavior (side-effect) triggered when the {@link Flux} is subscribed. * <p> * This method is <strong>not</strong> intended for capturing the subscription and calling its methods, * but for side effects like monitoring. For instance, the correct way to cancel a subscription is * to call {@link Disposable#dispose()} on the Disposable returned by {@link Flux#subscribe()}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/doonsubscribe.png" alt=""> * <p> * @param onSubscribe the callback to call on {@link Subscriber#onSubscribe} * * @return an observed {@link Flux} */
public final Flux<T> doOnSubscribe(Consumer<? super Subscription> onSubscribe) { Objects.requireNonNull(onSubscribe, "onSubscribe"); return doOnSignal(this, onSubscribe, null, null, null, null, null, null); }
Add behavior (side-effect) triggered when the Flux terminates, either by completing successfully or with an error.

Params:
Returns:an observed Flux
/** * Add behavior (side-effect) triggered when the {@link Flux} terminates, either by * completing successfully or with an error. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/doonterminate.png" alt=""> * <p> * @param onTerminate the callback to call on {@link Subscriber#onComplete} or {@link Subscriber#onError} * * @return an observed {@link Flux} */
public final Flux<T> doOnTerminate(Runnable onTerminate) { Objects.requireNonNull(onTerminate, "onTerminate"); return doOnSignal(this, null, null, e -> onTerminate.run(), onTerminate, null, null, null); }
Add behavior (side-effect) triggered after the Flux terminates for any reason, including cancellation. The terminating event (SignalType.ON_COMPLETE, SignalType.ON_ERROR and SignalType.CANCEL) is passed to the consumer, which is executed after the signal has been passed downstream.

Note that the fact that the signal is propagated downstream before the callback is executed means that several doFinally in a row will be executed in reverse order. If you want to assert the execution of the callback please keep in mind that the Flux will complete before it is executed, so its effect might not be visible immediately after eg. a blockLast().

Params:
  • onFinally – the callback to execute after a terminal signal (complete, error or cancel)
Returns:an observed Flux
/** * Add behavior (side-effect) triggered <strong>after</strong> the {@link Flux} terminates for any reason, * including cancellation. The terminating event ({@link SignalType#ON_COMPLETE}, * {@link SignalType#ON_ERROR} and {@link SignalType#CANCEL}) is passed to the consumer, * which is executed after the signal has been passed downstream. * <p> * Note that the fact that the signal is propagated downstream before the callback is * executed means that several doFinally in a row will be executed in * <strong>reverse order</strong>. If you want to assert the execution of the callback * please keep in mind that the Flux will complete before it is executed, so its * effect might not be visible immediately after eg. a {@link #blockLast()}. * * @param onFinally the callback to execute after a terminal signal (complete, error * or cancel) * @return an observed {@link Flux} */
public final Flux<T> doFinally(Consumer<SignalType> onFinally) { Objects.requireNonNull(onFinally, "onFinally"); if (this instanceof Fuseable) { return onAssembly(new FluxDoFinallyFuseable<>(this, onFinally)); } return onAssembly(new FluxDoFinally<>(this, onFinally)); }
Map this Flux into Tuple2<Long, T> of timemillis and source data. The timemillis corresponds to the elapsed time between each signal as measured by the parallel scheduler. First duration is measured between the subscription and the first element.

Returns:a new Flux that emits a tuple of time elapsed in milliseconds and matching data
/** * Map this {@link Flux} into {@link reactor.util.function.Tuple2 Tuple2&lt;Long, T&gt;} * of timemillis and source data. The timemillis corresponds to the elapsed time * between each signal as measured by the {@link Schedulers#parallel() parallel} scheduler. * First duration is measured between the subscription and the first element. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/elapsed.png" alt=""> * * @return a new {@link Flux} that emits a tuple of time elapsed in milliseconds and matching data */
public final Flux<Tuple2<Long, T>> elapsed() { return elapsed(Schedulers.parallel()); }
Map this Flux into Tuple2<Long, T> of timemillis and source data. The timemillis corresponds to the elapsed time between each signal as measured by the provided Scheduler. First duration is measured between the subscription and the first element.

Params:
Returns:a new Flux that emits tuples of time elapsed in milliseconds and matching data
/** * Map this {@link Flux} into {@link reactor.util.function.Tuple2 Tuple2&lt;Long, T&gt;} * of timemillis and source data. The timemillis corresponds to the elapsed time * between each signal as measured by the provided {@link Scheduler}. * First duration is measured between the subscription and the first element. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/elapsed.png" * alt=""> * * @param scheduler a {@link Scheduler} instance to {@link Scheduler#now(TimeUnit) read time from} * * @return a new {@link Flux} that emits tuples of time elapsed in milliseconds and matching data */
public final Flux<Tuple2<Long, T>> elapsed(Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler"); return onAssembly(new FluxElapsed<>(this, scheduler)); }
Emit only the element at the given index position or IndexOutOfBoundsException if the sequence is shorter.

Params:
  • index – zero-based index of the only item to emit
@reactor.discardThis operator discards elements that appear before the requested index.
Returns:a Mono of the item at the specified zero-based index
/** * Emit only the element at the given index position or {@link IndexOutOfBoundsException} * if the sequence is shorter. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/elementat.png" alt=""> * * @reactor.discard This operator discards elements that appear before the requested index. * * @param index zero-based index of the only item to emit * * @return a {@link Mono} of the item at the specified zero-based index */
public final Mono<T> elementAt(int index) { return Mono.onAssembly(new MonoElementAt<>(this, index)); }
Emit only the element at the given index position or fall back to a default value if the sequence is shorter.

Params:
  • index – zero-based index of the only item to emit
  • defaultValue – a default value to emit if the sequence is shorter
@reactor.discardThis operator discards elements that appear before the requested index.
Returns:a Mono of the item at the specified zero-based index or a default value
/** * Emit only the element at the given index position or fall back to a * default value if the sequence is shorter. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/elementatd.png" alt=""> * * @reactor.discard This operator discards elements that appear before the requested index. * * @param index zero-based index of the only item to emit * @param defaultValue a default value to emit if the sequence is shorter * * @return a {@link Mono} of the item at the specified zero-based index or a default value */
public final Mono<T> elementAt(int index, T defaultValue) { return Mono.onAssembly(new MonoElementAt<>(this, index, defaultValue)); }
Recursively expand elements into a graph and emit all the resulting element, in a depth-first traversal order.

That is: emit one value from this Flux, expand it and emit the first value at this first level of recursion, and so on... When no more recursion is possible, backtrack to the previous level and re-apply the strategy.

For example, given the hierarchical structure

 A
  - AA
    - aa1
 B
  - BB
    - bb1
Expands Flux.just(A, B) into
 A
 AA
 aa1
 B
 BB
 bb1
Params:
  • expander – the Function applied at each level of recursion to expand values into a Publisher, producing a graph.
  • capacityHint – a capacity hint to prepare the inner queues to accommodate n elements per level of recursion.
Returns:a Flux expanded depth-first
/** * Recursively expand elements into a graph and emit all the resulting element, * in a depth-first traversal order. * <p> * That is: emit one value from this {@link Flux}, expand it and emit the first value * at this first level of recursion, and so on... When no more recursion is possible, * backtrack to the previous level and re-apply the strategy. * <p> * For example, given the hierarchical structure * <pre> * A * - AA * - aa1 * B * - BB * - bb1 * </pre> * * Expands {@code Flux.just(A, B)} into * <pre> * A * AA * aa1 * B * BB * bb1 * </pre> * * @param expander the {@link Function} applied at each level of recursion to expand * values into a {@link Publisher}, producing a graph. * @param capacityHint a capacity hint to prepare the inner queues to accommodate n * elements per level of recursion. * * @return a {@link Flux} expanded depth-first */
public final Flux<T> expandDeep(Function<? super T, ? extends Publisher<? extends T>> expander, int capacityHint) { return onAssembly(new FluxExpand<>(this, expander, false, capacityHint)); }
Recursively expand elements into a graph and emit all the resulting element, in a depth-first traversal order.

That is: emit one value from this Flux, expand it and emit the first value at this first level of recursion, and so on... When no more recursion is possible, backtrack to the previous level and re-apply the strategy.

For example, given the hierarchical structure

 A
  - AA
    - aa1
 B
  - BB
    - bb1
Expands Flux.just(A, B) into
 A
 AA
 aa1
 B
 BB
 bb1
Params:
  • expander – the Function applied at each level of recursion to expand values into a Publisher, producing a graph.
Returns:a Flux expanded depth-first
/** * Recursively expand elements into a graph and emit all the resulting element, * in a depth-first traversal order. * <p> * That is: emit one value from this {@link Flux}, expand it and emit the first value * at this first level of recursion, and so on... When no more recursion is possible, * backtrack to the previous level and re-apply the strategy. * <p> * For example, given the hierarchical structure * <pre> * A * - AA * - aa1 * B * - BB * - bb1 * </pre> * * Expands {@code Flux.just(A, B)} into * <pre> * A * AA * aa1 * B * BB * bb1 * </pre> * * @param expander the {@link Function} applied at each level of recursion to expand * values into a {@link Publisher}, producing a graph. * * @return a {@link Flux} expanded depth-first */
public final Flux<T> expandDeep(Function<? super T, ? extends Publisher<? extends T>> expander) { return expandDeep(expander, Queues.SMALL_BUFFER_SIZE); }
Recursively expand elements into a graph and emit all the resulting element using a breadth-first traversal strategy.

That is: emit the values from this Flux first, then expand each at a first level of recursion and emit all of the resulting values, then expand all of these at a second level and so on..

For example, given the hierarchical structure

 A
  - AA
    - aa1
 B
  - BB
    - bb1
Expands Flux.just(A, B) into
 A
 B
 AA
 BB
 aa1
 bb1
Params:
  • expander – the Function applied at each level of recursion to expand values into a Publisher, producing a graph.
  • capacityHint – a capacity hint to prepare the inner queues to accommodate n elements per level of recursion.
Returns:an breadth-first expanded Flux
/** * Recursively expand elements into a graph and emit all the resulting element using * a breadth-first traversal strategy. * <p> * That is: emit the values from this {@link Flux} first, then expand each at a first level of * recursion and emit all of the resulting values, then expand all of these at a second * level and so on.. * <p> * For example, given the hierarchical structure * <pre> * A * - AA * - aa1 * B * - BB * - bb1 * </pre> * * Expands {@code Flux.just(A, B)} into * <pre> * A * B * AA * BB * aa1 * bb1 * </pre> * * @param expander the {@link Function} applied at each level of recursion to expand * values into a {@link Publisher}, producing a graph. * @param capacityHint a capacity hint to prepare the inner queues to accommodate n * elements per level of recursion. * * @return an breadth-first expanded {@link Flux} */
public final Flux<T> expand(Function<? super T, ? extends Publisher<? extends T>> expander, int capacityHint) { return Flux.onAssembly(new FluxExpand<>(this, expander, true, capacityHint)); }
Recursively expand elements into a graph and emit all the resulting element using a breadth-first traversal strategy.

That is: emit the values from this Flux first, then expand each at a first level of recursion and emit all of the resulting values, then expand all of these at a second level and so on..

For example, given the hierarchical structure

 A
  - AA
    - aa1
 B
  - BB
    - bb1
Expands Flux.just(A, B) into
 A
 B
 AA
 BB
 aa1
 bb1
Params:
  • expander – the Function applied at each level of recursion to expand values into a Publisher, producing a graph.
Returns:an breadth-first expanded Flux
/** * Recursively expand elements into a graph and emit all the resulting element using * a breadth-first traversal strategy. * <p> * That is: emit the values from this {@link Flux} first, then expand each at a first level of * recursion and emit all of the resulting values, then expand all of these at a second * level and so on.. * <p> * For example, given the hierarchical structure * <pre> * A * - AA * - aa1 * B * - BB * - bb1 * </pre> * * Expands {@code Flux.just(A, B)} into * <pre> * A * B * AA * BB * aa1 * bb1 * </pre> * * @param expander the {@link Function} applied at each level of recursion to expand * values into a {@link Publisher}, producing a graph. * * @return an breadth-first expanded {@link Flux} */
public final Flux<T> expand(Function<? super T, ? extends Publisher<? extends T>> expander) { return expand(expander, Queues.SMALL_BUFFER_SIZE); }
Evaluate each source value against the given Predicate. If the predicate test succeeds, the value is emitted. If the predicate test fails, the value is ignored and a request of 1 is made upstream.

Params:
@reactor.errorModeThis operator supports resuming on errors (including when fusion is enabled). Exceptions thrown by the predicate are considered as if the predicate returned false: they cause the source value to be dropped and a new element (request(1)) being requested from upstream.
@reactor.discardThis operator discards elements that do not match the filter. It also discards elements internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a new Flux containing only values that pass the predicate test
/** * Evaluate each source value against the given {@link Predicate}. If the predicate test succeeds, the value is * emitted. If the predicate test fails, the value is ignored and a request of 1 is made upstream. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/filter.png" alt=""> * * @param p the {@link Predicate} to test values against * * @reactor.errorMode This operator supports {@link #onErrorContinue(BiConsumer) resuming on errors} * (including when fusion is enabled). Exceptions thrown by the predicate are * considered as if the predicate returned false: they cause the source value to be * dropped and a new element ({@code request(1)}) being requested from upstream. * * @reactor.discard This operator discards elements that do not match the filter. It * also discards elements internally queued for backpressure upon cancellation or error triggered by a data signal. * * @return a new {@link Flux} containing only values that pass the predicate test */
public final Flux<T> filter(Predicate<? super T> p) { if (this instanceof Fuseable) { return onAssembly(new FluxFilterFuseable<>(this, p)); } return onAssembly(new FluxFilter<>(this, p)); }
Test each value emitted by this Flux asynchronously using a generated Publisher<Boolean> test. A value is replayed if the first item emitted by its corresponding test is true. It is dropped if its test is either empty or its first emitted value is false.

Note that only the first value of the test publisher is considered, and unless it is a Mono, test will be cancelled after receiving that first value. Test publishers are generated and subscribed to in sequence.

Params:
  • asyncPredicate – the function generating a Publisher of Boolean for each value, to filter the Flux with
@reactor.discardThis operator discards elements that do not match the filter. It also discards elements internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a filtered Flux
/** * Test each value emitted by this {@link Flux} asynchronously using a generated * {@code Publisher<Boolean>} test. A value is replayed if the first item emitted * by its corresponding test is {@literal true}. It is dropped if its test is either * empty or its first emitted value is {@literal false}. * <p> * Note that only the first value of the test publisher is considered, and unless it * is a {@link Mono}, test will be cancelled after receiving that first value. Test * publishers are generated and subscribed to in sequence. * * @reactor.discard This operator discards elements that do not match the filter. It * also discards elements internally queued for backpressure upon cancellation or error triggered by a data signal. * * @param asyncPredicate the function generating a {@link Publisher} of {@link Boolean} * for each value, to filter the Flux with * @return a filtered {@link Flux} */
public final Flux<T> filterWhen(Function<? super T, ? extends Publisher<Boolean>> asyncPredicate) { return filterWhen(asyncPredicate, Queues.SMALL_BUFFER_SIZE); }
Test each value emitted by this Flux asynchronously using a generated Publisher<Boolean> test. A value is replayed if the first item emitted by its corresponding test is true. It is dropped if its test is either empty or its first emitted value is false.

Note that only the first value of the test publisher is considered, and unless it is a Mono, test will be cancelled after receiving that first value. Test publishers are generated and subscribed to in sequence.

Params:
  • asyncPredicate – the function generating a Publisher of Boolean for each value, to filter the Flux with
  • bufferSize – the maximum expected number of values to hold pending a result of their respective asynchronous predicates, rounded to the next power of two. This is capped depending on the size of the heap and the JVM limits, so be careful with large values (although eg. 65536 should still be fine). Also serves as the initial request size for the source.
@reactor.discardThis operator discards elements that do not match the filter. It also discards elements internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a filtered Flux
/** * Test each value emitted by this {@link Flux} asynchronously using a generated * {@code Publisher<Boolean>} test. A value is replayed if the first item emitted * by its corresponding test is {@literal true}. It is dropped if its test is either * empty or its first emitted value is {@literal false}. * <p> * Note that only the first value of the test publisher is considered, and unless it * is a {@link Mono}, test will be cancelled after receiving that first value. Test * publishers are generated and subscribed to in sequence. * * @reactor.discard This operator discards elements that do not match the filter. It * also discards elements internally queued for backpressure upon cancellation or error triggered by a data signal. * * @param asyncPredicate the function generating a {@link Publisher} of {@link Boolean} * for each value, to filter the Flux with * @param bufferSize the maximum expected number of values to hold pending a result of * their respective asynchronous predicates, rounded to the next power of two. This is * capped depending on the size of the heap and the JVM limits, so be careful with * large values (although eg. {@literal 65536} should still be fine). Also serves as * the initial request size for the source. * @return a filtered {@link Flux} */
public final Flux<T> filterWhen(Function<? super T, ? extends Publisher<Boolean>> asyncPredicate, int bufferSize) { return onAssembly(new FluxFilterWhen<>(this, asyncPredicate, bufferSize)); }
Transform the elements emitted by this Flux asynchronously into Publishers, then flatten these inner publishers into a single Flux through merging, which allow them to interleave.

There are three dimensions to this operator that can be compared with flatMapSequential and concatMap:

  • Generation of inners and subscription: this operator is eagerly subscribing to its inners.
  • Ordering of the flattened values: this operator does not necessarily preserve original ordering, as inner element are flattened as they arrive.
  • Interleaving: this operator lets values from different inners interleave (similar to merging the inner sequences).

Params:
Type parameters:
  • <R> – the merged output sequence type
@reactor.discardThis operator discards elements internally queued for backpressure upon cancellation or error triggered by a data signal.
@reactor.errorModeThis operator supports resuming on errors in the mapper Function. Exceptions thrown by the mapper then behave as if it had mapped the value to an empty publisher. If the mapper does map to a scalar publisher (an optimization in which the value can be resolved immediately without subscribing to the publisher, e.g. a Mono.fromCallable(Callable<? extends Object>)) but said publisher throws, this can be resumed from in the same manner.
Returns:a new Flux
/** * Transform the elements emitted by this {@link Flux} asynchronously into Publishers, * then flatten these inner publishers into a single {@link Flux} through merging, * which allow them to interleave. * <p> * There are three dimensions to this operator that can be compared with * {@link #flatMapSequential(Function) flatMapSequential} and {@link #concatMap(Function) concatMap}: * <ul> * <li><b>Generation of inners and subscription</b>: this operator is eagerly * subscribing to its inners.</li> * <li><b>Ordering of the flattened values</b>: this operator does not necessarily preserve * original ordering, as inner element are flattened as they arrive.</li> * <li><b>Interleaving</b>: this operator lets values from different inners interleave * (similar to merging the inner sequences).</li> * </ul> * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/flatmap.png" alt=""> * <p> * * @reactor.discard This operator discards elements internally queued for backpressure upon cancellation or error triggered by a data signal. * * @param mapper the {@link Function} to transform input sequence into N sequences {@link Publisher} * @param <R> the merged output sequence type * * @reactor.errorMode This operator supports {@link #onErrorContinue(BiConsumer) resuming on errors} * in the mapper {@link Function}. Exceptions thrown by the mapper then behave as if * it had mapped the value to an empty publisher. If the mapper does map to a scalar * publisher (an optimization in which the value can be resolved immediately without * subscribing to the publisher, e.g. a {@link Mono#fromCallable(Callable)}) but said * publisher throws, this can be resumed from in the same manner. * * @return a new {@link Flux} */
public final <R> Flux<R> flatMap(Function<? super T, ? extends Publisher<? extends R>> mapper) { return flatMap(mapper, Queues.SMALL_BUFFER_SIZE, Queues .XS_BUFFER_SIZE); }
Transform the elements emitted by this Flux asynchronously into Publishers, then flatten these inner publishers into a single Flux through merging, which allow them to interleave.

There are three dimensions to this operator that can be compared with flatMapSequential and concatMap:

  • Generation of inners and subscription: this operator is eagerly subscribing to its inners.
  • Ordering of the flattened values: this operator does not necessarily preserve original ordering, as inner element are flattened as they arrive.
  • Interleaving: this operator lets values from different inners interleave (similar to merging the inner sequences).
The concurrency argument allows to control how many Publisher can be subscribed to and merged in parallel. In turn, that argument shows the size of the first Subscription.request to the upstream.

Params:
  • mapper – the Function to transform input sequence into N sequences Publisher
  • concurrency – the maximum number of in-flight inner sequences
Type parameters:
  • <V> – the merged output sequence type
@reactor.discardThis operator discards elements internally queued for backpressure upon cancellation or error triggered by a data signal.
@reactor.errorModeThis operator supports resuming on errors in the mapper Function. Exceptions thrown by the mapper then behave as if it had mapped the value to an empty publisher. If the mapper does map to a scalar publisher (an optimization in which the value can be resolved immediately without subscribing to the publisher, e.g. a Mono.fromCallable(Callable<? extends Object>)) but said publisher throws, this can be resumed from in the same manner.
Returns:a new Flux
/** * Transform the elements emitted by this {@link Flux} asynchronously into Publishers, * then flatten these inner publishers into a single {@link Flux} through merging, * which allow them to interleave. * <p> * There are three dimensions to this operator that can be compared with * {@link #flatMapSequential(Function) flatMapSequential} and {@link #concatMap(Function) concatMap}: * <ul> * <li><b>Generation of inners and subscription</b>: this operator is eagerly * subscribing to its inners.</li> * <li><b>Ordering of the flattened values</b>: this operator does not necessarily preserve * original ordering, as inner element are flattened as they arrive.</li> * <li><b>Interleaving</b>: this operator lets values from different inners interleave * (similar to merging the inner sequences).</li> * </ul> * The concurrency argument allows to control how many {@link Publisher} can be * subscribed to and merged in parallel. In turn, that argument shows the size of * the first {@link Subscription#request} to the upstream. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/flatmapc.png" alt=""> * * @reactor.discard This operator discards elements internally queued for backpressure upon cancellation or error triggered by a data signal. * * @param mapper the {@link Function} to transform input sequence into N sequences {@link Publisher} * @param concurrency the maximum number of in-flight inner sequences * @param <V> the merged output sequence type * * @reactor.errorMode This operator supports {@link #onErrorContinue(BiConsumer) resuming on errors} * in the mapper {@link Function}. Exceptions thrown by the mapper then behave as if * it had mapped the value to an empty publisher. If the mapper does map to a scalar * publisher (an optimization in which the value can be resolved immediately without * subscribing to the publisher, e.g. a {@link Mono#fromCallable(Callable)}) but said * publisher throws, this can be resumed from in the same manner. * * @return a new {@link Flux} */
public final <V> Flux<V> flatMap(Function<? super T, ? extends Publisher<? extends V>> mapper, int concurrency) { return flatMap(mapper, concurrency, Queues.XS_BUFFER_SIZE); }
Transform the elements emitted by this Flux asynchronously into Publishers, then flatten these inner publishers into a single Flux through merging, which allow them to interleave.

There are three dimensions to this operator that can be compared with flatMapSequential and concatMap:

  • Generation of inners and subscription: this operator is eagerly subscribing to its inners.
  • Ordering of the flattened values: this operator does not necessarily preserve original ordering, as inner element are flattened as they arrive.
  • Interleaving: this operator lets values from different inners interleave (similar to merging the inner sequences).
The concurrency argument allows to control how many Publisher can be subscribed to and merged in parallel. In turn, that argument shows the size of the first Subscription.request to the upstream. The prefetch argument allows to give an arbitrary prefetch size to the merged Publisher (in other words prefetch size means the size of the first Subscription.request to the merged Publisher).

Params:
  • mapper – the Function to transform input sequence into N sequences Publisher
  • concurrency – the maximum number of in-flight inner sequences
  • prefetch – the maximum in-flight elements from each inner Publisher sequence
Type parameters:
  • <V> – the merged output sequence type
@reactor.discardThis operator discards elements internally queued for backpressure upon cancellation or error triggered by a data signal.
@reactor.errorModeThis operator supports resuming on errors in the mapper Function. Exceptions thrown by the mapper then behave as if it had mapped the value to an empty publisher. If the mapper does map to a scalar publisher (an optimization in which the value can be resolved immediately without subscribing to the publisher, e.g. a Mono.fromCallable(Callable<? extends Object>)) but said publisher throws, this can be resumed from in the same manner.
Returns:a merged Flux
/** * Transform the elements emitted by this {@link Flux} asynchronously into Publishers, * then flatten these inner publishers into a single {@link Flux} through merging, * which allow them to interleave. * <p> * There are three dimensions to this operator that can be compared with * {@link #flatMapSequential(Function) flatMapSequential} and {@link #concatMap(Function) concatMap}: * <ul> * <li><b>Generation of inners and subscription</b>: this operator is eagerly * subscribing to its inners.</li> * <li><b>Ordering of the flattened values</b>: this operator does not necessarily preserve * original ordering, as inner element are flattened as they arrive.</li> * <li><b>Interleaving</b>: this operator lets values from different inners interleave * (similar to merging the inner sequences).</li> * </ul> * The concurrency argument allows to control how many {@link Publisher} can be * subscribed to and merged in parallel. In turn, that argument shows the size of * the first {@link Subscription#request} to the upstream. * The prefetch argument allows to give an arbitrary prefetch size to the merged * {@link Publisher} (in other words prefetch size means the size of the first * {@link Subscription#request} to the merged {@link Publisher}). * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/flatmapc.png" alt=""> * * @reactor.discard This operator discards elements internally queued for backpressure upon cancellation or error triggered by a data signal. * * @param mapper the {@link Function} to transform input sequence into N sequences {@link Publisher} * @param concurrency the maximum number of in-flight inner sequences * @param prefetch the maximum in-flight elements from each inner {@link Publisher} sequence * @param <V> the merged output sequence type * * @reactor.errorMode This operator supports {@link #onErrorContinue(BiConsumer) resuming on errors} * in the mapper {@link Function}. Exceptions thrown by the mapper then behave as if * it had mapped the value to an empty publisher. If the mapper does map to a scalar * publisher (an optimization in which the value can be resolved immediately without * subscribing to the publisher, e.g. a {@link Mono#fromCallable(Callable)}) but said * publisher throws, this can be resumed from in the same manner. * * @return a merged {@link Flux} */
public final <V> Flux<V> flatMap(Function<? super T, ? extends Publisher<? extends V>> mapper, int concurrency, int prefetch) { return flatMap(mapper, false, concurrency, prefetch); }
Transform the elements emitted by this Flux asynchronously into Publishers, then flatten these inner publishers into a single Flux through merging, which allow them to interleave.

There are three dimensions to this operator that can be compared with flatMapSequential and concatMap:

  • Generation of inners and subscription: this operator is eagerly subscribing to its inners.
  • Ordering of the flattened values: this operator does not necessarily preserve original ordering, as inner element are flattened as they arrive.
  • Interleaving: this operator lets values from different inners interleave (similar to merging the inner sequences).
The concurrency argument allows to control how many Publisher can be subscribed to and merged in parallel. The prefetch argument allows to give an arbitrary prefetch size to the merged Publisher. This variant will delay any error until after the rest of the flatMap backlog has been processed.

Params:
  • mapper – the Function to transform input sequence into N sequences Publisher
  • concurrency – the maximum number of in-flight inner sequences
  • prefetch – the maximum in-flight elements from each inner Publisher sequence
Type parameters:
  • <V> – the merged output sequence type
@reactor.discardThis operator discards elements internally queued for backpressure upon cancellation or error triggered by a data signal.
@reactor.errorModeThis operator supports resuming on errors in the mapper Function. Exceptions thrown by the mapper then behave as if it had mapped the value to an empty publisher. If the mapper does map to a scalar publisher (an optimization in which the value can be resolved immediately without subscribing to the publisher, e.g. a Mono.fromCallable(Callable<? extends Object>)) but said publisher throws, this can be resumed from in the same manner.
Returns:a merged Flux
/** * Transform the elements emitted by this {@link Flux} asynchronously into Publishers, * then flatten these inner publishers into a single {@link Flux} through merging, * which allow them to interleave. * <p> * There are three dimensions to this operator that can be compared with * {@link #flatMapSequential(Function) flatMapSequential} and {@link #concatMap(Function) concatMap}: * <ul> * <li><b>Generation of inners and subscription</b>: this operator is eagerly * subscribing to its inners.</li> * <li><b>Ordering of the flattened values</b>: this operator does not necessarily preserve * original ordering, as inner element are flattened as they arrive.</li> * <li><b>Interleaving</b>: this operator lets values from different inners interleave * (similar to merging the inner sequences).</li> * </ul> * The concurrency argument allows to control how many {@link Publisher} can be * subscribed to and merged in parallel. The prefetch argument allows to give an * arbitrary prefetch size to the merged {@link Publisher}. This variant will delay * any error until after the rest of the flatMap backlog has been processed. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/flatmapc.png" alt=""> * * @reactor.discard This operator discards elements internally queued for backpressure upon cancellation or error triggered by a data signal. * * @param mapper the {@link Function} to transform input sequence into N sequences {@link Publisher} * @param concurrency the maximum number of in-flight inner sequences * @param prefetch the maximum in-flight elements from each inner {@link Publisher} sequence * @param <V> the merged output sequence type * * @reactor.errorMode This operator supports {@link #onErrorContinue(BiConsumer) resuming on errors} * in the mapper {@link Function}. Exceptions thrown by the mapper then behave as if * it had mapped the value to an empty publisher. If the mapper does map to a scalar * publisher (an optimization in which the value can be resolved immediately without * subscribing to the publisher, e.g. a {@link Mono#fromCallable(Callable)}) but said * publisher throws, this can be resumed from in the same manner. * * @return a merged {@link Flux} */
public final <V> Flux<V> flatMapDelayError(Function<? super T, ? extends Publisher<? extends V>> mapper, int concurrency, int prefetch) { return flatMap(mapper, true, concurrency, prefetch); }
Transform the signals emitted by this Flux asynchronously into Publishers, then flatten these inner publishers into a single Flux through merging, which allow them to interleave. Note that at least one of the signal mappers must be provided, and all provided mappers must produce a publisher.

There are three dimensions to this operator that can be compared with flatMapSequential and concatMap:

  • Generation of inners and subscription: this operator is eagerly subscribing to its inners.
  • Ordering of the flattened values: this operator does not necessarily preserve original ordering, as inner element are flattened as they arrive.
  • Interleaving: this operator lets values from different inners interleave (similar to merging the inner sequences).

OnError will be transformed into completion signal after its mapping callback has been applied.

Params:
  • mapperOnNext – the Function to call on next data and returning a sequence to merge. Use null to ignore (provided at least one other mapper is specified).
  • mapperOnError – the Function to call on error signal and returning a sequence to merge. Use null to ignore (provided at least one other mapper is specified).
  • mapperOnComplete – the Function to call on complete signal and returning a sequence to merge. Use null to ignore (provided at least one other mapper is specified).
Type parameters:
Returns:a new Flux
/** * Transform the signals emitted by this {@link Flux} asynchronously into Publishers, * then flatten these inner publishers into a single {@link Flux} through merging, * which allow them to interleave. Note that at least one of the signal mappers must * be provided, and all provided mappers must produce a publisher. * <p> * There are three dimensions to this operator that can be compared with * {@link #flatMapSequential(Function) flatMapSequential} and {@link #concatMap(Function) concatMap}: * <ul> * <li><b>Generation of inners and subscription</b>: this operator is eagerly * subscribing to its inners.</li> * <li><b>Ordering of the flattened values</b>: this operator does not necessarily preserve * original ordering, as inner element are flattened as they arrive.</li> * <li><b>Interleaving</b>: this operator lets values from different inners interleave * (similar to merging the inner sequences).</li> * </ul> * <p> * OnError will be transformed into completion signal after its mapping callback has been applied. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/flatmaps.png" alt=""> * <p> * * @param mapperOnNext the {@link Function} to call on next data and returning a sequence to merge. * Use {@literal null} to ignore (provided at least one other mapper is specified). * @param mapperOnError the {@link Function} to call on error signal and returning a sequence to merge. * Use {@literal null} to ignore (provided at least one other mapper is specified). * @param mapperOnComplete the {@link Function} to call on complete signal and returning a sequence to merge. * Use {@literal null} to ignore (provided at least one other mapper is specified). * @param <R> the output {@link Publisher} type target * * @return a new {@link Flux} */
public final <R> Flux<R> flatMap( @Nullable Function<? super T, ? extends Publisher<? extends R>> mapperOnNext, @Nullable Function<? super Throwable, ? extends Publisher<? extends R>> mapperOnError, @Nullable Supplier<? extends Publisher<? extends R>> mapperOnComplete) { return onAssembly(new FluxFlatMap<>( new FluxMapSignal<>(this, mapperOnNext, mapperOnError, mapperOnComplete), identityFunction(), false, Queues.XS_BUFFER_SIZE, Queues.xs(), Queues.XS_BUFFER_SIZE, Queues.xs() )); }
Transform the items emitted by this Flux into Iterable, then flatten the elements from those by merging them into a single Flux.

Note that unlike flatMap(Function) and concatMap(Function), with Iterable there is no notion of eager vs lazy inner subscription. The content of the Iterables are all played sequentially. Thus flatMapIterable and concatMapIterable are equivalent offered as a discoverability improvement for users that explore the API with the concat vs flatMap expectation.

Params:
Type parameters:
  • <R> – the merged output sequence type
@reactor.discardThis operator discards elements internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a concatenation of the values from the Iterables obtained from each element in this Flux
/** * Transform the items emitted by this {@link Flux} into {@link Iterable}, then flatten the elements from those by * merging them into a single {@link Flux}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concatmap.png" alt=""> * <p> * Note that unlike {@link #flatMap(Function)} and {@link #concatMap(Function)}, with Iterable there is * no notion of eager vs lazy inner subscription. The content of the Iterables are all played sequentially. * Thus {@code flatMapIterable} and {@code concatMapIterable} are equivalent offered as a discoverability * improvement for users that explore the API with the concat vs flatMap expectation. * * @reactor.discard This operator discards elements internally queued for backpressure upon cancellation or error triggered by a data signal. * * @param mapper the {@link Function} to transform input sequence into N {@link Iterable} * @param <R> the merged output sequence type * * @return a concatenation of the values from the Iterables obtained from each element in this {@link Flux} */
public final <R> Flux<R> flatMapIterable(Function<? super T, ? extends Iterable<? extends R>> mapper) { return flatMapIterable(mapper, Queues.SMALL_BUFFER_SIZE); }
Transform the items emitted by this Flux into Iterable, then flatten the emissions from those by merging them into a single Flux. The prefetch argument allows to give an arbitrary prefetch size to the merged Iterable.

Note that unlike flatMap(Function) and concatMap(Function), with Iterable there is no notion of eager vs lazy inner subscription. The content of the Iterables are all played sequentially. Thus flatMapIterable and concatMapIterable are equivalent offered as a discoverability improvement for users that explore the API with the concat vs flatMap expectation.

Params:
  • mapper – the Function to transform input sequence into N Iterable
  • prefetch – the maximum in-flight elements from each inner Iterable sequence
Type parameters:
  • <R> – the merged output sequence type
@reactor.discardThis operator discards elements internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a concatenation of the values from the Iterables obtained from each element in this Flux
/** * Transform the items emitted by this {@link Flux} into {@link Iterable}, then flatten the emissions from those by * merging them into a single {@link Flux}. The prefetch argument allows to give an * arbitrary prefetch size to the merged {@link Iterable}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/concatmap.png" alt=""> * <p> * Note that unlike {@link #flatMap(Function)} and {@link #concatMap(Function)}, with Iterable there is * no notion of eager vs lazy inner subscription. The content of the Iterables are all played sequentially. * Thus {@code flatMapIterable} and {@code concatMapIterable} are equivalent offered as a discoverability * improvement for users that explore the API with the concat vs flatMap expectation. * * @reactor.discard This operator discards elements internally queued for backpressure upon cancellation or error triggered by a data signal. * * @param mapper the {@link Function} to transform input sequence into N {@link Iterable} * @param prefetch the maximum in-flight elements from each inner {@link Iterable} sequence * @param <R> the merged output sequence type * * @return a concatenation of the values from the Iterables obtained from each element in this {@link Flux} */
public final <R> Flux<R> flatMapIterable(Function<? super T, ? extends Iterable<? extends R>> mapper, int prefetch) { return onAssembly(new FluxFlattenIterable<>(this, mapper, prefetch, Queues.get(prefetch))); }
Transform the elements emitted by this Flux asynchronously into Publishers, then flatten these inner publishers into a single Flux, but merge them in the order of their source element.

There are three dimensions to this operator that can be compared with flatMap and concatMap:

  • Generation of inners and subscription: this operator is eagerly subscribing to its inners (like flatMap).
  • Ordering of the flattened values: this operator queues elements from late inners until all elements from earlier inners have been emitted, thus emitting inner sequences as a whole, in an order that matches their source's order.
  • Interleaving: this operator does not let values from different inners interleave (similar looking result to concatMap, but due to queueing of values that would have been interleaved otherwise).

That is to say, whenever a source element is emitted it is transformed to an inner Publisher. However, if such an early inner takes more time to complete than subsequent faster inners, the data from these faster inners will be queued until the earlier inner completes, so as to maintain source ordering.

Params:
Type parameters:
  • <R> – the merged output sequence type
Returns:a merged Flux, subscribing early but keeping the original ordering
/** * Transform the elements emitted by this {@link Flux} asynchronously into Publishers, * then flatten these inner publishers into a single {@link Flux}, but merge them in * the order of their source element. * <p> * There are three dimensions to this operator that can be compared with * {@link #flatMap(Function) flatMap} and {@link #concatMap(Function) concatMap}: * <ul> * <li><b>Generation of inners and subscription</b>: this operator is eagerly * subscribing to its inners (like flatMap).</li> * <li><b>Ordering of the flattened values</b>: this operator queues elements from * late inners until all elements from earlier inners have been emitted, thus emitting * inner sequences as a whole, in an order that matches their source's order.</li> * <li><b>Interleaving</b>: this operator does not let values from different inners * interleave (similar looking result to concatMap, but due to queueing of values * that would have been interleaved otherwise).</li> * </ul> * * <p> * That is to say, whenever a source element is emitted it is transformed to an inner * {@link Publisher}. However, if such an early inner takes more time to complete than * subsequent faster inners, the data from these faster inners will be queued until * the earlier inner completes, so as to maintain source ordering. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/flatmapsequential.png" alt=""> * * @param mapper the {@link Function} to transform input sequence into N sequences {@link Publisher} * @param <R> the merged output sequence type * * @return a merged {@link Flux}, subscribing early but keeping the original ordering */
public final <R> Flux<R> flatMapSequential(Function<? super T, ? extends Publisher<? extends R>> mapper) { return flatMapSequential(mapper, Queues.SMALL_BUFFER_SIZE); }
Transform the elements emitted by this Flux asynchronously into Publishers, then flatten these inner publishers into a single Flux, but merge them in the order of their source element.

There are three dimensions to this operator that can be compared with flatMap and concatMap:

  • Generation of inners and subscription: this operator is eagerly subscribing to its inners (like flatMap).
  • Ordering of the flattened values: this operator queues elements from late inners until all elements from earlier inners have been emitted, thus emitting inner sequences as a whole, in an order that matches their source's order.
  • Interleaving: this operator does not let values from different inners interleave (similar looking result to concatMap, but due to queueing of values that would have been interleaved otherwise).

That is to say, whenever a source element is emitted it is transformed to an inner Publisher. However, if such an early inner takes more time to complete than subsequent faster inners, the data from these faster inners will be queued until the earlier inner completes, so as to maintain source ordering.

The concurrency argument allows to control how many merged Publisher can happen in parallel.

Params:
  • mapper – the Function to transform input sequence into N sequences Publisher
  • maxConcurrency – the maximum number of in-flight inner sequences
Type parameters:
  • <R> – the merged output sequence type
Returns:a merged Flux, subscribing early but keeping the original ordering
/** * Transform the elements emitted by this {@link Flux} asynchronously into Publishers, * then flatten these inner publishers into a single {@link Flux}, but merge them in * the order of their source element. * <p> * There are three dimensions to this operator that can be compared with * {@link #flatMap(Function) flatMap} and {@link #concatMap(Function) concatMap}: * <ul> * <li><b>Generation of inners and subscription</b>: this operator is eagerly * subscribing to its inners (like flatMap).</li> * <li><b>Ordering of the flattened values</b>: this operator queues elements from * late inners until all elements from earlier inners have been emitted, thus emitting * inner sequences as a whole, in an order that matches their source's order.</li> * <li><b>Interleaving</b>: this operator does not let values from different inners * interleave (similar looking result to concatMap, but due to queueing of values * that would have been interleaved otherwise).</li> * </ul> * * <p> * That is to say, whenever a source element is emitted it is transformed to an inner * {@link Publisher}. However, if such an early inner takes more time to complete than * subsequent faster inners, the data from these faster inners will be queued until * the earlier inner completes, so as to maintain source ordering. * * <p> * The concurrency argument allows to control how many merged {@link Publisher} can happen in parallel. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/flatmapsequential.png" alt=""> * * @param mapper the {@link Function} to transform input sequence into N sequences {@link Publisher} * @param maxConcurrency the maximum number of in-flight inner sequences * @param <R> the merged output sequence type * * @return a merged {@link Flux}, subscribing early but keeping the original ordering */
public final <R> Flux<R> flatMapSequential(Function<? super T, ? extends Publisher<? extends R>> mapper, int maxConcurrency) { return flatMapSequential(mapper, maxConcurrency, Queues.XS_BUFFER_SIZE); }
Transform the elements emitted by this Flux asynchronously into Publishers, then flatten these inner publishers into a single Flux, but merge them in the order of their source element.

There are three dimensions to this operator that can be compared with flatMap and concatMap:

  • Generation of inners and subscription: this operator is eagerly subscribing to its inners (like flatMap).
  • Ordering of the flattened values: this operator queues elements from late inners until all elements from earlier inners have been emitted, thus emitting inner sequences as a whole, in an order that matches their source's order.
  • Interleaving: this operator does not let values from different inners interleave (similar looking result to concatMap, but due to queueing of values that would have been interleaved otherwise).

That is to say, whenever a source element is emitted it is transformed to an inner Publisher. However, if such an early inner takes more time to complete than subsequent faster inners, the data from these faster inners will be queued until the earlier inner completes, so as to maintain source ordering.

The concurrency argument allows to control how many merged Publisher can happen in parallel. The prefetch argument allows to give an arbitrary prefetch size to the merged Publisher.

Params:
  • mapper – the Function to transform input sequence into N sequences Publisher
  • maxConcurrency – the maximum number of in-flight inner sequences
  • prefetch – the maximum in-flight elements from each inner Publisher sequence
Type parameters:
  • <R> – the merged output sequence type
Returns:a merged Flux, subscribing early but keeping the original ordering
/** * Transform the elements emitted by this {@link Flux} asynchronously into Publishers, * then flatten these inner publishers into a single {@link Flux}, but merge them in * the order of their source element. * <p> * There are three dimensions to this operator that can be compared with * {@link #flatMap(Function) flatMap} and {@link #concatMap(Function) concatMap}: * <ul> * <li><b>Generation of inners and subscription</b>: this operator is eagerly * subscribing to its inners (like flatMap).</li> * <li><b>Ordering of the flattened values</b>: this operator queues elements from * late inners until all elements from earlier inners have been emitted, thus emitting * inner sequences as a whole, in an order that matches their source's order.</li> * <li><b>Interleaving</b>: this operator does not let values from different inners * interleave (similar looking result to concatMap, but due to queueing of values * that would have been interleaved otherwise).</li> * </ul> * * <p> * That is to say, whenever a source element is emitted it is transformed to an inner * {@link Publisher}. However, if such an early inner takes more time to complete than * subsequent faster inners, the data from these faster inners will be queued until * the earlier inner completes, so as to maintain source ordering. * * <p> * The concurrency argument allows to control how many merged {@link Publisher} * can happen in parallel. The prefetch argument allows to give an arbitrary prefetch * size to the merged {@link Publisher}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/flatmapsequential.png" alt=""> * * @param mapper the {@link Function} to transform input sequence into N sequences {@link Publisher} * @param maxConcurrency the maximum number of in-flight inner sequences * @param prefetch the maximum in-flight elements from each inner {@link Publisher} sequence * @param <R> the merged output sequence type * * @return a merged {@link Flux}, subscribing early but keeping the original ordering */
public final <R> Flux<R> flatMapSequential(Function<? super T, ? extends Publisher<? extends R>> mapper, int maxConcurrency, int prefetch) { return flatMapSequential(mapper, false, maxConcurrency, prefetch); }
Transform the elements emitted by this Flux asynchronously into Publishers, then flatten these inner publishers into a single Flux, but merge them in the order of their source element.

There are three dimensions to this operator that can be compared with flatMap and concatMap:

  • Generation of inners and subscription: this operator is eagerly subscribing to its inners (like flatMap).
  • Ordering of the flattened values: this operator queues elements from late inners until all elements from earlier inners have been emitted, thus emitting inner sequences as a whole, in an order that matches their source's order.
  • Interleaving: this operator does not let values from different inners interleave (similar looking result to concatMap, but due to queueing of values that would have been interleaved otherwise).

That is to say, whenever a source element is emitted it is transformed to an inner Publisher. However, if such an early inner takes more time to complete than subsequent faster inners, the data from these faster inners will be queued until the earlier inner completes, so as to maintain source ordering.

The concurrency argument allows to control how many merged Publisher can happen in parallel. The prefetch argument allows to give an arbitrary prefetch size to the merged Publisher. This variant will delay any error until after the rest of the flatMap backlog has been processed.

Params:
  • mapper – the Function to transform input sequence into N sequences Publisher
  • maxConcurrency – the maximum number of in-flight inner sequences
  • prefetch – the maximum in-flight elements from each inner Publisher sequence
Type parameters:
  • <R> – the merged output sequence type
Returns:a merged Flux, subscribing early but keeping the original ordering
/** * Transform the elements emitted by this {@link Flux} asynchronously into Publishers, * then flatten these inner publishers into a single {@link Flux}, but merge them in * the order of their source element. * <p> * There are three dimensions to this operator that can be compared with * {@link #flatMap(Function) flatMap} and {@link #concatMap(Function) concatMap}: * <ul> * <li><b>Generation of inners and subscription</b>: this operator is eagerly * subscribing to its inners (like flatMap).</li> * <li><b>Ordering of the flattened values</b>: this operator queues elements from * late inners until all elements from earlier inners have been emitted, thus emitting * inner sequences as a whole, in an order that matches their source's order.</li> * <li><b>Interleaving</b>: this operator does not let values from different inners * interleave (similar looking result to concatMap, but due to queueing of values * that would have been interleaved otherwise).</li> * </ul> * * <p> * That is to say, whenever a source element is emitted it is transformed to an inner * {@link Publisher}. However, if such an early inner takes more time to complete than * subsequent faster inners, the data from these faster inners will be queued until * the earlier inner completes, so as to maintain source ordering. * * <p> * The concurrency argument allows to control how many merged {@link Publisher} * can happen in parallel. The prefetch argument allows to give an arbitrary prefetch * size to the merged {@link Publisher}. This variant will delay any error until after the * rest of the flatMap backlog has been processed. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/flatmapsequential.png" alt=""> * * @param mapper the {@link Function} to transform input sequence into N sequences {@link Publisher} * @param maxConcurrency the maximum number of in-flight inner sequences * @param prefetch the maximum in-flight elements from each inner {@link Publisher} sequence * @param <R> the merged output sequence type * * @return a merged {@link Flux}, subscribing early but keeping the original ordering */
public final <R> Flux<R> flatMapSequentialDelayError(Function<? super T, ? extends Publisher<? extends R>> mapper, int maxConcurrency, int prefetch) { return flatMapSequential(mapper, true, maxConcurrency, prefetch); }
The prefetch configuration of the Flux
Returns:the prefetch configuration of the Flux, -1 if unspecified
/** * The prefetch configuration of the {@link Flux} * @return the prefetch configuration of the {@link Flux}, -1 if unspecified */
public int getPrefetch() { return -1; }
Divide this sequence into dynamically created Flux (or groups) for each unique key, as produced by the provided keyMapper Function. Note that groupBy works best with a low cardinality of groups, so chose your keyMapper function accordingly.

The groups need to be drained and consumed downstream for groupBy to work correctly. Notably when the criteria produces a large amount of groups, it can lead to hanging if the groups are not suitably consumed downstream (eg. due to a flatMap with a maxConcurrency parameter that is set too low).

Params:
  • keyMapper – the key mapping Function that evaluates an incoming data and returns a key.
Type parameters:
  • <K> – the key type extracted from each value of this sequence
Returns:a Flux of GroupedFlux grouped sequences
/** * Divide this sequence into dynamically created {@link Flux} (or groups) for each * unique key, as produced by the provided keyMapper {@link Function}. Note that * groupBy works best with a low cardinality of groups, so chose your keyMapper * function accordingly. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/groupby.png" alt=""> * * <p> * The groups need to be drained and consumed downstream for groupBy to work correctly. * Notably when the criteria produces a large amount of groups, it can lead to hanging * if the groups are not suitably consumed downstream (eg. due to a {@code flatMap} * with a {@code maxConcurrency} parameter that is set too low). * * @param keyMapper the key mapping {@link Function} that evaluates an incoming data and returns a key. * @param <K> the key type extracted from each value of this sequence * * @return a {@link Flux} of {@link GroupedFlux} grouped sequences */
public final <K> Flux<GroupedFlux<K, T>> groupBy(Function<? super T, ? extends K> keyMapper) { return groupBy(keyMapper, identityFunction()); }
Divide this sequence into dynamically created Flux (or groups) for each unique key, as produced by the provided keyMapper Function. Note that groupBy works best with a low cardinality of groups, so chose your keyMapper function accordingly.

The groups need to be drained and consumed downstream for groupBy to work correctly. Notably when the criteria produces a large amount of groups, it can lead to hanging if the groups are not suitably consumed downstream (eg. due to a flatMap with a maxConcurrency parameter that is set too low).

Params:
  • keyMapper – the key mapping Function that evaluates an incoming data and returns a key.
  • prefetch – the number of values to prefetch from the source
Type parameters:
  • <K> – the key type extracted from each value of this sequence
Returns:a Flux of GroupedFlux grouped sequences
/** * Divide this sequence into dynamically created {@link Flux} (or groups) for each * unique key, as produced by the provided keyMapper {@link Function}. Note that * groupBy works best with a low cardinality of groups, so chose your keyMapper * function accordingly. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/groupby.png" alt=""> * * <p> * The groups need to be drained and consumed downstream for groupBy to work correctly. * Notably when the criteria produces a large amount of groups, it can lead to hanging * if the groups are not suitably consumed downstream (eg. due to a {@code flatMap} * with a {@code maxConcurrency} parameter that is set too low). * * @param keyMapper the key mapping {@link Function} that evaluates an incoming data and returns a key. * @param prefetch the number of values to prefetch from the source * @param <K> the key type extracted from each value of this sequence * * @return a {@link Flux} of {@link GroupedFlux} grouped sequences */
public final <K> Flux<GroupedFlux<K, T>> groupBy(Function<? super T, ? extends K> keyMapper, int prefetch) { return groupBy(keyMapper, identityFunction(), prefetch); }
Divide this sequence into dynamically created Flux (or groups) for each unique key, as produced by the provided keyMapper Function. Source elements are also mapped to a different value using the valueMapper. Note that groupBy works best with a low cardinality of groups, so chose your keyMapper function accordingly.

The groups need to be drained and consumed downstream for groupBy to work correctly. Notably when the criteria produces a large amount of groups, it can lead to hanging if the groups are not suitably consumed downstream (eg. due to a flatMap with a maxConcurrency parameter that is set too low).

Params:
  • keyMapper – the key mapping function that evaluates an incoming data and returns a key.
  • valueMapper – the value mapping function that evaluates which data to extract for re-routing.
Type parameters:
  • <K> – the key type extracted from each value of this sequence
  • <V> – the value type extracted from each value of this sequence
Returns:a Flux of GroupedFlux grouped sequences
/** * Divide this sequence into dynamically created {@link Flux} (or groups) for each * unique key, as produced by the provided keyMapper {@link Function}. Source elements * are also mapped to a different value using the {@code valueMapper}. Note that * groupBy works best with a low cardinality of groups, so chose your keyMapper * function accordingly. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/groupby.png" alt=""> * * <p> * The groups need to be drained and consumed downstream for groupBy to work correctly. * Notably when the criteria produces a large amount of groups, it can lead to hanging * if the groups are not suitably consumed downstream (eg. due to a {@code flatMap} * with a {@code maxConcurrency} parameter that is set too low). * * @param keyMapper the key mapping function that evaluates an incoming data and returns a key. * @param valueMapper the value mapping function that evaluates which data to extract for re-routing. * @param <K> the key type extracted from each value of this sequence * @param <V> the value type extracted from each value of this sequence * * @return a {@link Flux} of {@link GroupedFlux} grouped sequences * */
public final <K, V> Flux<GroupedFlux<K, V>> groupBy(Function<? super T, ? extends K> keyMapper, Function<? super T, ? extends V> valueMapper) { return groupBy(keyMapper, valueMapper, Queues.SMALL_BUFFER_SIZE); }
Divide this sequence into dynamically created Flux (or groups) for each unique key, as produced by the provided keyMapper Function. Source elements are also mapped to a different value using the valueMapper. Note that groupBy works best with a low cardinality of groups, so chose your keyMapper function accordingly.

The groups need to be drained and consumed downstream for groupBy to work correctly. Notably when the criteria produces a large amount of groups, it can lead to hanging if the groups are not suitably consumed downstream (eg. due to a flatMap with a maxConcurrency parameter that is set too low).

Params:
  • keyMapper – the key mapping function that evaluates an incoming data and returns a key.
  • valueMapper – the value mapping function that evaluates which data to extract for re-routing.
  • prefetch – the number of values to prefetch from the source
Type parameters:
  • <K> – the key type extracted from each value of this sequence
  • <V> – the value type extracted from each value of this sequence
Returns:a Flux of GroupedFlux grouped sequences
/** * Divide this sequence into dynamically created {@link Flux} (or groups) for each * unique key, as produced by the provided keyMapper {@link Function}. Source elements * are also mapped to a different value using the {@code valueMapper}. Note that * groupBy works best with a low cardinality of groups, so chose your keyMapper * function accordingly. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/groupby.png" alt=""> * * <p> * The groups need to be drained and consumed downstream for groupBy to work correctly. * Notably when the criteria produces a large amount of groups, it can lead to hanging * if the groups are not suitably consumed downstream (eg. due to a {@code flatMap} * with a {@code maxConcurrency} parameter that is set too low). * * @param keyMapper the key mapping function that evaluates an incoming data and returns a key. * @param valueMapper the value mapping function that evaluates which data to extract for re-routing. * @param prefetch the number of values to prefetch from the source * * @param <K> the key type extracted from each value of this sequence * @param <V> the value type extracted from each value of this sequence * * @return a {@link Flux} of {@link GroupedFlux} grouped sequences * */
public final <K, V> Flux<GroupedFlux<K, V>> groupBy(Function<? super T, ? extends K> keyMapper, Function<? super T, ? extends V> valueMapper, int prefetch) { return onAssembly(new FluxGroupBy<>(this, keyMapper, valueMapper, Queues.unbounded(prefetch), Queues.unbounded(prefetch), prefetch)); }
Map values from two Publishers into time windows and emit combination of values in case their windows overlap. The emitted elements are obtained by passing the value from this Flux and a Flux emitting the value from the other Publisher to a BiFunction.

There are no guarantees in what order the items get combined when multiple items from one or both source Publishers overlap.

Unlike join, items from the second Publisher will be provided as a Flux to the resultSelector.

Params:
  • other – the other Publisher to correlate items with
  • leftEnd – a function that returns a Publisher whose emissions indicate the time window for the source value to be considered
  • rightEnd – a function that returns a Publisher whose emissions indicate the time window for the right Publisher value to be considered
  • resultSelector – a function that takes an item emitted by this Flux and a Flux representation of the overlapping item from the other Publisher and returns the value to be emitted by the resulting Flux
Type parameters:
See Also:
Returns:a joining Flux
/** * Map values from two Publishers into time windows and emit combination of values * in case their windows overlap. The emitted elements are obtained by passing the * value from this {@link Flux} and a {@link Flux} emitting the value from the other * {@link Publisher} to a {@link BiFunction}. * <p> * There are no guarantees in what order the items get combined when multiple items from * one or both source Publishers overlap. * <p> * Unlike {@link Flux#join}, items from the second {@link Publisher} will be provided * as a {@link Flux} to the {@code resultSelector}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/groupjoin.png" alt=""> * * @param other the other {@link Publisher} to correlate items with * @param leftEnd a function that returns a Publisher whose emissions indicate the * time window for the source value to be considered * @param rightEnd a function that returns a Publisher whose emissions indicate the * time window for the {@code right} Publisher value to be considered * @param resultSelector a function that takes an item emitted by this {@link Flux} and * a {@link Flux} representation of the overlapping item from the other {@link Publisher} * and returns the value to be emitted by the resulting {@link Flux} * @param <TRight> the type of the elements from the right {@link Publisher} * @param <TLeftEnd> the type for this {@link Flux} window signals * @param <TRightEnd> the type for the right {@link Publisher} window signals * @param <R> the combined result type * * @return a joining {@link Flux} * @see #join(Publisher, Function, Function, BiFunction) */
public final <TRight, TLeftEnd, TRightEnd, R> Flux<R> groupJoin( Publisher<? extends TRight> other, Function<? super T, ? extends Publisher<TLeftEnd>> leftEnd, Function<? super TRight, ? extends Publisher<TRightEnd>> rightEnd, BiFunction<? super T, ? super Flux<TRight>, ? extends R> resultSelector ) { return onAssembly(new FluxGroupJoin<T, TRight, TLeftEnd, TRightEnd, R>( this, other, leftEnd, rightEnd, resultSelector, Queues.unbounded(Queues.XS_BUFFER_SIZE), Queues.unbounded(Queues.XS_BUFFER_SIZE))); }
Handle the items emitted by this Flux by calling a biconsumer with the output sink for each onNext. At most one SynchronousSink.next(Object) call must be performed and/or 0 or 1 SynchronousSink.error(Throwable) or SynchronousSink.complete().
Params:
Type parameters:
  • <R> – the transformed type
@reactor.errorModeThis operator supports resuming on errors (including when fusion is enabled) when the BiConsumer throws an exception or if an error is signaled explicitly via SynchronousSink.error(Throwable).
Returns:a transformed Flux
/** * Handle the items emitted by this {@link Flux} by calling a biconsumer with the * output sink for each onNext. At most one {@link SynchronousSink#next(Object)} * call must be performed and/or 0 or 1 {@link SynchronousSink#error(Throwable)} or * {@link SynchronousSink#complete()}. * * @param handler the handling {@link BiConsumer} * @param <R> the transformed type * * @reactor.errorMode This operator supports {@link #onErrorContinue(BiConsumer) resuming on errors} (including when * fusion is enabled) when the {@link BiConsumer} throws an exception or if an error is signaled explicitly via * {@link SynchronousSink#error(Throwable)}. * * @return a transformed {@link Flux} */
public final <R> Flux<R> handle(BiConsumer<? super T, SynchronousSink<R>> handler) { if (this instanceof Fuseable) { return onAssembly(new FluxHandleFuseable<>(this, handler)); } return onAssembly(new FluxHandle<>(this, handler)); }
Emit a single boolean true if any of the elements of this Flux sequence is equal to the provided value.

The implementation uses short-circuit logic and completes with true if an element matches the value.

Params:
  • value – constant compared to incoming signals
Returns:a new Flux with true if any element is equal to a given value and false otherwise
/** * Emit a single boolean true if any of the elements of this {@link Flux} sequence is * equal to the provided value. * <p> * The implementation uses short-circuit logic and completes with true if * an element matches the value. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/haselement.png" alt=""> * * @param value constant compared to incoming signals * * @return a new {@link Flux} with <code>true</code> if any element is equal to a given value and <code>false</code> * otherwise * */
public final Mono<Boolean> hasElement(T value) { Objects.requireNonNull(value, "value"); return any(t -> Objects.equals(value, t)); }
Emit a single boolean true if this Flux sequence has at least one element.

The implementation uses short-circuit logic and completes with true on onNext.

Returns:a new Mono with true if any value is emitted and false otherwise
/** * Emit a single boolean true if this {@link Flux} sequence has at least one element. * <p> * The implementation uses short-circuit logic and completes with true on onNext. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/haselements.png" alt=""> * * @return a new {@link Mono} with <code>true</code> if any value is emitted and <code>false</code> * otherwise */
public final Mono<Boolean> hasElements() { return Mono.onAssembly(new MonoHasElements<>(this)); }
Hides the identities of this Flux instance.

The main purpose of this operator is to prevent certain identity-based optimizations from happening, mostly for diagnostic purposes.

Returns:a new Flux preventing Publisher / Subscription based Reactor optimizations
/** * Hides the identities of this {@link Flux} instance. * <p>The main purpose of this operator is to prevent certain identity-based * optimizations from happening, mostly for diagnostic purposes. * * @return a new {@link Flux} preventing {@link Publisher} / {@link Subscription} based Reactor optimizations */
public final Flux<T> hide() { return new FluxHide<>(this); }
Keep information about the order in which source values were received by indexing them with a 0-based incrementing long, returning a Flux of Tuple2<(index, value)>.
Returns:an indexed Flux with each source value combined with its 0-based index.
/** * Keep information about the order in which source values were received by * indexing them with a 0-based incrementing long, returning a {@link Flux} * of {@link Tuple2 Tuple2<(index, value)>}. * * @return an indexed {@link Flux} with each source value combined with its 0-based index. */
public final Flux<Tuple2<Long, T>> index() { return index(tuple2Function()); }
Keep information about the order in which source values were received by indexing them internally with a 0-based incrementing long then combining this information with the source value into a I using the provided BiFunction, returning a Flux<I>.

Typical usage would be to produce a Tuple2 similar to index(), but 1-based instead of 0-based:

index((i, v) -> Tuples.of(i+1, v))

Params:
  • indexMapper – the BiFunction to use to combine elements and their index.
Returns:an indexed Flux with each source value combined with its computed index.
/** * Keep information about the order in which source values were received by * indexing them internally with a 0-based incrementing long then combining this * information with the source value into a {@code I} using the provided {@link BiFunction}, * returning a {@link Flux Flux&lt;I&gt;}. * <p> * Typical usage would be to produce a {@link Tuple2} similar to {@link #index()}, but * 1-based instead of 0-based: * <p> * {@code index((i, v) -> Tuples.of(i+1, v))} * * @param indexMapper the {@link BiFunction} to use to combine elements and their index. * @return an indexed {@link Flux} with each source value combined with its computed index. */
public final <I> Flux<I> index(BiFunction<? super Long, ? super T, ? extends I> indexMapper) { if (this instanceof Fuseable) { return onAssembly(new FluxIndexFuseable<>(this, indexMapper)); } return onAssembly(new FluxIndex<>(this, indexMapper)); }
Ignores onNext signals (dropping them) and only propagate termination events.

@reactor.discardThis operator discards the upstream's elements.
Returns:a new empty Mono representing the completion of this Flux.
/** * Ignores onNext signals (dropping them) and only propagate termination events. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/ignoreelements.png" alt=""> * <p> * * @reactor.discard This operator discards the upstream's elements. * * @return a new empty {@link Mono} representing the completion of this {@link Flux}. */
public final Mono<T> ignoreElements() { return Mono.onAssembly(new MonoIgnoreElements<>(this)); }
Map values from two Publishers into time windows and emit combination of values in case their windows overlap. The emitted elements are obtained by passing the values from this Flux and the other Publisher to a BiFunction.

There are no guarantees in what order the items get combined when multiple items from one or both source Publishers overlap.

Params:
  • other – the other Publisher to correlate items with
  • leftEnd – a function that returns a Publisher whose emissions indicate the time window for the source value to be considered
  • rightEnd – a function that returns a Publisher whose emissions indicate the time window for the right Publisher value to be considered
  • resultSelector – a function that takes an item emitted by each Publisher and returns the value to be emitted by the resulting Flux
Type parameters:
See Also:
Returns:a joining Flux
/** * Map values from two Publishers into time windows and emit combination of values * in case their windows overlap. The emitted elements are obtained by passing the * values from this {@link Flux} and the other {@link Publisher} to a {@link BiFunction}. * <p> * There are no guarantees in what order the items get combined when multiple items from * one or both source Publishers overlap. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/join.png" alt=""> * * * @param other the other {@link Publisher} to correlate items with * @param leftEnd a function that returns a Publisher whose emissions indicate the * time window for the source value to be considered * @param rightEnd a function that returns a Publisher whose emissions indicate the * time window for the {@code right} Publisher value to be considered * @param resultSelector a function that takes an item emitted by each Publisher and returns the * value to be emitted by the resulting {@link Flux} * @param <TRight> the type of the elements from the right {@link Publisher} * @param <TLeftEnd> the type for this {@link Flux} window signals * @param <TRightEnd> the type for the right {@link Publisher} window signals * @param <R> the combined result type * * @return a joining {@link Flux} * @see #groupJoin(Publisher, Function, Function, BiFunction) */
public final <TRight, TLeftEnd, TRightEnd, R> Flux<R> join( Publisher<? extends TRight> other, Function<? super T, ? extends Publisher<TLeftEnd>> leftEnd, Function<? super TRight, ? extends Publisher<TRightEnd>> rightEnd, BiFunction<? super T, ? super TRight, ? extends R> resultSelector ) { return onAssembly(new FluxJoin<T, TRight, TLeftEnd, TRightEnd, R>( this, other, leftEnd, rightEnd, resultSelector)); }
Emit the last element observed before complete signal as a Mono, or emit NoSuchElementException error if the source was empty. For a passive version use takeLast(int)

@reactor.discardThis operator discards elements before the last.
Returns:a Mono with the last value in this Flux
/** * Emit the last element observed before complete signal as a {@link Mono}, or emit * {@link NoSuchElementException} error if the source was empty. * For a passive version use {@link #takeLast(int)} * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/last.png" alt=""> * * @reactor.discard This operator discards elements before the last. * * @return a {@link Mono} with the last value in this {@link Flux} */
public final Mono<T> last() { if (this instanceof Callable) { @SuppressWarnings("unchecked") Callable<T> thiz = (Callable<T>) this; Mono<T> callableMono = convertToMono(thiz); if (callableMono == Mono.empty()) { return Mono.error(new NoSuchElementException("Flux#last() didn't observe any onNext signal from Callable flux")); } return callableMono; } return Mono.onAssembly(new MonoTakeLastOne<>(this)); }
Emit the last element observed before complete signal as a Mono, or emit the defaultValue if the source was empty. For a passive version use takeLast(int)

Params:
  • defaultValue – a single fallback item if this Flux is empty
@reactor.discardThis operator discards elements before the last.
Returns:a Mono with the last value in this Flux
/** * Emit the last element observed before complete signal as a {@link Mono}, or emit * the {@code defaultValue} if the source was empty. * For a passive version use {@link #takeLast(int)} * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/last.png" alt=""> * * @reactor.discard This operator discards elements before the last. * * @param defaultValue a single fallback item if this {@link Flux} is empty * @return a {@link Mono} with the last value in this {@link Flux} */
public final Mono<T> last(T defaultValue) { if (this instanceof Callable) { @SuppressWarnings("unchecked") Callable<T> thiz = (Callable<T>)this; if(thiz instanceof Fuseable.ScalarCallable){ @SuppressWarnings("unchecked") Fuseable.ScalarCallable<T> c = (Fuseable.ScalarCallable<T>)thiz; T v; try { v = c.call(); } catch (Exception e) { return Mono.error(e); } if(v == null){ return Mono.just(defaultValue); } return Mono.just(v); } Mono.onAssembly(new MonoCallable<>(thiz)); } return Mono.onAssembly(new MonoTakeLastOne<>(this, defaultValue)); }
Ensure that backpressure signals from downstream subscribers are split into batches capped at the provided prefetchRate when propagated upstream, effectively rate limiting the upstream Publisher.

Note that this is an upper bound, and that this operator uses a prefetch-and-replenish strategy, requesting a replenishing amount when 75% of the prefetch amount has been emitted.

Typically used for scenarios where consumer(s) request a large amount of data (eg. Long.MAX_VALUE) but the data source behaves better or can be optimized with smaller requests (eg. database paging, etc...). All data is still processed, unlike with limitRequest(long) which will cap the grand total request amount.

Equivalent to flux.publishOn(Schedulers.immediate(), prefetchRate).subscribe() . Note that the prefetchRate is an upper bound, and that this operator uses a prefetch-and-replenish strategy, requesting a replenishing amount when 75% of the prefetch amount has been emitted.

Params:
  • prefetchRate – the limit to apply to downstream's backpressure
See Also:
Returns:a Flux limiting downstream's backpressure
/** * Ensure that backpressure signals from downstream subscribers are split into batches * capped at the provided {@code prefetchRate} when propagated upstream, effectively * rate limiting the upstream {@link Publisher}. * <p> * Note that this is an upper bound, and that this operator uses a prefetch-and-replenish * strategy, requesting a replenishing amount when 75% of the prefetch amount has been * emitted. * <p> * Typically used for scenarios where consumer(s) request a large amount of data * (eg. {@code Long.MAX_VALUE}) but the data source behaves better or can be optimized * with smaller requests (eg. database paging, etc...). All data is still processed, * unlike with {@link #limitRequest(long)} which will cap the grand total request * amount. * <p> * Equivalent to {@code flux.publishOn(Schedulers.immediate(), prefetchRate).subscribe() }. * Note that the {@code prefetchRate} is an upper bound, and that this operator uses a * prefetch-and-replenish strategy, requesting a replenishing amount when 75% of the * prefetch amount has been emitted. * * @param prefetchRate the limit to apply to downstream's backpressure * * @return a {@link Flux} limiting downstream's backpressure * @see #publishOn(Scheduler, int) * @see #limitRequest(long) */
public final Flux<T> limitRate(int prefetchRate) { return onAssembly(this.publishOn(Schedulers.immediate(), prefetchRate)); }
Ensure that backpressure signals from downstream subscribers are split into batches capped at the provided highTide first, then replenishing at the provided lowTide, effectively rate limiting the upstream Publisher.

Note that this is an upper bound, and that this operator uses a prefetch-and-replenish strategy, requesting a replenishing amount when 75% of the prefetch amount has been emitted.

Typically used for scenarios where consumer(s) request a large amount of data (eg. Long.MAX_VALUE) but the data source behaves better or can be optimized with smaller requests (eg. database paging, etc...). All data is still processed, unlike with limitRequest(long) which will cap the grand total request amount.

Similar to flux.publishOn(Schedulers.immediate(), prefetchRate).subscribe() , except with a customized "low tide" instead of the default 75%. Note that the smaller the lowTide is, the higher the potential for concurrency between request and data production. And thus the more extraneous replenishment requests this operator could make. For example, for a global downstream request of 14, with a highTide of 10 and a lowTide of 2, the operator would perform 7 low tide requests, whereas with the default lowTide of 8 it would only perform one. Using a lowTide equal to highTide reverts to the default 75% strategy, while using a lowTide of 0 disables the lowTide, resulting in all requests strictly adhering to the highTide.

Params:
  • highTide – the initial request amount
  • lowTide – the subsequent (or replenishing) request amount, 0 to disable early replenishing, highTide to revert to a 75% replenish strategy.
See Also:
Returns:a Flux limiting downstream's backpressure and customizing the replenishment request amount
/** * Ensure that backpressure signals from downstream subscribers are split into batches * capped at the provided {@code highTide} first, then replenishing at the provided * {@code lowTide}, effectively rate limiting the upstream {@link Publisher}. * <p> * Note that this is an upper bound, and that this operator uses a prefetch-and-replenish * strategy, requesting a replenishing amount when 75% of the prefetch amount has been * emitted. * <p> * Typically used for scenarios where consumer(s) request a large amount of data * (eg. {@code Long.MAX_VALUE}) but the data source behaves better or can be optimized * with smaller requests (eg. database paging, etc...). All data is still processed, * unlike with {@link #limitRequest(long)} which will cap the grand total request * amount. * <p> * Similar to {@code flux.publishOn(Schedulers.immediate(), prefetchRate).subscribe() }, * except with a customized "low tide" instead of the default 75%. * Note that the smaller the lowTide is, the higher the potential for concurrency * between request and data production. And thus the more extraneous replenishment * requests this operator could make. For example, for a global downstream * request of 14, with a highTide of 10 and a lowTide of 2, the operator would perform * 7 low tide requests, whereas with the default lowTide of 8 it would only perform one. * Using a {@code lowTide} equal to {@code highTide} reverts to the default 75% strategy, * while using a {@code lowTide} of {@literal 0} disables the lowTide, resulting in * all requests strictly adhering to the highTide. * * @param highTide the initial request amount * @param lowTide the subsequent (or replenishing) request amount, {@literal 0} to * disable early replenishing, {@literal highTide} to revert to a 75% replenish strategy. * * @return a {@link Flux} limiting downstream's backpressure and customizing the * replenishment request amount * @see #publishOn(Scheduler, int) * @see #limitRequest(long) */
public final Flux<T> limitRate(int highTide, int lowTide) { return onAssembly(this.publishOn(Schedulers.immediate(), true, highTide, lowTide)); }
Ensure that the total amount requested upstream is capped at cap. Backpressure signals from downstream subscribers are smaller than the cap are propagated as is, but if they would cause the total requested amount to go over the cap, they are reduced to the minimum value that doesn't go over.

As a result, this operator never let the upstream produce more elements than the cap, and it can be used as a stricter form of take(long). Typically useful for cases where a race between request and cancellation can lead the upstream to producing a lot of extraneous data, and such a production is undesirable (e.g. a source that would send the extraneous data over the network).

Params:
  • requestCap – the global backpressure limit to apply to the sum of downstream's requests
See Also:
Returns:a Flux that requests AT MOST cap from upstream in total.
/** * Ensure that the total amount requested upstream is capped at {@code cap}. * Backpressure signals from downstream subscribers are smaller than the cap are * propagated as is, but if they would cause the total requested amount to go over the * cap, they are reduced to the minimum value that doesn't go over. * <p> * As a result, this operator never let the upstream produce more elements than the * cap, and it can be used as a stricter form of {@link #take(long)}. Typically useful * for cases where a race between request and cancellation can lead the upstream to * producing a lot of extraneous data, and such a production is undesirable (e.g. * a source that would send the extraneous data over the network). * * @param requestCap the global backpressure limit to apply to the sum of downstream's requests * * @return a {@link Flux} that requests AT MOST {@code cap} from upstream in total. * @see #limitRate(int) * @see #take(long) */
public final Flux<T> limitRequest(long requestCap) { return onAssembly(new FluxLimitRequest<>(this, requestCap)); }
Observe all Reactive Streams signals and trace them using Logger support. Default will use Level.INFO and java.util.logging. If SLF4J is available, it will be used instead.

The default log category will be "reactor.Flux.", followed by a suffix generated from the source operator, e.g. "reactor.Flux.Map".

Returns:a new Flux that logs signals
/** * Observe all Reactive Streams signals and trace them using {@link Logger} support. * Default will use {@link Level#INFO} and {@code java.util.logging}. * If SLF4J is available, it will be used instead. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/log.png" alt=""> * <p> * The default log category will be "reactor.Flux.", followed by a suffix generated from * the source operator, e.g. "reactor.Flux.Map". * * @return a new {@link Flux} that logs signals */
public final Flux<T> log() { return log(null, Level.INFO); }
Observe all Reactive Streams signals and trace them using Logger support. Default will use Level.INFO and java.util.logging. If SLF4J is available, it will be used instead.

Params:
  • category – to be mapped into logger configuration (e.g. org.springframework .reactor). If category ends with "." like "reactor.", a generated operator suffix will be added, e.g. "reactor.Flux.Map".
Returns:a new Flux that logs signals
/** * Observe all Reactive Streams signals and trace them using {@link Logger} support. * Default will use {@link Level#INFO} and {@code java.util.logging}. * If SLF4J is available, it will be used instead. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/log.png" alt=""> * <p> * @param category to be mapped into logger configuration (e.g. org.springframework * .reactor). If category ends with "." like "reactor.", a generated operator * suffix will be added, e.g. "reactor.Flux.Map". * * @return a new {@link Flux} that logs signals */
public final Flux<T> log(String category) { return log(category, Level.INFO); }
Observe Reactive Streams signals matching the passed filter options and trace them using Logger support. Default will use Level.INFO and java.util.logging. If SLF4J is available, it will be used instead.

Options allow fine grained filtering of the traced signal, for instance to only capture onNext and onError:

    flux.log("category", Level.INFO, SignalType.ON_NEXT, SignalType.ON_ERROR)

Params:
  • category – to be mapped into logger configuration (e.g. org.springframework .reactor). If category ends with "." like "reactor.", a generated operator suffix will be added, e.g. "reactor.Flux.Map".
  • level – the Level to enforce for this tracing Flux (only FINEST, FINE, INFO, WARNING and SEVERE are taken into account)
  • options – a vararg SignalType option to filter log messages
Returns:a new Flux that logs signals
/** * Observe Reactive Streams signals matching the passed filter {@code options} and * trace them using {@link Logger} support. Default will use {@link Level#INFO} and * {@code java.util.logging}. If SLF4J is available, it will be used instead. * <p> * Options allow fine grained filtering of the traced signal, for instance to only * capture onNext and onError: * <pre> * flux.log("category", Level.INFO, SignalType.ON_NEXT, SignalType.ON_ERROR) * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/log.png" alt=""> * <p> * @param category to be mapped into logger configuration (e.g. org.springframework * .reactor). If category ends with "." like "reactor.", a generated operator * suffix will be added, e.g. "reactor.Flux.Map". * @param level the {@link Level} to enforce for this tracing Flux (only FINEST, FINE, * INFO, WARNING and SEVERE are taken into account) * @param options a vararg {@link SignalType} option to filter log messages * * @return a new {@link Flux} that logs signals */
public final Flux<T> log(@Nullable String category, Level level, SignalType... options) { return log(category, level, false, options); }
Observe Reactive Streams signals matching the passed filter options and trace them using Logger support. Default will use Level.INFO and java.util.logging. If SLF4J is available, it will be used instead.

Options allow fine grained filtering of the traced signal, for instance to only capture onNext and onError:

    flux.log("category", Level.INFO, SignalType.ON_NEXT, SignalType.ON_ERROR)

Params:
  • category – to be mapped into logger configuration (e.g. org.springframework .reactor). If category ends with "." like "reactor.", a generated operator suffix will be added, e.g. "reactor.Flux.Map".
  • level – the Level to enforce for this tracing Flux (only FINEST, FINE, INFO, WARNING and SEVERE are taken into account)
  • showOperatorLine – capture the current stack to display operator class/line number.
  • options – a vararg SignalType option to filter log messages
Returns:a new Flux that logs signals
/** * Observe Reactive Streams signals matching the passed filter {@code options} and * trace them using {@link Logger} support. Default will use {@link Level#INFO} and * {@code java.util.logging}. If SLF4J is available, it will be used instead. * <p> * Options allow fine grained filtering of the traced signal, for instance to only * capture onNext and onError: * <pre> * flux.log("category", Level.INFO, SignalType.ON_NEXT, SignalType.ON_ERROR) * * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/log.png" * alt=""> * * @param category to be mapped into logger configuration (e.g. org.springframework * .reactor). If category ends with "." like "reactor.", a generated operator * suffix will be added, e.g. "reactor.Flux.Map". * @param level the {@link Level} to enforce for this tracing Flux (only FINEST, FINE, * INFO, WARNING and SEVERE are taken into account) * @param showOperatorLine capture the current stack to display operator class/line number. * @param options a vararg {@link SignalType} option to filter log messages * * @return a new {@link Flux} that logs signals */
public final Flux<T> log(@Nullable String category, Level level, boolean showOperatorLine, SignalType... options) { SignalLogger<T> log = new SignalLogger<>(this, category, level, showOperatorLine, options); if (this instanceof Fuseable) { return onAssembly(new FluxLogFuseable<>(this, log)); } return onAssembly(new FluxLog<>(this, log)); }
Observe Reactive Streams signals matching the passed filter options and trace them using a specific user-provided Logger, at Level.INFO level.
Params:
  • logger – the Logger to use, instead of resolving one through a category.
Returns:a new Flux that logs signals
/** * Observe Reactive Streams signals matching the passed filter {@code options} and * trace them using a specific user-provided {@link Logger}, at {@link Level#INFO} level. * * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/log.png" alt=""> * * @param logger the {@link Logger} to use, instead of resolving one through a category. * * @return a new {@link Flux} that logs signals */
public final Flux<T> log(Logger logger) { return log(logger, Level.INFO, false); }
Observe Reactive Streams signals matching the passed filter options and trace them using a specific user-provided Logger, at the given Level.

Options allow fine grained filtering of the traced signal, for instance to only capture onNext and onError:

    flux.log(myCustomLogger, Level.INFO, SignalType.ON_NEXT, SignalType.ON_ERROR)

Params:
  • logger – the Logger to use, instead of resolving one through a category.
  • level – the Level to enforce for this tracing Flux (only FINEST, FINE, INFO, WARNING and SEVERE are taken into account)
  • showOperatorLine – capture the current stack to display operator class/line number (default in overload is false).
  • options – a vararg SignalType option to filter log messages
Returns:a new Flux that logs signals
/** * Observe Reactive Streams signals matching the passed filter {@code options} and * trace them using a specific user-provided {@link Logger}, at the given {@link Level}. * <p> * Options allow fine grained filtering of the traced signal, for instance to only * capture onNext and onError: * <pre> * flux.log(myCustomLogger, Level.INFO, SignalType.ON_NEXT, SignalType.ON_ERROR) * * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/log.png" * alt=""> * * @param logger the {@link Logger} to use, instead of resolving one through a category. * @param level the {@link Level} to enforce for this tracing Flux (only FINEST, FINE, * INFO, WARNING and SEVERE are taken into account) * @param showOperatorLine capture the current stack to display operator class/line number (default in overload is false). * @param options a vararg {@link SignalType} option to filter log messages * * @return a new {@link Flux} that logs signals */
public final Flux<T> log(Logger logger, Level level, boolean showOperatorLine, SignalType... options) { SignalLogger<T> log = new SignalLogger<>(this, "IGNORED", level, showOperatorLine, s -> logger, options); if (this instanceof Fuseable) { return onAssembly(new FluxLogFuseable<>(this, log)); } return onAssembly(new FluxLog<>(this, log)); }
Transform the items emitted by this Flux by applying a synchronous function to each item.

Params:
  • mapper – the synchronous transforming Function
Type parameters:
  • <V> – the transformed type
@reactor.errorModeThis operator supports resuming on errors (including when fusion is enabled). Exceptions thrown by the mapper then cause the source value to be dropped and a new element (request(1)) being requested from upstream.
Returns:a transformed Flux
/** * Transform the items emitted by this {@link Flux} by applying a synchronous function * to each item. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/map.png" alt=""> * <p> * @param mapper the synchronous transforming {@link Function} * @param <V> the transformed type * * @reactor.errorMode This operator supports {@link #onErrorContinue(BiConsumer) resuming on errors} * (including when fusion is enabled). Exceptions thrown by the mapper then cause the * source value to be dropped and a new element ({@code request(1)}) being requested * from upstream. * * @return a transformed {@link Flux} */
public final <V> Flux<V> map(Function<? super T, ? extends V> mapper) { if (this instanceof Fuseable) { return onAssembly(new FluxMapFuseable<>(this, mapper)); } return onAssembly(new FluxMap<>(this, mapper)); }
Transform incoming onNext, onError and onComplete signals into Signal instances, materializing these signals. Since the error is materialized as a Signal, the propagation will be stopped and onComplete will be emitted. Complete signal will first emit a Signal.complete() and then effectively complete the flux. All these Signal have a Context associated to them.

See Also:
Returns:a Flux of materialized Signal
/** * Transform incoming onNext, onError and onComplete signals into {@link Signal} instances, * materializing these signals. * Since the error is materialized as a {@code Signal}, the propagation will be stopped and onComplete will be * emitted. Complete signal will first emit a {@code Signal.complete()} and then effectively complete the flux. * All these {@link Signal} have a {@link Context} associated to them. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/materialize.png" alt=""> * * @return a {@link Flux} of materialized {@link Signal} * @see #dematerialize() */
public final Flux<Signal<T>> materialize() { return onAssembly(new FluxMaterialize<>(this)); }
Merge data from this Flux and a Publisher into a reordered merge sequence, by picking the smallest value from each sequence as defined by a provided Comparator. Note that subsequent calls are combined, and their comparators are in lexicographic order as defined by Comparator.thenComparing(Comparator).

The combination step is avoided if the two Comparators are equal (which can easily be achieved by using the same reference, and is also always true of Comparator.naturalOrder()).

Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to another source.

Params:
Returns:a new Flux
/** * Merge data from this {@link Flux} and a {@link Publisher} into a reordered merge * sequence, by picking the smallest value from each sequence as defined by a provided * {@link Comparator}. Note that subsequent calls are combined, and their comparators are * in lexicographic order as defined by {@link Comparator#thenComparing(Comparator)}. * <p> * The combination step is avoided if the two {@link Comparator Comparators} are * {@link Comparator#equals(Object) equal} (which can easily be achieved by using the * same reference, and is also always true of {@link Comparator#naturalOrder()}). * <p> * Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with * an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source * in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to * another source. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.2.0.M2/src/docs/marble/mergeordered.png" alt=""> * <p> * * @param other the {@link Publisher} to merge with * @param otherComparator the {@link Comparator} to use for merging * * @return a new {@link Flux} */
public final Flux<T> mergeOrderedWith(Publisher<? extends T> other, Comparator<? super T> otherComparator) { if (this instanceof FluxMergeOrdered) { FluxMergeOrdered<T> fluxMerge = (FluxMergeOrdered<T>) this; return fluxMerge.mergeAdditionalSource(other, otherComparator); } return mergeOrdered(otherComparator, this, other); }
Merge data from this Flux and a Publisher into an interleaved merged sequence. Unlike concat, inner sources are subscribed to eagerly.

Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to another source.

Params:
Returns:a new Flux
/** * Merge data from this {@link Flux} and a {@link Publisher} into an interleaved merged * sequence. Unlike {@link #concatWith(Publisher) concat}, inner sources are subscribed * to eagerly. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/merge.png" alt=""> * <p> * Note that merge is tailored to work with asynchronous sources or finite sources. When dealing with * an infinite source that doesn't already publish on a dedicated Scheduler, you must isolate that source * in its own Scheduler, as merge would otherwise attempt to drain it before subscribing to * another source. * * @param other the {@link Publisher} to merge with * * @return a new {@link Flux} */
public final Flux<T> mergeWith(Publisher<? extends T> other) { if (this instanceof FluxMerge) { FluxMerge<T> fluxMerge = (FluxMerge<T>) this; return fluxMerge.mergeAdditionalSource(other, Queues::get); } return merge(this, other); }
Activate metrics for this sequence, provided there is an instrumentation facade on the classpath (otherwise this method is a pure no-op).

Metrics are gathered on Subscriber events, and it is recommended to also name (and optionally tag) the sequence.

Returns:an instrumented Flux
/** * Activate metrics for this sequence, provided there is an instrumentation facade * on the classpath (otherwise this method is a pure no-op). * <p> * Metrics are gathered on {@link Subscriber} events, and it is recommended to also * {@link #name(String) name} (and optionally {@link #tag(String, String) tag}) the * sequence. * * @return an instrumented {@link Flux} */
public final Flux<T> metrics() { if (!FluxMetrics.isMicrometerAvailable()) { return this; } if (this instanceof Fuseable) { return onAssembly(new FluxMetricsFuseable<>(this)); } return onAssembly(new FluxMetrics<>(this)); }
Give a name to this sequence, which can be retrieved using Scannable.name() as long as this is the first reachable Scannable.parents().
Params:
  • name – a name for the sequence
Returns:the same sequence, but bearing a name
/** * Give a name to this sequence, which can be retrieved using {@link Scannable#name()} * as long as this is the first reachable {@link Scannable#parents()}. * * @param name a name for the sequence * @return the same sequence, but bearing a name */
public final Flux<T> name(String name) { return FluxName.createOrAppend(this, name); }
Emit only the first item emitted by this Flux, into a new Mono.

Returns:a new Mono emitting the first value in this Flux
/** * Emit only the first item emitted by this {@link Flux}, into a new {@link Mono}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/next.png" alt=""> * <p> * * @return a new {@link Mono} emitting the first value in this {@link Flux} */
public final Mono<T> next() { if(this instanceof Callable){ @SuppressWarnings("unchecked") Callable<T> m = (Callable<T>)this; return convertToMono(m); } return Mono.onAssembly(new MonoNext<>(this)); }
Evaluate each accepted value against the given Class type. If the a value matches the type, it is passed into the resulting Flux. Otherwise the value is ignored and a request of 1 is emitted.

Params:
  • clazz – the Class type to test values against
Returns:a new Flux filtered on items of the requested type
/** * Evaluate each accepted value against the given {@link Class} type. If the * a value matches the type, it is passed into the resulting {@link Flux}. Otherwise * the value is ignored and a request of 1 is emitted. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/filter.png" alt=""> * * @param clazz the {@link Class} type to test values against * * @return a new {@link Flux} filtered on items of the requested type */
public final <U> Flux<U> ofType(final Class<U> clazz) { Objects.requireNonNull(clazz, "clazz"); return filter(o -> clazz.isAssignableFrom(o.getClass())).cast(clazz); }
Request an unbounded demand and push to the returned Flux, or park the observed elements if not enough demand is requested downstream. Errors will be delayed until the buffer gets consumed.

@reactor.discardThis operator discards the buffered overflow elements upon cancellation or error triggered by a data signal.
Returns:a backpressured Flux that buffers with unbounded capacity
/** * Request an unbounded demand and push to the returned {@link Flux}, or park the * observed elements if not enough demand is requested downstream. Errors will be * delayed until the buffer gets consumed. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/onbackpressurebuffer.png" alt=""> * * @reactor.discard This operator discards the buffered overflow elements upon cancellation or error triggered by a data signal. * * @return a backpressured {@link Flux} that buffers with unbounded capacity * */
public final Flux<T> onBackpressureBuffer() { return onAssembly(new FluxOnBackpressureBuffer<>(this, Queues .SMALL_BUFFER_SIZE, true, null)); }
Request an unbounded demand and push to the returned Flux, or park the observed elements if not enough demand is requested downstream. Errors will be immediately emitted on overflow regardless of the pending buffer.

Params:
  • maxSize – maximum buffer backlog size before immediate error
@reactor.discardThis operator discards the buffered overflow elements upon cancellation or error triggered by a data signal, as well as elements that are rejected by the buffer due to maxSize.
Returns:a backpressured Flux that buffers with bounded capacity
/** * Request an unbounded demand and push to the returned {@link Flux}, or park the * observed elements if not enough demand is requested downstream. Errors will be * immediately emitted on overflow regardless of the pending buffer. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/onbackpressurebuffer.png" alt=""> * * @reactor.discard This operator discards the buffered overflow elements upon cancellation or error triggered by a data signal, * as well as elements that are rejected by the buffer due to {@code maxSize}. * * @param maxSize maximum buffer backlog size before immediate error * * @return a backpressured {@link Flux} that buffers with bounded capacity * */
public final Flux<T> onBackpressureBuffer(int maxSize) { return onAssembly(new FluxOnBackpressureBuffer<>(this, maxSize, false, null)); }
Request an unbounded demand and push to the returned Flux, or park the observed elements if not enough demand is requested downstream. Overflow error will be delayed after the current backlog is consumed. However the Consumer will be immediately invoked.

Params:
  • maxSize – maximum buffer backlog size before overflow callback is called
  • onOverflow – callback to invoke on overflow
@reactor.discardThis operator discards the buffered overflow elements upon cancellation or error triggered by a data signal, as well as elements that are rejected by the buffer due to maxSize (even though they are passed to the onOverflow Consumer first).
Returns:a backpressured Flux that buffers with a bounded capacity
/** * Request an unbounded demand and push to the returned {@link Flux}, or park the * observed elements if not enough demand is requested downstream. Overflow error * will be delayed after the current backlog is consumed. However the * {@link Consumer} will be immediately invoked. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/onbackpressurebuffer.png" alt=""> * * @reactor.discard This operator discards the buffered overflow elements upon cancellation or error triggered by a data signal, * as well as elements that are rejected by the buffer due to {@code maxSize} (even though * they are passed to the {@code onOverflow} {@link Consumer} first). * * @param maxSize maximum buffer backlog size before overflow callback is called * @param onOverflow callback to invoke on overflow * * @return a backpressured {@link Flux} that buffers with a bounded capacity * */
public final Flux<T> onBackpressureBuffer(int maxSize, Consumer<? super T> onOverflow) { Objects.requireNonNull(onOverflow, "onOverflow"); return onAssembly(new FluxOnBackpressureBuffer<>(this, maxSize, false, onOverflow)); }
Request an unbounded demand and push to the returned Flux, or park the observed elements if not enough demand is requested downstream, within a maxSize limit. Over that limit, the overflow strategy is applied (see BufferOverflowStrategy).

Note that for the ERROR strategy, the overflow error will be delayed after the current backlog is consumed.

Params:
  • maxSize – maximum buffer backlog size before overflow strategy is applied
  • bufferOverflowStrategy – strategy to apply to overflowing elements
@reactor.discardThis operator discards the buffered overflow elements upon cancellation or error triggered by a data signal, as well as elements that are rejected by the buffer due to maxSize (even though they are passed to the bufferOverflowStrategy first).
Returns:a backpressured Flux that buffers up to a capacity then applies an overflow strategy
/** * Request an unbounded demand and push to the returned {@link Flux}, or park the observed * elements if not enough demand is requested downstream, within a {@code maxSize} * limit. Over that limit, the overflow strategy is applied (see {@link BufferOverflowStrategy}). * <p> * Note that for the {@link BufferOverflowStrategy#ERROR ERROR} strategy, the overflow * error will be delayed after the current backlog is consumed. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/onbackpressurebuffer.png" alt=""> * * @reactor.discard This operator discards the buffered overflow elements upon cancellation or error triggered by a data signal, * as well as elements that are rejected by the buffer due to {@code maxSize} (even though * they are passed to the {@code bufferOverflowStrategy} first). * * * @param maxSize maximum buffer backlog size before overflow strategy is applied * @param bufferOverflowStrategy strategy to apply to overflowing elements * * @return a backpressured {@link Flux} that buffers up to a capacity then applies an * overflow strategy */
public final Flux<T> onBackpressureBuffer(int maxSize, BufferOverflowStrategy bufferOverflowStrategy) { Objects.requireNonNull(bufferOverflowStrategy, "bufferOverflowStrategy"); return onAssembly(new FluxOnBackpressureBufferStrategy<>(this, maxSize, null, bufferOverflowStrategy)); }
Request an unbounded demand and push to the returned Flux, or park the observed elements if not enough demand is requested downstream, within a maxSize limit. Over that limit, the overflow strategy is applied (see BufferOverflowStrategy).

A Consumer is immediately invoked when there is an overflow, receiving the value that was discarded because of the overflow (which can be different from the latest element emitted by the source in case of a DROP_LATEST strategy).

Note that for the ERROR strategy, the overflow error will be delayed after the current backlog is consumed. The consumer is still invoked immediately.

Params:
  • maxSize – maximum buffer backlog size before overflow callback is called
  • onBufferOverflow – callback to invoke on overflow
  • bufferOverflowStrategy – strategy to apply to overflowing elements
@reactor.discardThis operator discards the buffered overflow elements upon cancellation or error triggered by a data signal, as well as elements that are rejected by the buffer due to maxSize (even though they are passed to the onOverflow Consumer AND the bufferOverflowStrategy first).
Returns:a backpressured Flux that buffers up to a capacity then applies an overflow strategy
/** * Request an unbounded demand and push to the returned {@link Flux}, or park the observed * elements if not enough demand is requested downstream, within a {@code maxSize} * limit. Over that limit, the overflow strategy is applied (see {@link BufferOverflowStrategy}). * <p> * A {@link Consumer} is immediately invoked when there is an overflow, receiving the * value that was discarded because of the overflow (which can be different from the * latest element emitted by the source in case of a * {@link BufferOverflowStrategy#DROP_LATEST DROP_LATEST} strategy). * * <p> * Note that for the {@link BufferOverflowStrategy#ERROR ERROR} strategy, the overflow * error will be delayed after the current backlog is consumed. The consumer is still * invoked immediately. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/onbackpressurebuffer.png" alt=""> * * @reactor.discard This operator discards the buffered overflow elements upon cancellation or error triggered by a data signal, * as well as elements that are rejected by the buffer due to {@code maxSize} (even though * they are passed to the {@code onOverflow} {@link Consumer} AND the {@code bufferOverflowStrategy} first). * * @param maxSize maximum buffer backlog size before overflow callback is called * @param onBufferOverflow callback to invoke on overflow * @param bufferOverflowStrategy strategy to apply to overflowing elements * * @return a backpressured {@link Flux} that buffers up to a capacity then applies an * overflow strategy */
public final Flux<T> onBackpressureBuffer(int maxSize, Consumer<? super T> onBufferOverflow, BufferOverflowStrategy bufferOverflowStrategy) { Objects.requireNonNull(onBufferOverflow, "onBufferOverflow"); Objects.requireNonNull(bufferOverflowStrategy, "bufferOverflowStrategy"); return onAssembly(new FluxOnBackpressureBufferStrategy<>(this, maxSize, onBufferOverflow, bufferOverflowStrategy)); }
Request an unbounded demand and push to the returned Flux, or park the observed elements if not enough demand is requested downstream, within a maxSize limit and for a maximum Duration of ttl (as measured on the elastic Scheduler). Over that limit, oldest elements from the source are dropped.

Elements evicted based on the TTL are passed to a cleanup Consumer, which is also immediately invoked when there is an overflow.

Params:
  • ttl – maximum Duration for which an element is kept in the backlog
  • maxSize – maximum buffer backlog size before overflow callback is called
  • onBufferEviction – callback to invoke once TTL is reached or on overflow
@reactor.discardThis operator discards its internal buffer of elements that overflow, after having applied the onBufferEviction handler.
Returns:a backpressured Flux that buffers with a TTL and up to a capacity then applies an overflow strategy
/** * Request an unbounded demand and push to the returned {@link Flux}, or park the observed * elements if not enough demand is requested downstream, within a {@code maxSize} * limit and for a maximum {@link Duration} of {@code ttl} (as measured on the * {@link Schedulers#elastic() elastic Scheduler}). Over that limit, oldest * elements from the source are dropped. * <p> * Elements evicted based on the TTL are passed to a cleanup {@link Consumer}, which * is also immediately invoked when there is an overflow. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/onbackpressurebuffer.png" alt=""> * * @reactor.discard This operator discards its internal buffer of elements that overflow, * after having applied the {@code onBufferEviction} handler. * * @param ttl maximum {@link Duration} for which an element is kept in the backlog * @param maxSize maximum buffer backlog size before overflow callback is called * @param onBufferEviction callback to invoke once TTL is reached or on overflow * * @return a backpressured {@link Flux} that buffers with a TTL and up to a capacity then applies an * overflow strategy */
public final Flux<T> onBackpressureBuffer(Duration ttl, int maxSize, Consumer<? super T> onBufferEviction) { return onBackpressureBuffer(ttl, maxSize, onBufferEviction, Schedulers.parallel()); }
Request an unbounded demand and push to the returned Flux, or park the observed elements if not enough demand is requested downstream, within a maxSize limit and for a maximum Duration of ttl (as measured on the provided Scheduler). Over that limit, oldest elements from the source are dropped.

Elements evicted based on the TTL are passed to a cleanup Consumer, which is also immediately invoked when there is an overflow.

Params:
  • ttl – maximum Duration for which an element is kept in the backlog
  • maxSize – maximum buffer backlog size before overflow callback is called
  • onBufferEviction – callback to invoke once TTL is reached or on overflow
  • scheduler – the scheduler on which to run the timeout check
@reactor.discardThis operator discards its internal buffer of elements that overflow, after having applied the onBufferEviction handler.
Returns:a backpressured Flux that buffers with a TTL and up to a capacity then applies an overflow strategy
/** * Request an unbounded demand and push to the returned {@link Flux}, or park the observed * elements if not enough demand is requested downstream, within a {@code maxSize} * limit and for a maximum {@link Duration} of {@code ttl} (as measured on the provided * {@link Scheduler}). Over that limit, oldest elements from the source are dropped. * <p> * Elements evicted based on the TTL are passed to a cleanup {@link Consumer}, which * is also immediately invoked when there is an overflow. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/onbackpressurebuffer.png" alt=""> * * @reactor.discard This operator discards its internal buffer of elements that overflow, * after having applied the {@code onBufferEviction} handler. * * @param ttl maximum {@link Duration} for which an element is kept in the backlog * @param maxSize maximum buffer backlog size before overflow callback is called * @param onBufferEviction callback to invoke once TTL is reached or on overflow * @param scheduler the scheduler on which to run the timeout check * * @return a backpressured {@link Flux} that buffers with a TTL and up to a capacity then applies an * overflow strategy */
public final Flux<T> onBackpressureBuffer(Duration ttl, int maxSize, Consumer<? super T> onBufferEviction, Scheduler scheduler) { Objects.requireNonNull(ttl, "ttl"); Objects.requireNonNull(onBufferEviction, "onBufferEviction"); return onAssembly(new FluxOnBackpressureBufferTimeout<>(this, ttl, scheduler, maxSize, onBufferEviction)); }
Request an unbounded demand and push to the returned Flux, or drop the observed elements if not enough demand is requested downstream.

@reactor.discardThis operator discards elements that it drops.
Returns:a backpressured Flux that drops overflowing elements
/** * Request an unbounded demand and push to the returned {@link Flux}, or drop * the observed elements if not enough demand is requested downstream. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/onbackpressuredrop.png" alt=""> * * @reactor.discard This operator discards elements that it drops. * * @return a backpressured {@link Flux} that drops overflowing elements */
public final Flux<T> onBackpressureDrop() { return onAssembly(new FluxOnBackpressureDrop<>(this)); }
Request an unbounded demand and push to the returned Flux, or drop and notify dropping Consumer with the observed elements if not enough demand is requested downstream.

Params:
  • onDropped – the Consumer called when an value gets dropped due to lack of downstream requests
@reactor.discardThis operator discards elements that it drops after having passed them to the provided onDropped handler.
Returns:a backpressured Flux that drops overflowing elements
/** * Request an unbounded demand and push to the returned {@link Flux}, or drop and * notify dropping {@link Consumer} with the observed elements if not enough demand * is requested downstream. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/onbackpressuredropc.png" alt=""> * * @reactor.discard This operator discards elements that it drops after having passed * them to the provided {@code onDropped} handler. * * @param onDropped the Consumer called when an value gets dropped due to lack of downstream requests * @return a backpressured {@link Flux} that drops overflowing elements */
public final Flux<T> onBackpressureDrop(Consumer<? super T> onDropped) { return onAssembly(new FluxOnBackpressureDrop<>(this, onDropped)); }
Request an unbounded demand and push to the returned Flux, or emit onError fom Exceptions.failWithOverflow if not enough demand is requested downstream.

@reactor.discardThis operator discards elements that it drops, after having propagated the error.
Returns:a backpressured Flux that errors on overflowing elements
/** * Request an unbounded demand and push to the returned {@link Flux}, or emit onError * fom {@link Exceptions#failWithOverflow} if not enough demand is requested * downstream. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/onbackpressureerror.png" alt=""> * * @reactor.discard This operator discards elements that it drops, after having propagated * the error. * * @return a backpressured {@link Flux} that errors on overflowing elements */
public final Flux<T> onBackpressureError() { return onBackpressureDrop(t -> { throw Exceptions.failWithOverflow();}); }
Request an unbounded demand and push to the returned Flux, or only keep the most recent observed item if not enough demand is requested downstream.

@reactor.discardEach time a new element comes in (the new "latest"), this operator discards the previously retained element.
Returns:a backpressured Flux that will only keep a reference to the last observed item
/** * Request an unbounded demand and push to the returned {@link Flux}, or only keep * the most recent observed item if not enough demand is requested downstream. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/onbackpressurelatest.png" alt=""> * * @reactor.discard Each time a new element comes in (the new "latest"), this operator * discards the previously retained element. * * @return a backpressured {@link Flux} that will only keep a reference to the last observed item */
public final Flux<T> onBackpressureLatest() { return onAssembly(new FluxOnBackpressureLatest<>(this)); }
Let compatible operators upstream recover from errors by dropping the incriminating element from the sequence and continuing with subsequent elements. Only errors matching the Predicate are recovered from. The recovered error and associated value are notified via the provided BiConsumer.

Note that this error handling mode is not necessarily implemented by all operators (look for the Error Mode Support javadoc section to find operators that support it).

Returns:a Flux that attempts to continue processing on errors.
/** * Let compatible operators <strong>upstream</strong> recover from errors by dropping the * incriminating element from the sequence and continuing with subsequent elements. * Only errors matching the {@link Predicate} are recovered from. * The recovered error and associated value are notified via the provided {@link BiConsumer}. * <p> * Note that this error handling mode is not necessarily implemented by all operators * (look for the {@code Error Mode Support} javadoc section to find operators that * support it). * * @return a {@link Flux} that attempts to continue processing on errors. */
public final Flux<T> onErrorContinue(BiConsumer<Throwable, Object> errorConsumer) { BiConsumer<Throwable, Object> genericConsumer = errorConsumer; return subscriberContext(Context.of( OnNextFailureStrategy.KEY_ON_NEXT_ERROR_STRATEGY, OnNextFailureStrategy.resume(genericConsumer) )); }
Let compatible operators upstream recover from errors by dropping the incriminating element from the sequence and continuing with subsequent elements. Only errors matching the specified type are recovered from. The recovered error and associated value are notified via the provided BiConsumer.

Note that this error handling mode is not necessarily implemented by all operators (look for the Error Mode Support javadoc section to find operators that support it).

Returns:a Flux that attempts to continue processing on some errors.
/** * Let compatible operators <strong>upstream</strong> recover from errors by dropping the * incriminating element from the sequence and continuing with subsequent elements. * Only errors matching the specified {@code type} are recovered from. * The recovered error and associated value are notified via the provided {@link BiConsumer}. * <p> * Note that this error handling mode is not necessarily implemented by all operators * (look for the {@code Error Mode Support} javadoc section to find operators that * support it). * * @return a {@link Flux} that attempts to continue processing on some errors. */
public final <E extends Throwable> Flux<T> onErrorContinue(Class<E> type, BiConsumer<Throwable, Object> errorConsumer) { return onErrorContinue(type::isInstance, errorConsumer); }
Let compatible operators upstream recover from errors by dropping the incriminating element from the sequence and continuing with subsequent elements. Only errors matching the Predicate are recovered from. The recovered error and associated value are notified via the provided BiConsumer.

Note that this error handling mode is not necessarily implemented by all operators (look for the Error Mode Support javadoc section to find operators that support it).

Returns:a Flux that attempts to continue processing on some errors.
/** * Let compatible operators <strong>upstream</strong> recover from errors by dropping the * incriminating element from the sequence and continuing with subsequent elements. * Only errors matching the {@link Predicate} are recovered from. * The recovered error and associated value are notified via the provided {@link BiConsumer}. * <p> * Note that this error handling mode is not necessarily implemented by all operators * (look for the {@code Error Mode Support} javadoc section to find operators that * support it). * * @return a {@link Flux} that attempts to continue processing on some errors. */
public final <E extends Throwable> Flux<T> onErrorContinue(Predicate<E> errorPredicate, BiConsumer<Throwable, Object> errorConsumer) { //this cast is ok as only T values will be propagated in this sequence @SuppressWarnings("unchecked") Predicate<Throwable> genericPredicate = (Predicate<Throwable>) errorPredicate; BiConsumer<Throwable, Object> genericErrorConsumer = errorConsumer; return subscriberContext(Context.of( OnNextFailureStrategy.KEY_ON_NEXT_ERROR_STRATEGY, OnNextFailureStrategy.resumeIf(genericPredicate, genericErrorConsumer) )); }
If an onErrorContinue(BiConsumer) variant has been used before, reverts to the default 'STOP' mode where errors are terminal events. It can be used for easier scoping of the on next failure strategy or to override the inherited strategy in a sub-stream (for example in a flatMap). It has no effect if onErrorContinue(BiConsumer) has not been used.
Returns:a Flux that terminates on errors, even if onErrorContinue(BiConsumer) was used before
/** * If an {@link #onErrorContinue(BiConsumer)} variant has been used before, reverts to the default * 'STOP' mode where errors are terminal events. It can be used for easier scoping of * the on next failure strategy or to override the inherited strategy in a sub-stream * (for example in a flatMap). It has no effect if {@link #onErrorContinue(BiConsumer)} has not * been used. * * @return a {@link Flux} that terminates on errors, even if {@link #onErrorContinue(BiConsumer)} * was used before */
public final Flux<T> onErrorStop() { return subscriberContext(Context.of( OnNextFailureStrategy.KEY_ON_NEXT_ERROR_STRATEGY, OnNextFailureStrategy.stop())); }
Transform any error emitted by this Flux by synchronously applying a function to it.

Params:
  • mapper – the error transforming Function
Returns:a Flux that transforms source errors to other errors
/** * Transform any error emitted by this {@link Flux} by synchronously applying a function to it. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/maperror.png" * alt=""> * <p> * * @param mapper the error transforming {@link Function} * * @return a {@link Flux} that transforms source errors to other errors */
public final Flux<T> onErrorMap(Function<? super Throwable, ? extends Throwable> mapper) { return onErrorResume(e -> Mono.error(mapper.apply(e))); }
Transform an error emitted by this Flux by synchronously applying a function to it if the error matches the given type. Otherwise let the error pass through.

Params:
  • type – the class of the exception type to react to
  • mapper – the error transforming Function
Type parameters:
  • <E> – the error type
Returns:a Flux that transforms some source errors to other errors
/** * Transform an error emitted by this {@link Flux} by synchronously applying a function * to it if the error matches the given type. Otherwise let the error pass through. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/maperror.png" alt=""> * <p> * @param type the class of the exception type to react to * @param mapper the error transforming {@link Function} * @param <E> the error type * * @return a {@link Flux} that transforms some source errors to other errors */
public final <E extends Throwable> Flux<T> onErrorMap(Class<E> type, Function<? super E, ? extends Throwable> mapper) { @SuppressWarnings("unchecked") Function<Throwable, Throwable> handler = (Function<Throwable, Throwable>)mapper; return onErrorMap(type::isInstance, handler); }
Transform an error emitted by this Flux by synchronously applying a function to it if the error matches the given predicate. Otherwise let the error pass through.

Params:
  • predicate – the error predicate
  • mapper – the error transforming Function
Returns:a Flux that transforms some source errors to other errors
/** * Transform an error emitted by this {@link Flux} by synchronously applying a function * to it if the error matches the given predicate. Otherwise let the error pass through. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/maperror.png" * alt=""> * * @param predicate the error predicate * @param mapper the error transforming {@link Function} * * @return a {@link Flux} that transforms some source errors to other errors */
public final Flux<T> onErrorMap(Predicate<? super Throwable> predicate, Function<? super Throwable, ? extends Throwable> mapper) { return onErrorResume(predicate, e -> Mono.error(mapper.apply(e))); }
Subscribe to a returned fallback publisher when any error occurs, using a function to choose the fallback depending on the error.

Params:
  • fallback – the function to choose the fallback to an alternative Publisher
Returns:a Flux falling back upon source onError
/** * Subscribe to a returned fallback publisher when any error occurs, using a function to * choose the fallback depending on the error. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.2.0.M2/src/docs/marble/onerrorresume.png" alt=""> * <p> * @param fallback the function to choose the fallback to an alternative {@link Publisher} * * @return a {@link Flux} falling back upon source onError */
public final Flux<T> onErrorResume(Function<? super Throwable, ? extends Publisher<? extends T>> fallback) { return onAssembly(new FluxOnErrorResume<>(this, fallback)); }
Subscribe to a fallback publisher when an error matching the given type occurs, using a function to choose the fallback depending on the error.

Params:
  • type – the error type to match
  • fallback – the function to choose the fallback to an alternative Publisher
Type parameters:
  • <E> – the error type
Returns:a Flux falling back upon source onError
/** * Subscribe to a fallback publisher when an error matching the given type * occurs, using a function to choose the fallback depending on the error. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.2.0.M2/src/docs/marble/onerrorresume.png" * alt=""> * * @param type the error type to match * @param fallback the function to choose the fallback to an alternative {@link Publisher} * @param <E> the error type * * @return a {@link Flux} falling back upon source onError */
public final <E extends Throwable> Flux<T> onErrorResume(Class<E> type, Function<? super E, ? extends Publisher<? extends T>> fallback) { Objects.requireNonNull(type, "type"); @SuppressWarnings("unchecked") Function<? super Throwable, Publisher<? extends T>> handler = (Function<? super Throwable, Publisher<? extends T>>)fallback; return onErrorResume(type::isInstance, handler); }
Subscribe to a fallback publisher when an error matching a given predicate occurs.

Params:
  • predicate – the error predicate to match
  • fallback – the function to choose the fallback to an alternative Publisher
Returns:a Flux falling back upon source onError
/** * Subscribe to a fallback publisher when an error matching a given predicate * occurs. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.2.0.M2/src/docs/marble/onerrorresume.png" * alt=""> * * @param predicate the error predicate to match * @param fallback the function to choose the fallback to an alternative {@link Publisher} * * @return a {@link Flux} falling back upon source onError */
public final Flux<T> onErrorResume(Predicate<? super Throwable> predicate, Function<? super Throwable, ? extends Publisher<? extends T>> fallback) { Objects.requireNonNull(predicate, "predicate"); return onErrorResume(e -> predicate.test(e) ? fallback.apply(e) : error(e)); }
Simply emit a captured fallback value when any error is observed on this Flux.

Params:
  • fallbackValue – the value to emit if an error occurs
Returns:a new falling back Flux
/** * Simply emit a captured fallback value when any error is observed on this {@link Flux}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/onerrorreturn.png" alt=""> * <p> * @param fallbackValue the value to emit if an error occurs * * @return a new falling back {@link Flux} */
public final Flux<T> onErrorReturn(T fallbackValue) { return onErrorResume(t -> just(fallbackValue)); }
Simply emit a captured fallback value when an error of the specified type is observed on this Flux.

Params:
  • type – the error type to match
  • fallbackValue – the value to emit if an error occurs that matches the type
Type parameters:
  • <E> – the error type
Returns:a new falling back Flux
/** * Simply emit a captured fallback value when an error of the specified type is * observed on this {@link Flux}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/onerrorreturn.png" alt=""> * @param type the error type to match * @param fallbackValue the value to emit if an error occurs that matches the type * @param <E> the error type * * @return a new falling back {@link Flux} */
public final <E extends Throwable> Flux<T> onErrorReturn(Class<E> type, T fallbackValue) { return onErrorResume(type, t -> just(fallbackValue)); }
Simply emit a captured fallback value when an error matching the given predicate is observed on this Flux.

Params:
  • predicate – the error predicate to match
  • fallbackValue – the value to emit if an error occurs that matches the predicate
Returns:a new falling back Flux
/** * Simply emit a captured fallback value when an error matching the given predicate is * observed on this {@link Flux}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/onerrorreturn.png" alt=""> * @param predicate the error predicate to match * @param fallbackValue the value to emit if an error occurs that matches the predicate * * @return a new falling back {@link Flux} */
public final Flux<T> onErrorReturn(Predicate<? super Throwable> predicate, T fallbackValue) { return onErrorResume(predicate, t -> just(fallbackValue)); }
Detaches both the child Subscriber and the Subscription on termination or cancellation.

This is an advanced interoperability operator that should help with odd retention scenarios when running with non-reactor Subscriber.

Returns:a detachable Flux
/** * Detaches both the child {@link Subscriber} and the {@link Subscription} on * termination or cancellation. * <p>This is an advanced interoperability operator that should help with odd * retention scenarios when running with non-reactor {@link Subscriber}. * * @return a detachable {@link Flux} */
public final Flux<T> onTerminateDetach() { return new FluxDetach<>(this); }
Pick the first Publisher between this Flux and another publisher to emit any signal (onNext/onError/onComplete) and replay all signals from that Publisher, effectively behaving like the fastest of these competing sources.

Params:
See Also:
Returns:the fastest sequence
/** * Pick the first {@link Publisher} between this {@link Flux} and another publisher * to emit any signal (onNext/onError/onComplete) and replay all signals from that * {@link Publisher}, effectively behaving like the fastest of these competing sources. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/firstemitting.png" alt=""> * <p> * @param other the {@link Publisher} to race with * * @return the fastest sequence * @see #first */
public final Flux<T> or(Publisher<? extends T> other) { if (this instanceof FluxFirstEmitting) { FluxFirstEmitting<T> publisherAmb = (FluxFirstEmitting<T>) this; FluxFirstEmitting<T> result = publisherAmb.ambAdditionalSource(other); if (result != null) { return result; } } return first(this, other); }
Prepare this Flux by dividing data on a number of 'rails' matching the number of CPU cores, in a round-robin fashion. Note that to actually perform the work in parallel, you should call ParallelFlux.runOn(Scheduler) afterward.

Returns:a new ParallelFlux instance
/** * Prepare this {@link Flux} by dividing data on a number of 'rails' matching the * number of CPU cores, in a round-robin fashion. Note that to actually perform the * work in parallel, you should call {@link ParallelFlux#runOn(Scheduler)} afterward. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/parallel.png" alt=""> * * @return a new {@link ParallelFlux} instance */
public final ParallelFlux<T> parallel() { return parallel(Runtime.getRuntime() .availableProcessors()); }
Prepare this Flux by dividing data on a number of 'rails' matching the provided parallelism parameter, in a round-robin fashion. Note that to actually perform the work in parallel, you should call ParallelFlux.runOn(Scheduler) afterward.

Params:
  • parallelism – the number of parallel rails
Returns:a new ParallelFlux instance
/** * Prepare this {@link Flux} by dividing data on a number of 'rails' matching the * provided {@code parallelism} parameter, in a round-robin fashion. Note that to * actually perform the work in parallel, you should call {@link ParallelFlux#runOn(Scheduler)} * afterward. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/parallel.png" alt=""> * * @param parallelism the number of parallel rails * * @return a new {@link ParallelFlux} instance */
public final ParallelFlux<T> parallel(int parallelism) { return parallel(parallelism, Queues.SMALL_BUFFER_SIZE); }
Prepare this Flux by dividing data on a number of 'rails' matching the provided parallelism parameter, in a round-robin fashion and using a custom prefetch amount and queue for dealing with the source Flux's values. Note that to actually perform the work in parallel, you should call ParallelFlux.runOn(Scheduler) afterward.

Params:
  • parallelism – the number of parallel rails
  • prefetch – the number of values to prefetch from the source
Returns:a new ParallelFlux instance
/** * Prepare this {@link Flux} by dividing data on a number of 'rails' matching the * provided {@code parallelism} parameter, in a round-robin fashion and using a * custom prefetch amount and queue for dealing with the source {@link Flux}'s values. * Note that to actually perform the work in parallel, you should call * {@link ParallelFlux#runOn(Scheduler)} afterward. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/parallel.png" alt=""> * * @param parallelism the number of parallel rails * @param prefetch the number of values to prefetch from the source * * @return a new {@link ParallelFlux} instance */
public final ParallelFlux<T> parallel(int parallelism, int prefetch) { return ParallelFlux.from(this, parallelism, prefetch, Queues.get(prefetch)); }
Prepare a ConnectableFlux which shares this Flux sequence and dispatches values to subscribers in a backpressure-aware manner. Prefetch will default to Queues.SMALL_BUFFER_SIZE. This will effectively turn any type of sequence into a hot sequence.

Backpressure will be coordinated on Subscription.request and if any Subscriber is missing demand (requested = 0), multicast will pause pushing/pulling.

Returns:a new ConnectableFlux
/** * Prepare a {@link ConnectableFlux} which shares this {@link Flux} sequence and * dispatches values to subscribers in a backpressure-aware manner. Prefetch will * default to {@link Queues#SMALL_BUFFER_SIZE}. This will effectively turn * any type of sequence into a hot sequence. * <p> * Backpressure will be coordinated on {@link Subscription#request} and if any * {@link Subscriber} is missing demand (requested = 0), multicast will pause * pushing/pulling. * <p> * <img width="500" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/publish.png" alt=""> * * @return a new {@link ConnectableFlux} */
public final ConnectableFlux<T> publish() { return publish(Queues.SMALL_BUFFER_SIZE); }
Prepare a ConnectableFlux which shares this Flux sequence and dispatches values to subscribers in a backpressure-aware manner. This will effectively turn any type of sequence into a hot sequence.

Backpressure will be coordinated on Subscription.request and if any Subscriber is missing demand (requested = 0), multicast will pause pushing/pulling.

Params:
  • prefetch – bounded requested demand
Returns:a new ConnectableFlux
/** * Prepare a {@link ConnectableFlux} which shares this {@link Flux} sequence and * dispatches values to subscribers in a backpressure-aware manner. This will * effectively turn any type of sequence into a hot sequence. * <p> * Backpressure will be coordinated on {@link Subscription#request} and if any * {@link Subscriber} is missing demand (requested = 0), multicast will pause * pushing/pulling. * <p> * <img width="500" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/publish.png" alt=""> * * @param prefetch bounded requested demand * * @return a new {@link ConnectableFlux} */
public final ConnectableFlux<T> publish(int prefetch) { return onAssembly(new FluxPublish<>(this, prefetch, Queues .get(prefetch))); }
Shares a sequence for the duration of a function that may transform it and consume it as many times as necessary without causing multiple subscriptions to the upstream.
Params:
  • transform – the transformation function
Type parameters:
  • <R> – the output value type
Returns:a new Flux
/** * Shares a sequence for the duration of a function that may transform it and * consume it as many times as necessary without causing multiple subscriptions * to the upstream. * * @param transform the transformation function * @param <R> the output value type * * @return a new {@link Flux} */
public final <R> Flux<R> publish(Function<? super Flux<T>, ? extends Publisher<? extends R>> transform) { return publish(transform, Queues.SMALL_BUFFER_SIZE); }
Shares a sequence for the duration of a function that may transform it and consume it as many times as necessary without causing multiple subscriptions to the upstream.
Params:
  • transform – the transformation function
  • prefetch – the request size
Type parameters:
  • <R> – the output value type
Returns:a new Flux
/** * Shares a sequence for the duration of a function that may transform it and * consume it as many times as necessary without causing multiple subscriptions * to the upstream. * * @param transform the transformation function * @param prefetch the request size * @param <R> the output value type * * @return a new {@link Flux} */
public final <R> Flux<R> publish(Function<? super Flux<T>, ? extends Publisher<? extends R>> transform, int prefetch) { return onAssembly(new FluxPublishMulticast<>(this, transform, prefetch, Queues .get(prefetch))); }
Prepare a Mono which shares this Flux sequence and dispatches the first observed item to subscribers in a backpressure-aware manner. This will effectively turn any type of sequence into a hot sequence when the first Subscriber subscribes.

Returns:a new Mono
/** * Prepare a {@link Mono} which shares this {@link Flux} sequence and dispatches the * first observed item to subscribers in a backpressure-aware manner. * This will effectively turn any type of sequence into a hot sequence when the first * {@link Subscriber} subscribes. * <p> * <img width="500" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/publishnext.png" alt=""> * * @return a new {@link Mono} */
public final Mono<T> publishNext() { return Mono.onAssembly(new MonoProcessor<>(this)); }
Run onNext, onComplete and onError on a supplied Scheduler Worker.

This operator influences the threading context where the rest of the operators in the chain below it will execute, up to a new occurrence of publishOn.

Typically used for fast publisher, slow consumer(s) scenarios.

flux.publishOn(Schedulers.single()).subscribe()  
Params:
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a Flux producing asynchronously on a given Scheduler
/** * Run onNext, onComplete and onError on a supplied {@link Scheduler} * {@link Worker Worker}. * <p> * This operator influences the threading context where the rest of the operators in * the chain below it will execute, up to a new occurrence of {@code publishOn}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/publishon.png" alt=""> * <p> * Typically used for fast publisher, slow consumer(s) scenarios. * <blockquote><pre> * {@code flux.publishOn(Schedulers.single()).subscribe() } * </pre></blockquote> * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal. * * @param scheduler a {@link Scheduler} providing the {@link Worker} where to publish * * @return a {@link Flux} producing asynchronously on a given {@link Scheduler} */
public final Flux<T> publishOn(Scheduler scheduler) { return publishOn(scheduler, Queues.SMALL_BUFFER_SIZE); }
Run onNext, onComplete and onError on a supplied Scheduler Worker.

This operator influences the threading context where the rest of the operators in the chain below it will execute, up to a new occurrence of publishOn.

Typically used for fast publisher, slow consumer(s) scenarios.

flux.publishOn(Schedulers.single()).subscribe()  
Params:
  • scheduler – a Scheduler providing the Worker where to publish
  • prefetch – the asynchronous boundary capacity
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a Flux producing asynchronously
/** * Run onNext, onComplete and onError on a supplied {@link Scheduler} * {@link Worker}. * <p> * This operator influences the threading context where the rest of the operators in * the chain below it will execute, up to a new occurrence of {@code publishOn}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/publishon.png" alt=""> * <p> * Typically used for fast publisher, slow consumer(s) scenarios. * <blockquote><pre> * {@code flux.publishOn(Schedulers.single()).subscribe() } * </pre></blockquote> * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal. * * @param scheduler a {@link Scheduler} providing the {@link Worker} where to publish * @param prefetch the asynchronous boundary capacity * * @return a {@link Flux} producing asynchronously */
public final Flux<T> publishOn(Scheduler scheduler, int prefetch) { return publishOn(scheduler, true, prefetch); }
Run onNext, onComplete and onError on a supplied Scheduler Worker.

This operator influences the threading context where the rest of the operators in the chain below it will execute, up to a new occurrence of publishOn.

Typically used for fast publisher, slow consumer(s) scenarios.

flux.publishOn(Schedulers.single()).subscribe()  
Params:
  • scheduler – a Scheduler providing the Worker where to publish
  • delayError – should the buffer be consumed before forwarding any error
  • prefetch – the asynchronous boundary capacity
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a Flux producing asynchronously
/** * Run onNext, onComplete and onError on a supplied {@link Scheduler} * {@link Worker}. * <p> * This operator influences the threading context where the rest of the operators in * the chain below it will execute, up to a new occurrence of {@code publishOn}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/publishon.png" alt=""> * <p> * Typically used for fast publisher, slow consumer(s) scenarios. * <blockquote><pre> * {@code flux.publishOn(Schedulers.single()).subscribe() } * </pre></blockquote> * * @reactor.discard This operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal. * * @param scheduler a {@link Scheduler} providing the {@link Worker} where to publish * @param delayError should the buffer be consumed before forwarding any error * @param prefetch the asynchronous boundary capacity * * @return a {@link Flux} producing asynchronously */
public final Flux<T> publishOn(Scheduler scheduler, boolean delayError, int prefetch) { return publishOn(scheduler, delayError, prefetch, prefetch); } final Flux<T> publishOn(Scheduler scheduler, boolean delayError, int prefetch, int lowTide) { if (this instanceof Callable) { if (this instanceof Fuseable.ScalarCallable) { @SuppressWarnings("unchecked") Fuseable.ScalarCallable<T> s = (Fuseable.ScalarCallable<T>) this; try { return onAssembly(new FluxSubscribeOnValue<>(s.call(), scheduler)); } catch (Exception e) { //leave FluxSubscribeOnCallable defer exception call } } @SuppressWarnings("unchecked") Callable<T> c = (Callable<T>)this; return onAssembly(new FluxSubscribeOnCallable<>(c, scheduler)); } return onAssembly(new FluxPublishOn<>(this, scheduler, delayError, prefetch, lowTide, Queues.get(prefetch))); }
Reduce the values from this Flux sequence into a single object of the same type than the emitted items. Reduction is performed using a BiFunction that takes the intermediate result of the reduction and the current value and returns the next intermediate value of the reduction. Note, BiFunction will not be invoked for a sequence with 0 or 1 elements. In case of one element's sequence, the result will be directly sent to the subscriber.

Params:
Returns:a reduced Flux
/** * Reduce the values from this {@link Flux} sequence into a single object of the same * type than the emitted items. Reduction is performed using a {@link BiFunction} that * takes the intermediate result of the reduction and the current value and returns * the next intermediate value of the reduction. Note, {@link BiFunction} will not * be invoked for a sequence with 0 or 1 elements. In case of one element's * sequence, the result will be directly sent to the subscriber. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/aggregate.png" alt=""> * * @param aggregator the reducing {@link BiFunction} * * @return a reduced {@link Flux} */
public final Mono<T> reduce(BiFunction<T, T, T> aggregator) { if (this instanceof Callable){ @SuppressWarnings("unchecked") Callable<T> thiz = (Callable<T>)this; return convertToMono(thiz); } return Mono.onAssembly(new MonoReduce<>(this, aggregator)); }
Reduce the values from this Flux sequence into a single object matching the type of a seed value. Reduction is performed using a BiFunction that takes the intermediate result of the reduction and the current value and returns the next intermediate value of the reduction. First element is paired with the seed value, initial.

Params:
  • accumulator – the reducing BiFunction
  • initial – the seed, the initial leftmost argument to pass to the reducing BiFunction
Type parameters:
  • <A> – the type of the seed and the reduced object
Returns:a reduced Flux
/** * Reduce the values from this {@link Flux} sequence into a single object matching the * type of a seed value. Reduction is performed using a {@link BiFunction} that * takes the intermediate result of the reduction and the current value and returns * the next intermediate value of the reduction. First element is paired with the seed * value, {@literal initial}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/reduce.png" alt=""> * * @param accumulator the reducing {@link BiFunction} * @param initial the seed, the initial leftmost argument to pass to the reducing {@link BiFunction} * @param <A> the type of the seed and the reduced object * * @return a reduced {@link Flux} * */
public final <A> Mono<A> reduce(A initial, BiFunction<A, ? super T, A> accumulator) { return reduceWith(() -> initial, accumulator); }
Reduce the values from this Flux sequence into a single object matching the type of a lazily supplied seed value. Reduction is performed using a BiFunction that takes the intermediate result of the reduction and the current value and returns the next intermediate value of the reduction. First element is paired with the seed value, supplied via initial.

Params:
Type parameters:
  • <A> – the type of the seed and the reduced object
Returns:a reduced Flux
/** * Reduce the values from this {@link Flux} sequence into a single object matching the * type of a lazily supplied seed value. Reduction is performed using a * {@link BiFunction} that takes the intermediate result of the reduction and the * current value and returns the next intermediate value of the reduction. First * element is paired with the seed value, supplied via {@literal initial}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/reduce.png" alt=""> * * @param accumulator the reducing {@link BiFunction} * @param initial a {@link Supplier} of the seed, called on subscription and passed to the the reducing {@link BiFunction} * @param <A> the type of the seed and the reduced object * * @return a reduced {@link Flux} * */
public final <A> Mono<A> reduceWith(Supplier<A> initial, BiFunction<A, ? super T, A> accumulator) { return Mono.onAssembly(new MonoReduceSeed<>(this, initial, accumulator)); }
Repeatedly and indefinitely subscribe to the source upon completion of the previous subscription.

Returns:an indefinitely repeated Flux on onComplete
/** * Repeatedly and indefinitely subscribe to the source upon completion of the * previous subscription. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/repeat.png" alt=""> * * @return an indefinitely repeated {@link Flux} on onComplete */
public final Flux<T> repeat() { return repeat(ALWAYS_BOOLEAN_SUPPLIER); }
Repeatedly subscribe to the source if the predicate returns true after completion of the previous subscription.

Params:
  • predicate – the boolean to evaluate on onComplete.
Returns:a Flux that repeats on onComplete while the predicate matches
/** * Repeatedly subscribe to the source if the predicate returns true after completion of the previous subscription. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/repeatb.png" alt=""> * * @param predicate the boolean to evaluate on onComplete. * * @return a {@link Flux} that repeats on onComplete while the predicate matches */
public final Flux<T> repeat(BooleanSupplier predicate) { return onAssembly(new FluxRepeatPredicate<>(this, predicate)); }
Repeatedly subscribe to the source numRepeat times. This results in numRepeat + 1 total subscriptions to the original source. As a consequence, using 0 plays the original sequence once.

Params:
  • numRepeat – the number of times to re-subscribe on onComplete (positive, or 0 for original sequence only)
Returns:a Flux that repeats on onComplete, up to the specified number of repetitions
/** * Repeatedly subscribe to the source {@code numRepeat} times. This results in * {@code numRepeat + 1} total subscriptions to the original source. As a consequence, * using 0 plays the original sequence once. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/repeatn.png" alt=""> * * @param numRepeat the number of times to re-subscribe on onComplete (positive, or 0 for original sequence only) * * @return a {@link Flux} that repeats on onComplete, up to the specified number of repetitions */
public final Flux<T> repeat(long numRepeat) { if(numRepeat == 0L){ return this; } return onAssembly(new FluxRepeat<>(this, numRepeat)); }
Repeatedly subscribe to the source if the predicate returns true after completion of the previous subscription. A specified maximum of repeat will limit the number of re-subscribe.

Params:
  • numRepeat – the number of times to re-subscribe on complete (positive, or 0 for original sequence only)
  • predicate – the boolean to evaluate on onComplete
Returns:a Flux that repeats on onComplete while the predicate matches, up to the specified number of repetitions
/** * Repeatedly subscribe to the source if the predicate returns true after completion of the previous * subscription. A specified maximum of repeat will limit the number of re-subscribe. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/repeatnb.png" alt=""> * * @param numRepeat the number of times to re-subscribe on complete (positive, or 0 for original sequence only) * @param predicate the boolean to evaluate on onComplete * * @return a {@link Flux} that repeats on onComplete while the predicate matches, * up to the specified number of repetitions */
public final Flux<T> repeat(long numRepeat, BooleanSupplier predicate) { if (numRepeat < 0L) { throw new IllegalArgumentException("numRepeat >= 0 required"); } if (numRepeat == 0) { return this; } return defer( () -> repeat(countingBooleanSupplier(predicate, numRepeat))); }
Repeatedly subscribe to this Flux when a companion sequence emits elements in response to the flux completion signal. Any terminal signal from the companion sequence will terminate the resulting Flux with the same signal immediately.

If the companion sequence signals when this Flux is active, the repeat attempt is suppressed.

Note that if the companion Publisher created by the repeatFactory emits Context as trigger objects, these Context will REPLACE the operator's own Context. Please be careful there: replacing the Context means that some keys you don't own could be removed, breaking libraries that depend on them. As a result, the recommended approach is to always create such a Context trigger by starting from the original Context (ensuring the trigger contains all the keys from the original, unless you absolutely know you want to remove one of these keys):


.repeatWhen(emittedEachAttempt -> emittedEachAttempt
    .flatMap(e -> Mono.subscriberContext().map(ctx -> Tuples.of(e, ctx)))
    .flatMap(t2 -> {
	    long lastEmitted = t2.getT1();
	    Context ctx = t2.getT2();
	    int rl = ctx.getOrDefault("repeatsLeft", 0);
	    if (rl > 0) {
	    // /!\ THE ctx.put HERE IS THE ESSENTIAL PART /!\
		    return Mono.just(ctx.put("repeatsLeft", rl - 1)
				    .put("emitted", lastEmitted));
	    } else {
		    return Mono.error(new IllegalStateException("repeats exhausted"));
	    }
    })
)
Params:
  • repeatFactory – the Function that returns the associated Publisher companion, given a Flux that signals each onComplete as a Long representing the number of source elements emitted in the latest attempt.
Returns:a Flux that repeats on onComplete when the companion Publisher produces an onNext signal
/** * Repeatedly subscribe to this {@link Flux} when a companion sequence emits elements in * response to the flux completion signal. Any terminal signal from the companion * sequence will terminate the resulting {@link Flux} with the same signal immediately. * <p>If the companion sequence signals when this {@link Flux} is active, the repeat * attempt is suppressed. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/repeatwhen.png" alt=""> * <p> * Note that if the companion {@link Publisher} created by the {@code repeatFactory} * emits {@link Context} as trigger objects, these {@link Context} will REPLACE the * operator's own Context. <strong>Please be careful there</strong>: replacing the * Context means that some keys you don't own could be removed, breaking libraries * that depend on them. As a result, the recommended approach is to always create such * a {@link Context} trigger by starting from the original Context (ensuring the trigger * contains all the keys from the original, unless you absolutely know you want to * remove one of these keys): * <pre><code> * .repeatWhen(emittedEachAttempt -> emittedEachAttempt * .flatMap(e -> Mono.subscriberContext().map(ctx -> Tuples.of(e, ctx))) * .flatMap(t2 -> { * long lastEmitted = t2.getT1(); * Context ctx = t2.getT2(); * int rl = ctx.getOrDefault("repeatsLeft", 0); * if (rl > 0) { * // /!\ THE ctx.put HERE IS THE ESSENTIAL PART /!\ * return Mono.just(ctx.put("repeatsLeft", rl - 1) * .put("emitted", lastEmitted)); * } else { * return Mono.<Context>error(new IllegalStateException("repeats exhausted")); * } * }) * ) * </code></pre> * * @param repeatFactory the {@link Function} that returns the associated {@link Publisher} * companion, given a {@link Flux} that signals each onComplete as a {@link Long} * representing the number of source elements emitted in the latest attempt. * * @return a {@link Flux} that repeats on onComplete when the companion {@link Publisher} produces an * onNext signal */
public final Flux<T> repeatWhen(Function<Flux<Long>, ? extends Publisher<?>> repeatFactory) { return onAssembly(new FluxRepeatWhen<>(this, repeatFactory)); }
Turn this Flux into a hot source and cache last emitted signals for further Subscriber. Will retain an unbounded amount of onNext signals. Completion and Error will also be replayed.

Returns:a replaying ConnectableFlux
/** * Turn this {@link Flux} into a hot source and cache last emitted signals for further {@link Subscriber}. Will * retain an unbounded amount of onNext signals. Completion and Error will also be * replayed. * <p> * <img width="500" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/replay.png" * alt=""> * * @return a replaying {@link ConnectableFlux} */
public final ConnectableFlux<T> replay() { return replay(Integer.MAX_VALUE); }
Turn this Flux into a connectable hot source and cache last emitted signals for further Subscriber. Will retain up to the given history size onNext signals. Completion and Error will also be replayed.

Note that cache(0) will only cache the terminal signal without expiration.

Params:
  • history – number of events retained in history excluding complete and error
Returns:a replaying ConnectableFlux
/** * Turn this {@link Flux} into a connectable hot source and cache last emitted * signals for further {@link Subscriber}. * Will retain up to the given history size onNext signals. Completion and Error will also be * replayed. * <p> * Note that {@code cache(0)} will only cache the terminal signal without * expiration. * * <p> * <img width="500" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/replay.png" alt=""> * * @param history number of events retained in history excluding complete and * error * * @return a replaying {@link ConnectableFlux} * */
public final ConnectableFlux<T> replay(int history) { return onAssembly(new FluxReplay<>(this, history, 0L, null)); }
Turn this Flux into a connectable hot source and cache last emitted signals for further Subscriber. Will retain each onNext up to the given per-item expiry timeout.

Completion and Error will also be replayed until ttl triggers in which case the next Subscriber will start over a new subscription

Params:
  • ttl – Per-item and post termination timeout duration
Returns:a replaying ConnectableFlux
/** * Turn this {@link Flux} into a connectable hot source and cache last emitted signals * for further {@link Subscriber}. Will retain each onNext up to the given per-item * expiry timeout. * <p> * Completion and Error will also be replayed until {@code ttl} triggers in which case * the next {@link Subscriber} will start over a new subscription * * <p> * <img width="500" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/replay.png" * alt=""> * * @param ttl Per-item and post termination timeout duration * * @return a replaying {@link ConnectableFlux} */
public final ConnectableFlux<T> replay(Duration ttl) { return replay(Integer.MAX_VALUE, ttl); }
Turn this Flux into a connectable hot source and cache last emitted signals for further Subscriber. Will retain up to the given history size onNext signals with a per-item ttl.

Completion and Error will also be replayed until ttl triggers in which case the next Subscriber will start over a new subscription

Params:
  • history – number of events retained in history excluding complete and error
  • ttl – Per-item and post termination timeout duration
Returns:a replaying ConnectableFlux
/** * Turn this {@link Flux} into a connectable hot source and cache last emitted signals * for further {@link Subscriber}. Will retain up to the given history size onNext * signals with a per-item ttl. * <p> * Completion and Error will also be replayed until {@code ttl} triggers in which case * the next {@link Subscriber} will start over a new subscription * * <p> * <img width="500" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/replay.png" * alt=""> * * @param history number of events retained in history excluding complete and error * @param ttl Per-item and post termination timeout duration * * @return a replaying {@link ConnectableFlux} */
public final ConnectableFlux<T> replay(int history, Duration ttl) { return replay(history, ttl, Schedulers.parallel()); }
Turn this Flux into a connectable hot source and cache last emitted signals for further Subscriber. Will retain onNext signal for up to the given Duration with a per-item ttl.

Completion and Error will also be replayed until ttl triggers in which case the next Subscriber will start over a new subscription

Params:
  • ttl – Per-item and post termination timeout duration
  • timer – a time-capable Scheduler instance to read current time from
Returns:a replaying ConnectableFlux
/** * Turn this {@link Flux} into a connectable hot source and cache last emitted signals * for further {@link Subscriber}. Will retain onNext signal for up to the given * {@link Duration} with a per-item ttl. * <p> * Completion and Error will also be replayed until {@code ttl} triggers in which case * the next {@link Subscriber} will start over a new subscription * <p> * <img width="500" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/replay.png" * alt=""> * * @param ttl Per-item and post termination timeout duration * @param timer a time-capable {@link Scheduler} instance to read current time from * * @return a replaying {@link ConnectableFlux} */
public final ConnectableFlux<T> replay(Duration ttl, Scheduler timer) { return replay(Integer.MAX_VALUE, ttl, timer); }
Turn this Flux into a connectable hot source and cache last emitted signals for further Subscriber. Will retain up to the given history size onNext signals with a per-item ttl.

Completion and Error will also be replayed until ttl triggers in which case the next Subscriber will start over a new subscription

Params:
  • history – number of events retained in history excluding complete and error
  • ttl – Per-item and post termination timeout duration
  • timer – a Scheduler instance to read current time from
Returns:a replaying ConnectableFlux
/** * Turn this {@link Flux} into a connectable hot source and cache last emitted signals * for further {@link Subscriber}. Will retain up to the given history size onNext * signals with a per-item ttl. * <p> * Completion and Error will also be replayed until {@code ttl} triggers in which case * the next {@link Subscriber} will start over a new subscription * <p> * <img width="500" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/replay.png" * alt=""> * * @param history number of events retained in history excluding complete and error * @param ttl Per-item and post termination timeout duration * @param timer a {@link Scheduler} instance to read current time from * * @return a replaying {@link ConnectableFlux} */
public final ConnectableFlux<T> replay(int history, Duration ttl, Scheduler timer) { Objects.requireNonNull(timer, "timer"); return onAssembly(new FluxReplay<>(this, history, ttl.toMillis(), timer)); }
Re-subscribes to this Flux sequence if it signals any error, indefinitely.

Returns:a Flux that retries on onError
/** * Re-subscribes to this {@link Flux} sequence if it signals any error, indefinitely. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/retry.png" alt=""> * * @return a {@link Flux} that retries on onError */
public final Flux<T> retry() { return retry(Long.MAX_VALUE); }
Re-subscribes to this Flux sequence if it signals any error, for a fixed number of times.

Note that passing Long.MAX_VALUE is treated as infinite retry.

Params:
  • numRetries – the number of times to tolerate an error
Returns:a Flux that retries on onError up to the specified number of retry attempts.
/** * Re-subscribes to this {@link Flux} sequence if it signals any error, for a fixed * number of times. * <p> * Note that passing {@literal Long.MAX_VALUE} is treated as infinite retry. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/retryn.png" alt=""> * * @param numRetries the number of times to tolerate an error * * @return a {@link Flux} that retries on onError up to the specified number of retry attempts. * */
public final Flux<T> retry(long numRetries) { return onAssembly(new FluxRetry<>(this, numRetries)); }
Re-subscribes to this Flux sequence if it signals any error that matches the given Predicate, otherwise push the error downstream.

Params:
  • retryMatcher – the predicate to evaluate if retry should occur based on a given error signal
Returns:a Flux that retries on onError if the predicates matches.
/** * Re-subscribes to this {@link Flux} sequence if it signals any error * that matches the given {@link Predicate}, otherwise push the error downstream. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/retryb.png" alt=""> * * @param retryMatcher the predicate to evaluate if retry should occur based on a given error signal * * @return a {@link Flux} that retries on onError if the predicates matches. */
public final Flux<T> retry(Predicate<? super Throwable> retryMatcher) { return onAssembly(new FluxRetryPredicate<>(this, retryMatcher)); }
Re-subscribes to this Flux sequence up to the specified number of retries if it signals any error that match the given Predicate, otherwise push the error downstream.

Params:
  • numRetries – the number of times to tolerate an error
  • retryMatcher – the predicate to evaluate if retry should occur based on a given error signal
Returns:a Flux that retries on onError up to the specified number of retry attempts, only if the predicate matches.
/** * Re-subscribes to this {@link Flux} sequence up to the specified number of retries if it signals any * error that match the given {@link Predicate}, otherwise push the error downstream. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/retrynb.png" alt=""> * * @param numRetries the number of times to tolerate an error * @param retryMatcher the predicate to evaluate if retry should occur based on a given error signal * * @return a {@link Flux} that retries on onError up to the specified number of retry * attempts, only if the predicate matches. */
public final Flux<T> retry(long numRetries, Predicate<? super Throwable> retryMatcher) { return defer(() -> retry(countingPredicate(retryMatcher, numRetries))); }
Retries this Flux when a companion sequence signals an item in response to this Flux error signal

If the companion sequence signals when the Flux is active, the retry attempt is suppressed and any terminal signal will terminate the Flux source with the same signal immediately.

Note that if the companion Publisher created by the whenFactory emits Context as trigger objects, these Context will REPLACE the operator's own Context. Please be careful there: replacing the Context means that some keys you don't own could be removed, breaking libraries that depend on them. As a result, the recommended approach is to always create such a Context trigger by starting from the original Context (ensuring the trigger contains all the keys from the original, unless you absolutely know you want to remove one of these keys):


.retryWhen(errorCurrentAttempt -> errorCurrentAttempt
    .flatMap(e -> Mono.subscriberContext().map(ctx -> Tuples.of(e, ctx)))
    .flatMap(t2 -> {
	    Throwable lastError = t2.getT1();
	    Context ctx = t2.getT2();
	    int rl = ctx.getOrDefault("retriesLeft", 0);
	    if (rl > 0) {
	    // /!\ THE ctx.put HERE IS THE ESSENTIAL PART /!\
		    return Mono.just(ctx.put("retriesLeft", rl - 1)
				    .put("lastError", lastError));
	    } else {
		    return Mono.error(new IllegalStateException("retries exhausted", lastError));
	    }
    })
)
Params:
Returns:a Flux that retries on onError when the companion Publisher produces an onNext signal
/** * Retries this {@link Flux} when a companion sequence signals * an item in response to this {@link Flux} error signal * <p>If the companion sequence signals when the {@link Flux} is active, the retry * attempt is suppressed and any terminal signal will terminate the {@link Flux} source with the same signal * immediately. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/retrywhen.png" alt=""> * <p> * Note that if the companion {@link Publisher} created by the {@code whenFactory} * emits {@link Context} as trigger objects, these {@link Context} will REPLACE the * operator's own Context. <strong>Please be careful there</strong>: replacing the * Context means that some keys you don't own could be removed, breaking libraries * that depend on them. As a result, the recommended approach is to always create such * a {@link Context} trigger by starting from the original Context (ensuring the trigger * contains all the keys from the original, unless you absolutely know you want to * remove one of these keys): * <pre><code> * .retryWhen(errorCurrentAttempt -> errorCurrentAttempt * .flatMap(e -> Mono.subscriberContext().map(ctx -> Tuples.of(e, ctx))) * .flatMap(t2 -> { * Throwable lastError = t2.getT1(); * Context ctx = t2.getT2(); * int rl = ctx.getOrDefault("retriesLeft", 0); * if (rl > 0) { * // /!\ THE ctx.put HERE IS THE ESSENTIAL PART /!\ * return Mono.just(ctx.put("retriesLeft", rl - 1) * .put("lastError", lastError)); * } else { * return Mono.<Context>error(new IllegalStateException("retries exhausted", lastError)); * } * }) * ) * </code></pre> * * @param whenFactory the {@link Function} that returns the associated {@link Publisher} * companion, given a {@link Flux} that signals each onError as a {@link Throwable}. * * @return a {@link Flux} that retries on onError when the companion {@link Publisher} produces an * onNext signal */
public final Flux<T> retryWhen(Function<Flux<Throwable>, ? extends Publisher<?>> whenFactory) { return onAssembly(new FluxRetryWhen<>(this, whenFactory)); }
In case of error, retry this Flux up to numRetries times using a randomized exponential backoff strategy (jitter). The jitter factor is 50% but the effective backoff delay cannot be less than firstBackoff.

The randomized exponential backoff is good at preventing two typical issues with other simpler backoff strategies, namely:

  • having an exponentially growing backoff delay with a small initial delay gives the best tradeoff between not overwhelming the server and serving the client as fast as possible
  • having a jitter, or randomized backoff delay, is beneficial in avoiding "retry-storms" where eg. numerous clients would hit the server at the same time, causing it to display transient failures which would cause all clients to retry at the same backoff times, ultimately sparing no load on the server.
Params:
  • numRetries – the maximum number of attempts before an IllegalStateException is raised (having the original retry-triggering exception as cause).
  • firstBackoff – the first backoff delay to apply then grow exponentially. Also minimum delay even taking jitter into account.
Returns:a Flux that retries on onError with exponentially growing randomized delays between retries.
/** * In case of error, retry this {@link Flux} up to {@code numRetries} times using a * randomized exponential backoff strategy (jitter). The jitter factor is {@code 50%} * but the effective backoff delay cannot be less than {@code firstBackoff}. * <p> * The randomized exponential backoff is good at preventing two typical issues with * other simpler backoff strategies, namely: * <ul> * <li> * having an exponentially growing backoff delay with a small initial delay gives * the best tradeoff between not overwhelming the server and serving the client as * fast as possible * </li> * <li> * having a jitter, or randomized backoff delay, is beneficial in avoiding "retry-storms" * where eg. numerous clients would hit the server at the same time, causing it to * display transient failures which would cause all clients to retry at the same * backoff times, ultimately sparing no load on the server. * </li> * </ul> * * @param numRetries the maximum number of attempts before an {@link IllegalStateException} * is raised (having the original retry-triggering exception as cause). * @param firstBackoff the first backoff delay to apply then grow exponentially. Also * minimum delay even taking jitter into account. * @return a {@link Flux} that retries on onError with exponentially growing randomized delays between retries. */
public final Flux<T> retryBackoff(long numRetries, Duration firstBackoff) { return retryBackoff(numRetries, firstBackoff, FluxRetryWhen.MAX_BACKOFF, 0.5d); }
In case of error, retry this Flux up to numRetries times using a randomized exponential backoff strategy. The jitter factor is 50% but the effective backoff delay cannot be less than firstBackoff nor more than maxBackoff.

The randomized exponential backoff is good at preventing two typical issues with other simpler backoff strategies, namely:

  • having an exponentially growing backoff delay with a small initial delay gives the best tradeoff between not overwhelming the server and serving the client as fast as possible
  • having a jitter, or randomized backoff delay, is beneficial in avoiding "retry-storms" where eg. numerous clients would hit the server at the same time, causing it to display transient failures which would cause all clients to retry at the same backoff times, ultimately sparing no load on the server.
Params:
  • numRetries – the maximum number of attempts before an IllegalStateException is raised (having the original retry-triggering exception as cause).
  • firstBackoff – the first backoff delay to apply then grow exponentially. Also minimum delay even taking jitter into account.
  • maxBackoff – the maximum delay to apply despite exponential growth and jitter.
Returns:a Flux that retries on onError with exponentially growing randomized delays between retries.
/** * In case of error, retry this {@link Flux} up to {@code numRetries} times using a * randomized exponential backoff strategy. The jitter factor is {@code 50%} * but the effective backoff delay cannot be less than {@code firstBackoff} nor more * than {@code maxBackoff}. <p> * The randomized exponential backoff is good at preventing two typical issues with * other simpler backoff strategies, namely: * <ul> * <li> * having an exponentially growing backoff delay with a small initial delay gives * the best tradeoff between not overwhelming the server and serving the client as * fast as possible * </li> * <li> * having a jitter, or randomized backoff delay, is beneficial in avoiding "retry-storms" * where eg. numerous clients would hit the server at the same time, causing it to * display transient failures which would cause all clients to retry at the same * backoff times, ultimately sparing no load on the server. * </li> * </ul> * * @param numRetries the maximum number of attempts before an {@link IllegalStateException} * is raised (having the original retry-triggering exception as cause). * @param firstBackoff the first backoff delay to apply then grow exponentially. Also * minimum delay even taking jitter into account. * @param maxBackoff the maximum delay to apply despite exponential growth and jitter. * @return a {@link Flux} that retries on onError with exponentially growing randomized delays between retries. */
public final Flux<T> retryBackoff(long numRetries, Duration firstBackoff, Duration maxBackoff) { return retryBackoff(numRetries, firstBackoff, maxBackoff, 0.5d); }
In case of error, retry this Flux up to numRetries times using a randomized exponential backoff strategy, randomized with a user-provided jitter factor between 0.d (no jitter) and 1.0 (default is 0.5). Even with the jitter, the effective backoff delay cannot be less than firstBackoff nor more than maxBackoff.

The randomized exponential backoff is good at preventing two typical issues with other simpler backoff strategies, namely:

  • having an exponentially growing backoff delay with a small initial delay gives the best tradeoff between not overwhelming the server and serving the client as fast as possible
  • having a jitter, or randomized backoff delay, is beneficial in avoiding "retry-storms" where eg. numerous clients would hit the server at the same time, causing it to display transient failures which would cause all clients to retry at the same backoff times, ultimately sparing no load on the server.
Params:
  • numRetries – the maximum number of attempts before an IllegalStateException is raised (having the original retry-triggering exception as cause).
  • firstBackoff – the first backoff delay to apply then grow exponentially. Also minimum delay even taking jitter into account.
  • maxBackoff – the maximum delay to apply despite exponential growth and jitter.
  • jitterFactor – the jitter percentage (as a double between 0.0 and 1.0).
Returns:a Flux that retries on onError with exponentially growing randomized delays between retries.
/** * In case of error, retry this {@link Flux} up to {@code numRetries} times using a * randomized exponential backoff strategy, randomized with a user-provided jitter * factor between {@code 0.d} (no jitter) and {@code 1.0} (default is {@code 0.5}). * Even with the jitter, the effective backoff delay cannot be less than * {@code firstBackoff} nor more than {@code maxBackoff}. <p> * The randomized exponential backoff is good at preventing two typical issues with * other simpler backoff strategies, namely: * <ul> * <li> * having an exponentially growing backoff delay with a small initial delay gives * the best tradeoff between not overwhelming the server and serving the client as * fast as possible * </li> * <li> * having a jitter, or randomized backoff delay, is beneficial in avoiding "retry-storms" * where eg. numerous clients would hit the server at the same time, causing it to * display transient failures which would cause all clients to retry at the same * backoff times, ultimately sparing no load on the server. * </li> * </ul> * * @param numRetries the maximum number of attempts before an {@link IllegalStateException} * is raised (having the original retry-triggering exception as cause). * @param firstBackoff the first backoff delay to apply then grow exponentially. Also * minimum delay even taking jitter into account. * @param maxBackoff the maximum delay to apply despite exponential growth and jitter. * @param jitterFactor the jitter percentage (as a double between 0.0 and 1.0). * @return a {@link Flux} that retries on onError with exponentially growing randomized delays between retries. */
public final Flux<T> retryBackoff(long numRetries, Duration firstBackoff, Duration maxBackoff, double jitterFactor) { return retryWhen(FluxRetryWhen.randomExponentialBackoffFunction(numRetries, firstBackoff, maxBackoff, jitterFactor)); }
Sample this Flux by periodically emitting an item corresponding to that Flux latest emitted value within the periodical time window. Note that if some elements are emitted quicker than the timespan just before source completion, the last of these elements will be emitted along with the onComplete signal.

Params:
  • timespan – the duration of the window after which to emit the latest observed item
@reactor.discardThis operator discards elements that are not part of the sampling.
Returns:a Flux sampled to the last item seen over each periodic window
/** * Sample this {@link Flux} by periodically emitting an item corresponding to that * {@link Flux} latest emitted value within the periodical time window. * Note that if some elements are emitted quicker than the timespan just before source * completion, the last of these elements will be emitted along with the onComplete * signal. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/sampletimespan.png" alt=""> * * @reactor.discard This operator discards elements that are not part of the sampling. * * @param timespan the duration of the window after which to emit the latest observed item * * @return a {@link Flux} sampled to the last item seen over each periodic window */
public final Flux<T> sample(Duration timespan) { return sample(interval(timespan)); }
Sample this Flux by emitting an item corresponding to that Flux latest emitted value whenever a companion sampler Publisher signals a value.

Termination of either Publisher will result in termination for the Subscriber as well. Note that if some elements are emitted just before source completion and before a last sampler can trigger, the last of these elements will be emitted along with the onComplete signal.

Both Publisher will run in unbounded mode because the backpressure would interfere with the sampling precision.

Params:
Type parameters:
  • <U> – the type of the sampler sequence
@reactor.discardThis operator discards elements that are not part of the sampling.
Returns:a Flux sampled to the last item observed each time the sampler Publisher signals
/** * Sample this {@link Flux} by emitting an item corresponding to that {@link Flux} * latest emitted value whenever a companion sampler {@link Publisher} signals a value. * <p> * Termination of either {@link Publisher} will result in termination for the {@link Subscriber} * as well. * Note that if some elements are emitted just before source completion and before a * last sampler can trigger, the last of these elements will be emitted along with the * onComplete signal. * <p> * Both {@link Publisher} will run in unbounded mode because the backpressure * would interfere with the sampling precision. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/sample.png" alt=""> * * @reactor.discard This operator discards elements that are not part of the sampling. * * @param sampler the sampler companion {@link Publisher} * * @param <U> the type of the sampler sequence * * @return a {@link Flux} sampled to the last item observed each time the sampler {@link Publisher} signals */
public final <U> Flux<T> sample(Publisher<U> sampler) { return onAssembly(new FluxSample<>(this, sampler)); }
Repeatedly take a value from this Flux then skip the values that follow within a given duration.

Params:
  • timespan – the duration during which to skip values after each sample
@reactor.discardThis operator discards elements that are not part of the sampling.
Returns:a Flux sampled to the first item of each duration-based window
/** * Repeatedly take a value from this {@link Flux} then skip the values that follow * within a given duration. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/samplefirsttimespan.png" alt=""> * * @reactor.discard This operator discards elements that are not part of the sampling. * * @param timespan the duration during which to skip values after each sample * * @return a {@link Flux} sampled to the first item of each duration-based window */
public final Flux<T> sampleFirst(Duration timespan) { return sampleFirst(t -> Mono.delay(timespan)); }
Repeatedly take a value from this Flux then skip the values that follow before the next signal from a companion sampler Publisher.

Params:
  • samplerFactory – supply a companion sampler Publisher which signals the end of the skip window
Type parameters:
  • <U> – the companion reified type
@reactor.discardThis operator discards elements that are not part of the sampling.
Returns:a Flux sampled to the first item observed in each window closed by the sampler signals
/** * Repeatedly take a value from this {@link Flux} then skip the values that follow * before the next signal from a companion sampler {@link Publisher}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/samplefirst.png" alt=""> * * @reactor.discard This operator discards elements that are not part of the sampling. * * @param samplerFactory supply a companion sampler {@link Publisher} which signals the end of the skip window * @param <U> the companion reified type * * @return a {@link Flux} sampled to the first item observed in each window closed by the sampler signals */
public final <U> Flux<T> sampleFirst(Function<? super T, ? extends Publisher<U>> samplerFactory) { return onAssembly(new FluxSampleFirst<>(this, samplerFactory)); }
Emit the latest value from this Flux only if there were no new values emitted during the window defined by a companion Publisher derived from that particular value.

Note that this means that the last value in the sequence is always emitted.

Params:
  • throttlerFactory – supply a companion sampler Publisher which signals the end of the window during which no new emission should occur. If it is the case, the original value triggering the window is emitted.
Type parameters:
  • <U> – the companion reified type
@reactor.discardThis operator discards elements that are not part of the sampling.
Returns:a Flux sampled to items not followed by any other item within a window defined by a companion Publisher
/** * Emit the latest value from this {@link Flux} only if there were no new values emitted * during the window defined by a companion {@link Publisher} derived from that particular * value. * <p> * Note that this means that the last value in the sequence is always emitted. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/sampletimeout.png" alt=""> * * @reactor.discard This operator discards elements that are not part of the sampling. * * @param throttlerFactory supply a companion sampler {@link Publisher} which signals * the end of the window during which no new emission should occur. If it is the case, * the original value triggering the window is emitted. * @param <U> the companion reified type * * @return a {@link Flux} sampled to items not followed by any other item within a window * defined by a companion {@link Publisher} */
public final <U> Flux<T> sampleTimeout(Function<? super T, ? extends Publisher<U>> throttlerFactory) { return sampleTimeout(throttlerFactory, Queues.XS_BUFFER_SIZE); }
Emit the latest value from this Flux only if there were no new values emitted during the window defined by a companion Publisher derived from that particular value.

The provided maxConcurrency will keep a bounded maximum of concurrent timeouts and drop any new items until at least one timeout terminates.

Note that this means that the last value in the sequence is always emitted.

Params:
  • throttlerFactory – supply a companion sampler Publisher which signals the end of the window during which no new emission should occur. If it is the case, the original value triggering the window is emitted.
  • maxConcurrency – the maximum number of concurrent timeouts
Type parameters:
  • <U> – the throttling type
@reactor.discardThis operator discards elements that are not part of the sampling.
Returns:a Flux sampled to items not followed by any other item within a window defined by a companion Publisher
/** * Emit the latest value from this {@link Flux} only if there were no new values emitted * during the window defined by a companion {@link Publisher} derived from that particular * value. * <p>The provided {@literal maxConcurrency} will keep a bounded maximum of concurrent timeouts and drop any new * items until at least one timeout terminates. * <p> * Note that this means that the last value in the sequence is always emitted. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/sampletimeoutm.png" alt=""> * * @reactor.discard This operator discards elements that are not part of the sampling. * * @param throttlerFactory supply a companion sampler {@link Publisher} which signals * the end of the window during which no new emission should occur. If it is the case, * the original value triggering the window is emitted. * @param maxConcurrency the maximum number of concurrent timeouts * @param <U> the throttling type * * @return a {@link Flux} sampled to items not followed by any other item within a window * defined by a companion {@link Publisher} */
public final <U> Flux<T> sampleTimeout(Function<? super T, ? extends Publisher<U>> throttlerFactory, int maxConcurrency) { return onAssembly(new FluxSampleTimeout<>(this, throttlerFactory, Queues.get(maxConcurrency))); }
Reduce this Flux values with an accumulator BiFunction and also emit the intermediate results of this function.

Unlike scan(Object, BiFunction), this operator doesn't take an initial value but treats the first Flux value as initial value.
The accumulation works as follows:


result[0] = source[0]
result[1] = accumulator(result[0], source[1])
result[2] = accumulator(result[1], source[2])
result[3] = accumulator(result[2], source[3])
...

Params:
Returns:an accumulating Flux
/** * Reduce this {@link Flux} values with an accumulator {@link BiFunction} and * also emit the intermediate results of this function. * <p> * Unlike {@link #scan(Object, BiFunction)}, this operator doesn't take an initial value * but treats the first {@link Flux} value as initial value. * <br> * The accumulation works as follows: * <pre><code> * result[0] = source[0] * result[1] = accumulator(result[0], source[1]) * result[2] = accumulator(result[1], source[2]) * result[3] = accumulator(result[2], source[3]) * ... * </code></pre> * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/accumulate.png" alt=""> * * @param accumulator the accumulating {@link BiFunction} * * @return an accumulating {@link Flux} */
public final Flux<T> scan(BiFunction<T, T, T> accumulator) { return onAssembly(new FluxScan<>(this, accumulator)); }
Reduce this Flux values with an accumulator BiFunction and also emit the intermediate results of this function.

The accumulation works as follows:


result[0] = initialValue;
result[1] = accumulator(result[0], source[0])
result[2] = accumulator(result[1], source[1])
result[3] = accumulator(result[2], source[2])
...

Params:
  • initial – the initial leftmost argument to pass to the reduce function
  • accumulator – the accumulating BiFunction
Type parameters:
  • <A> – the accumulated type
Returns:an accumulating Flux starting with initial state
/** * Reduce this {@link Flux} values with an accumulator {@link BiFunction} and * also emit the intermediate results of this function. * <p> * The accumulation works as follows: * <pre><code> * result[0] = initialValue; * result[1] = accumulator(result[0], source[0]) * result[2] = accumulator(result[1], source[1]) * result[3] = accumulator(result[2], source[2]) * ... * </code></pre> * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/scan.png" alt=""> * * @param initial the initial leftmost argument to pass to the reduce function * @param accumulator the accumulating {@link BiFunction} * @param <A> the accumulated type * * @return an accumulating {@link Flux} starting with initial state * */
public final <A> Flux<A> scan(A initial, BiFunction<A, ? super T, A> accumulator) { Objects.requireNonNull(initial, "seed"); return scanWith(() -> initial, accumulator); }
Reduce this Flux values with the help of an accumulator BiFunction and also emits the intermediate results. A seed value is lazily provided by a Supplier invoked for each Subscriber.

The accumulation works as follows:


result[0] = initialValue;
result[1] = accumulator(result[0], source[0])
result[2] = accumulator(result[1], source[1])
result[3] = accumulator(result[2], source[2])
...

Params:
  • initial – the supplier providing the seed, the leftmost parameter initially passed to the reduce function
  • accumulator – the accumulating BiFunction
Type parameters:
  • <A> – the accumulated type
Returns:an accumulating Flux starting with initial state
/** * Reduce this {@link Flux} values with the help of an accumulator {@link BiFunction} * and also emits the intermediate results. A seed value is lazily provided by a * {@link Supplier} invoked for each {@link Subscriber}. * <p> * The accumulation works as follows: * <pre><code> * result[0] = initialValue; * result[1] = accumulator(result[0], source[0]) * result[2] = accumulator(result[1], source[1]) * result[3] = accumulator(result[2], source[2]) * ... * </code></pre> * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/scan.png" alt=""> * * @param initial the supplier providing the seed, the leftmost parameter initially * passed to the reduce function * @param accumulator the accumulating {@link BiFunction} * @param <A> the accumulated type * * @return an accumulating {@link Flux} starting with initial state * */
public final <A> Flux<A> scanWith(Supplier<A> initial, BiFunction<A, ? super T, A> accumulator) { return onAssembly(new FluxScanSeed<>(this, initial, accumulator)); }
Returns a new Flux that multicasts (shares) the original Flux. As long as there is at least one Subscriber this Flux will be subscribed and emitting data. When all subscribers have cancelled it will cancel the source Flux.

This is an alias for publish().ConnectableFlux.refCount().

Returns:a Flux that upon first subscribe causes the source Flux to subscribe once, late subscribers might therefore miss items.
/** * Returns a new {@link Flux} that multicasts (shares) the original {@link Flux}. * As long as there is at least one {@link Subscriber} this {@link Flux} will be subscribed and * emitting data. * When all subscribers have cancelled it will cancel the source * {@link Flux}. * <p>This is an alias for {@link #publish()}.{@link ConnectableFlux#refCount()}. * * @return a {@link Flux} that upon first subscribe causes the source {@link Flux} * to subscribe once, late subscribers might therefore miss items. */
public final Flux<T> share() { return onAssembly(new FluxRefCount<>( new FluxPublish<>(this, Queues.SMALL_BUFFER_SIZE, Queues.small()), 1) ); }
Expect and emit a single item from this Flux source or signal NoSuchElementException for an empty source, or IndexOutOfBoundsException for a source with more than one element.

Returns:a Mono with the single item or an error signal
/** * Expect and emit a single item from this {@link Flux} source or signal * {@link java.util.NoSuchElementException} for an empty source, or * {@link IndexOutOfBoundsException} for a source with more than one element. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/single.png" alt=""> * * @return a {@link Mono} with the single item or an error signal */
public final Mono<T> single() { if (this instanceof Callable) { if (this instanceof Fuseable.ScalarCallable) { @SuppressWarnings("unchecked") Fuseable.ScalarCallable<T> scalarCallable = (Fuseable.ScalarCallable<T>) this; T v; try { v = scalarCallable.call(); } catch (Exception e) { return Mono.error(e); } if (v == null) { return Mono.error(new NoSuchElementException("Source was a (constant) empty")); } return Mono.just(v); } @SuppressWarnings("unchecked") Callable<T> thiz = (Callable<T>)this; return Mono.onAssembly(new MonoCallable<>(thiz)); } return Mono.onAssembly(new MonoSingle<>(this)); }
Expect and emit a single item from this Flux source and emit a default value for an empty source, but signal an IndexOutOfBoundsException for a source with more than one element.

Params:
  • defaultValue – a single fallback item if this Flux is empty
Returns:a Mono with the expected single item, the supplied default value or and error signal
/** * Expect and emit a single item from this {@link Flux} source and emit a default * value for an empty source, but signal an {@link IndexOutOfBoundsException} for a * source with more than one element. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/singleordefault.png" alt=""> * @param defaultValue a single fallback item if this {@link Flux} is empty * * @return a {@link Mono} with the expected single item, the supplied default value or * and error signal */
public final Mono<T> single(T defaultValue) { if (this instanceof Callable) { if (this instanceof Fuseable.ScalarCallable) { @SuppressWarnings("unchecked") Fuseable.ScalarCallable<T> scalarCallable = (Fuseable.ScalarCallable<T>) this; T v; try { v = scalarCallable.call(); } catch (Exception e) { return Mono.error(e); } if (v == null) { return Mono.just(defaultValue); } return Mono.just(v); } @SuppressWarnings("unchecked") Callable<T> thiz = (Callable<T>)this; return Mono.onAssembly(new MonoCallable<>(thiz)); } return Mono.onAssembly(new MonoSingle<>(this, defaultValue, false)); }
Expect and emit a single item from this Flux source, and accept an empty source but signal an IndexOutOfBoundsException for a source with more than one element.

Returns:a Mono with the expected single item, no item or an error
/** * Expect and emit a single item from this {@link Flux} source, and accept an empty * source but signal an {@link IndexOutOfBoundsException} for a source with more than * one element. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/singleorempty.png" alt=""> * * @return a {@link Mono} with the expected single item, no item or an error */
public final Mono<T> singleOrEmpty() { if (this instanceof Callable) { @SuppressWarnings("unchecked") Callable<T> thiz = (Callable<T>)this; return convertToMono(thiz); } return Mono.onAssembly(new MonoSingle<>(this, null, true)); }
Skip the specified number of elements from the beginning of this Flux then emit the remaining elements.

Params:
  • skipped – the number of elements to drop
@reactor.discardThis operator discards elements that are skipped.
Returns:a dropping Flux with the specified number of elements skipped at the beginning
/** * Skip the specified number of elements from the beginning of this {@link Flux} then * emit the remaining elements. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/skip.png" alt=""> * * @reactor.discard This operator discards elements that are skipped. * * @param skipped the number of elements to drop * * @return a dropping {@link Flux} with the specified number of elements skipped at * the beginning */
public final Flux<T> skip(long skipped) { if (skipped == 0L) { return this; } else { return onAssembly(new FluxSkip<>(this, skipped)); } }
Skip elements from this Flux emitted within the specified initial duration.

Params:
  • timespan – the initial time window during which to drop elements
@reactor.discardThis operator discards elements that are skipped.
Returns:a Flux dropping at the beginning until the end of the given duration
/** * Skip elements from this {@link Flux} emitted within the specified initial duration. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/skiptime.png" alt=""> * * @reactor.discard This operator discards elements that are skipped. * * @param timespan the initial time window during which to drop elements * * @return a {@link Flux} dropping at the beginning until the end of the given duration */
public final Flux<T> skip(Duration timespan) { return skip(timespan, Schedulers.parallel()); }
Skip elements from this Flux emitted within the specified initial duration, as measured on the provided Scheduler.

Params:
  • timespan – the initial time window during which to drop elements
  • timer – a time-capable Scheduler instance to measure the time window on
@reactor.discardThis operator discards elements that are skipped.
Returns:a Flux dropping at the beginning for the given duration
/** * Skip elements from this {@link Flux} emitted within the specified initial duration, * as measured on the provided {@link Scheduler}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/skiptime.png" alt=""> * * @reactor.discard This operator discards elements that are skipped. * * @param timespan the initial time window during which to drop elements * @param timer a time-capable {@link Scheduler} instance to measure the time window on * * @return a {@link Flux} dropping at the beginning for the given duration */
public final Flux<T> skip(Duration timespan, Scheduler timer) { if(!timespan.isZero()) { return skipUntilOther(Mono.delay(timespan, timer)); } else{ return this; } }
Skip a specified number of elements at the end of this Flux sequence.

Params:
  • n – the number of elements to drop before completion
@reactor.discardThis operator discards elements that are skipped.
Returns:a Flux dropping the specified number of elements at the end of the sequence
/** * Skip a specified number of elements at the end of this {@link Flux} sequence. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/skiplast.png" alt=""> * * @reactor.discard This operator discards elements that are skipped. * * @param n the number of elements to drop before completion * * @return a {@link Flux} dropping the specified number of elements at the end of the * sequence * */
public final Flux<T> skipLast(int n) { if (n == 0) { return this; } return onAssembly(new FluxSkipLast<>(this, n)); }
Skips values from this Flux until a Predicate returns true for the value. The resulting Flux will include and emit the matching value.

Params:
  • untilPredicate – the Predicate evaluated to stop skipping.
@reactor.discardThis operator discards elements that are skipped.
Returns:a Flux dropping until the Predicate matches
/** * Skips values from this {@link Flux} until a {@link Predicate} returns true for the * value. The resulting {@link Flux} will include and emit the matching value. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/skipuntil.png" alt=""> * * @reactor.discard This operator discards elements that are skipped. * * @param untilPredicate the {@link Predicate} evaluated to stop skipping. * * @return a {@link Flux} dropping until the {@link Predicate} matches */
public final Flux<T> skipUntil(Predicate<? super T> untilPredicate) { return onAssembly(new FluxSkipUntil<>(this, untilPredicate)); }
Skip values from this Flux until a specified Publisher signals an onNext or onComplete.

Params:
  • other – the companion Publisher to coordinate with to stop skipping
@reactor.discardThis operator discards elements that are skipped.
Returns:a Flux dropping until the other Publisher emits
/** * Skip values from this {@link Flux} until a specified {@link Publisher} signals * an onNext or onComplete. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/skipuntil.png" alt=""> * * @reactor.discard This operator discards elements that are skipped. * * @param other the companion {@link Publisher} to coordinate with to stop skipping * * @return a {@link Flux} dropping until the other {@link Publisher} emits * */
public final Flux<T> skipUntilOther(Publisher<?> other) { return onAssembly(new FluxSkipUntilOther<>(this, other)); }
Skips values from this Flux while a Predicate returns true for the value.

Params:
  • skipPredicate – the Predicate that causes skipping while evaluating to true.
@reactor.discardThis operator discards elements that are skipped.
Returns:a Flux dropping while the Predicate matches
/** * Skips values from this {@link Flux} while a {@link Predicate} returns true for the value. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/skipwhile.png" alt=""> * * @reactor.discard This operator discards elements that are skipped. * * @param skipPredicate the {@link Predicate} that causes skipping while evaluating to true. * * @return a {@link Flux} dropping while the {@link Predicate} matches */
public final Flux<T> skipWhile(Predicate<? super T> skipPredicate) { return onAssembly(new FluxSkipWhile<>(this, skipPredicate)); }
Sort elements from this Flux by collecting and sorting them in the background then emitting the sorted sequence once this sequence completes. Each item emitted by the Flux must implement Comparable with respect to all other items in the sequence.

Note that calling sort with long, non-terminating or infinite sources might cause OutOfMemoryError. Use sequence splitting like window to sort batches in that case.

Throws:
Returns:a sorted Flux
/** * Sort elements from this {@link Flux} by collecting and sorting them in the background * then emitting the sorted sequence once this sequence completes. * Each item emitted by the {@link Flux} must implement {@link Comparable} with * respect to all other items in the sequence. * * <p>Note that calling {@code sort} with long, non-terminating or infinite sources * might cause {@link OutOfMemoryError}. Use sequence splitting like {@link #window} to sort batches in that case. * * @throws ClassCastException if any item emitted by the {@link Flux} does not implement * {@link Comparable} with respect to all other items emitted by the {@link Flux} * @return a sorted {@link Flux} */
public final Flux<T> sort(){ return collectSortedList().flatMapIterable(identityFunction()); }
Sort elements from this Flux using a Comparator function, by collecting and sorting elements in the background then emitting the sorted sequence once this sequence completes.

Note that calling sort with long, non-terminating or infinite sources might cause OutOfMemoryError

Params:
  • sortFunction – a function that compares two items emitted by this Flux to indicate their sort order
Returns:a sorted Flux
/** * Sort elements from this {@link Flux} using a {@link Comparator} function, by * collecting and sorting elements in the background then emitting the sorted sequence * once this sequence completes. * * <p>Note that calling {@code sort} with long, non-terminating or infinite sources * might cause {@link OutOfMemoryError} * * @param sortFunction a function that compares two items emitted by this {@link Flux} * to indicate their sort order * @return a sorted {@link Flux} */
public final Flux<T> sort(Comparator<? super T> sortFunction) { return collectSortedList(sortFunction).flatMapIterable(identityFunction()); }
Prepend the given Iterable before this Flux sequence.

Params:
  • iterable – the sequence of values to start the resulting Flux with
Returns:a new Flux prefixed with elements from an Iterable
/** * Prepend the given {@link Iterable} before this {@link Flux} sequence. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/startwithi.png" alt=""> * * @param iterable the sequence of values to start the resulting {@link Flux} with * * @return a new {@link Flux} prefixed with elements from an {@link Iterable} */
public final Flux<T> startWith(Iterable<? extends T> iterable) { return startWith(fromIterable(iterable)); }
Prepend the given values before this Flux sequence.

Params:
  • values – the array of values to start the resulting Flux with
Returns:a new Flux prefixed with the given elements
/** * Prepend the given values before this {@link Flux} sequence. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/startwithv.png" alt=""> * * @param values the array of values to start the resulting {@link Flux} with * * @return a new {@link Flux} prefixed with the given elements */
@SafeVarargs public final Flux<T> startWith(T... values) { return startWith(just(values)); }
Prepend the given Publisher sequence to this Flux sequence.

Params:
  • publisher – the Publisher whose values to prepend
Returns:a new Flux prefixed with the given Publisher sequence
/** * Prepend the given {@link Publisher} sequence to this {@link Flux} sequence. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/startwith.png" alt=""> * * @param publisher the Publisher whose values to prepend * * @return a new {@link Flux} prefixed with the given {@link Publisher} sequence */
public final Flux<T> startWith(Publisher<? extends T> publisher) { if (this instanceof FluxConcatArray) { FluxConcatArray<T> fluxConcatArray = (FluxConcatArray<T>) this; return fluxConcatArray.concatAdditionalSourceFirst(publisher); } return concat(publisher, this); }
Subscribe to this Flux and request unbounded demand.

This version doesn't specify any consumption behavior for the events from the chain, especially no error handling, so other variants should usually be preferred.

Returns:a new Disposable that can be used to cancel the underlying Subscription
/** * Subscribe to this {@link Flux} and request unbounded demand. * <p> * This version doesn't specify any consumption behavior for the events from the * chain, especially no error handling, so other variants should usually be preferred. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/unbounded.png" alt=""> * <p> * * @return a new {@link Disposable} that can be used to cancel the underlying {@link Subscription} */
public final Disposable subscribe() { return subscribe(null, null, null); }
Subscribe a Consumer to this Flux that will consume all the elements in the sequence. It will request an unbounded demand (Long.MAX_VALUE).

For a passive version that observe and forward incoming data see doOnNext(Consumer).

For a version that gives you more control over backpressure and the request, see subscribe(Subscriber) with a BaseSubscriber.

Keep in mind that since the sequence can be asynchronous, this will immediately return control to the calling thread. This can give the impression the consumer is not invoked when executing in a main thread or a unit test for instance.

Params:
  • consumer – the consumer to invoke on each value (onNext signal)
Returns:a new Disposable that can be used to cancel the underlying Subscription
/** * Subscribe a {@link Consumer} to this {@link Flux} that will consume all the * elements in the sequence. It will request an unbounded demand ({@code Long.MAX_VALUE}). * <p> * For a passive version that observe and forward incoming data see {@link #doOnNext(java.util.function.Consumer)}. * <p> * For a version that gives you more control over backpressure and the request, see * {@link #subscribe(Subscriber)} with a {@link BaseSubscriber}. * <p> * Keep in mind that since the sequence can be asynchronous, this will immediately * return control to the calling thread. This can give the impression the consumer is * not invoked when executing in a main thread or a unit test for instance. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/subscribe.png" alt=""> * * @param consumer the consumer to invoke on each value (onNext signal) * * @return a new {@link Disposable} that can be used to cancel the underlying {@link Subscription} */
public final Disposable subscribe(Consumer<? super T> consumer) { Objects.requireNonNull(consumer, "consumer"); return subscribe(consumer, null, null); }
Subscribe to this Flux with a Consumer that will consume all the elements in the sequence, as well as a Consumer that will handle errors. The subscription will request an unbounded demand (Long.MAX_VALUE).

For a passive version that observe and forward incoming data see doOnNext(Consumer) and doOnError(Consumer).

For a version that gives you more control over backpressure and the request, see subscribe(Subscriber) with a BaseSubscriber.

Keep in mind that since the sequence can be asynchronous, this will immediately return control to the calling thread. This can give the impression the consumers are not invoked when executing in a main thread or a unit test for instance.

Params:
  • consumer – the consumer to invoke on each next signal
  • errorConsumer – the consumer to invoke on error signal
Returns:a new Disposable that can be used to cancel the underlying Subscription
/** * Subscribe to this {@link Flux} with a {@link Consumer} that will consume all the * elements in the sequence, as well as a {@link Consumer} that will handle errors. * The subscription will request an unbounded demand ({@code Long.MAX_VALUE}). * <p> * For a passive version that observe and forward incoming data see * {@link #doOnNext(java.util.function.Consumer)} and {@link #doOnError(java.util.function.Consumer)}. * <p>For a version that gives you more control over backpressure and the request, see * {@link #subscribe(Subscriber)} with a {@link BaseSubscriber}. * <p> * Keep in mind that since the sequence can be asynchronous, this will immediately * return control to the calling thread. This can give the impression the consumers are * not invoked when executing in a main thread or a unit test for instance. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/subscribeerror.png" alt=""> * * @param consumer the consumer to invoke on each next signal * @param errorConsumer the consumer to invoke on error signal * * @return a new {@link Disposable} that can be used to cancel the underlying {@link Subscription} */
public final Disposable subscribe(@Nullable Consumer<? super T> consumer, Consumer<? super Throwable> errorConsumer) { Objects.requireNonNull(errorConsumer, "errorConsumer"); return subscribe(consumer, errorConsumer, null); }
Subscribe Consumer to this Flux that will respectively consume all the elements in the sequence, handle errors and react to completion. The subscription will request unbounded demand (Long.MAX_VALUE).

For a passive version that observe and forward incoming data see doOnNext(Consumer), doOnError(Consumer) and doOnComplete(Runnable).

For a version that gives you more control over backpressure and the request, see subscribe(Subscriber) with a BaseSubscriber.

Keep in mind that since the sequence can be asynchronous, this will immediately return control to the calling thread. This can give the impression the consumer is not invoked when executing in a main thread or a unit test for instance.

Params:
  • consumer – the consumer to invoke on each value
  • errorConsumer – the consumer to invoke on error signal
  • completeConsumer – the consumer to invoke on complete signal
Returns:a new Disposable that can be used to cancel the underlying Subscription
/** * Subscribe {@link Consumer} to this {@link Flux} that will respectively consume all the * elements in the sequence, handle errors and react to completion. The subscription * will request unbounded demand ({@code Long.MAX_VALUE}). * <p> * For a passive version that observe and forward incoming data see {@link #doOnNext(java.util.function.Consumer)}, * {@link #doOnError(java.util.function.Consumer)} and {@link #doOnComplete(Runnable)}. * <p>For a version that gives you more control over backpressure and the request, see * {@link #subscribe(Subscriber)} with a {@link BaseSubscriber}. * <p> * Keep in mind that since the sequence can be asynchronous, this will immediately * return control to the calling thread. This can give the impression the consumer is * not invoked when executing in a main thread or a unit test for instance. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/subscribecomplete.png" alt=""> * * @param consumer the consumer to invoke on each value * @param errorConsumer the consumer to invoke on error signal * @param completeConsumer the consumer to invoke on complete signal * * @return a new {@link Disposable} that can be used to cancel the underlying {@link Subscription} */
public final Disposable subscribe( @Nullable Consumer<? super T> consumer, @Nullable Consumer<? super Throwable> errorConsumer, @Nullable Runnable completeConsumer) { return subscribe(consumer, errorConsumer, completeConsumer, null); }
Subscribe Consumer to this Flux that will respectively consume all the elements in the sequence, handle errors, react to completion, and request upon subscription. It will let the provided subscriptionConsumer request the adequate amount of data, or request unbounded demand Long.MAX_VALUE if no such consumer is provided.

For a passive version that observe and forward incoming data see doOnNext(Consumer), doOnError(Consumer), doOnComplete(Runnable) and doOnSubscribe(Consumer).

For a version that gives you more control over backpressure and the request, see subscribe(Subscriber) with a BaseSubscriber.

Keep in mind that since the sequence can be asynchronous, this will immediately return control to the calling thread. This can give the impression the consumer is not invoked when executing in a main thread or a unit test for instance.

Params:
  • consumer – the consumer to invoke on each value
  • errorConsumer – the consumer to invoke on error signal
  • completeConsumer – the consumer to invoke on complete signal
  • subscriptionConsumer – the consumer to invoke on subscribe signal, to be used for the initial request, or null for max request
Returns:a new Disposable that can be used to cancel the underlying Subscription
/** * Subscribe {@link Consumer} to this {@link Flux} that will respectively consume all the * elements in the sequence, handle errors, react to completion, and request upon subscription. * It will let the provided {@link Subscription subscriptionConsumer} * request the adequate amount of data, or request unbounded demand * {@code Long.MAX_VALUE} if no such consumer is provided. * <p> * For a passive version that observe and forward incoming data see {@link #doOnNext(java.util.function.Consumer)}, * {@link #doOnError(java.util.function.Consumer)}, {@link #doOnComplete(Runnable)} * and {@link #doOnSubscribe(Consumer)}. * <p>For a version that gives you more control over backpressure and the request, see * {@link #subscribe(Subscriber)} with a {@link BaseSubscriber}. * <p> * Keep in mind that since the sequence can be asynchronous, this will immediately * return control to the calling thread. This can give the impression the consumer is * not invoked when executing in a main thread or a unit test for instance. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/subscribecomplete.png" alt=""> * * @param consumer the consumer to invoke on each value * @param errorConsumer the consumer to invoke on error signal * @param completeConsumer the consumer to invoke on complete signal * @param subscriptionConsumer the consumer to invoke on subscribe signal, to be used * for the initial {@link Subscription#request(long) request}, or null for max request * * @return a new {@link Disposable} that can be used to cancel the underlying {@link Subscription} */
public final Disposable subscribe( @Nullable Consumer<? super T> consumer, @Nullable Consumer<? super Throwable> errorConsumer, @Nullable Runnable completeConsumer, @Nullable Consumer<? super Subscription> subscriptionConsumer) { return subscribeWith(new LambdaSubscriber<>(consumer, errorConsumer, completeConsumer, subscriptionConsumer)); } @Override public final void subscribe(Subscriber<? super T> actual) { onLastAssembly(this).subscribe(Operators.toCoreSubscriber(actual)); }
An internal Publisher.subscribe(Subscriber) that will bypass Hooks.onLastOperator(Function<? super Publisher<Object>,? extends Publisher<Object>>) pointcut.

In addition to behave as expected by Publisher.subscribe(Subscriber) in a controlled manner, it supports direct subscribe-time Context passing.

Params:
  • actual – the Subscriber interested into the published sequence
See Also:
/** * An internal {@link Publisher#subscribe(Subscriber)} that will bypass * {@link Hooks#onLastOperator(Function)} pointcut. * <p> * In addition to behave as expected by {@link Publisher#subscribe(Subscriber)} * in a controlled manner, it supports direct subscribe-time {@link Context} passing. * * @param actual the {@link Subscriber} interested into the published sequence * @see Flux#subscribe(Subscriber) */
public abstract void subscribe(CoreSubscriber<? super T> actual);
Enrich a potentially empty downstream Context by adding all values from the given Context, producing a new Context that is propagated upstream.

The Context propagation happens once per subscription (not on each onNext): it is done during the subscribe(Subscriber) phase, which runs from the last operator of a chain towards the first.

So this operator enriches a Context coming from under it in the chain (downstream, by default an empty one) and makes the new enriched Context visible to operators above it in the chain.

Params:
  • mergeContext – the Context to merge with a previous Context state, returning a new one.
See Also:
Returns:a contextualized Flux
/** * Enrich a potentially empty downstream {@link Context} by adding all values * from the given {@link Context}, producing a new {@link Context} that is propagated * upstream. * <p> * The {@link Context} propagation happens once per subscription (not on each onNext): * it is done during the {@code subscribe(Subscriber)} phase, which runs from * the last operator of a chain towards the first. * <p> * So this operator enriches a {@link Context} coming from under it in the chain * (downstream, by default an empty one) and makes the new enriched {@link Context} * visible to operators above it in the chain. * * @param mergeContext the {@link Context} to merge with a previous {@link Context} * state, returning a new one. * * @return a contextualized {@link Flux} * @see Context */
public final Flux<T> subscriberContext(Context mergeContext) { return subscriberContext(c -> c.putAll(mergeContext)); }
Enrich a potentially empty downstream Context by applying a Function to it, producing a new Context that is propagated upstream.

The Context propagation happens once per subscription (not on each onNext): it is done during the subscribe(Subscriber) phase, which runs from the last operator of a chain towards the first.

So this operator enriches a Context coming from under it in the chain (downstream, by default an empty one) and makes the new enriched Context visible to operators above it in the chain.

Params:
  • doOnContext – the function taking a previous Context state and returning a new one.
See Also:
Returns:a contextualized Flux
/** * Enrich a potentially empty downstream {@link Context} by applying a {@link Function} * to it, producing a new {@link Context} that is propagated upstream. * <p> * The {@link Context} propagation happens once per subscription (not on each onNext): * it is done during the {@code subscribe(Subscriber)} phase, which runs from * the last operator of a chain towards the first. * <p> * So this operator enriches a {@link Context} coming from under it in the chain * (downstream, by default an empty one) and makes the new enriched {@link Context} * visible to operators above it in the chain. * * @param doOnContext the function taking a previous {@link Context} state * and returning a new one. * * @return a contextualized {@link Flux} * @see Context */
public final Flux<T> subscriberContext(Function<Context, Context> doOnContext) { return new FluxContextStart<>(this, doOnContext); }
Run subscribe, onSubscribe and request on a specified Scheduler's Worker. As such, placing this operator anywhere in the chain will also impact the execution context of onNext/onError/onComplete signals from the beginning of the chain up to the next occurrence of a publishOn.

Note that if you are using an eager or blocking create(Consumer<? super FluxSink<Object>>, OverflowStrategy) as the source, it can lead to deadlocks due to requests piling up behind the emitter. In such case, you should call subscribeOn(scheduler, false) instead.

Typically used for slow publisher e.g., blocking IO, fast consumer(s) scenarios.

flux.subscribeOn(Schedulers.single()).subscribe()  

Note that Worker.schedule(Runnable) raising RejectedExecutionException on late Subscription.request(long) will be propagated to the request caller.

Params:
See Also:
Returns:a Flux requesting asynchronously
/** * Run subscribe, onSubscribe and request on a specified {@link Scheduler}'s {@link Worker}. * As such, placing this operator anywhere in the chain will also impact the execution * context of onNext/onError/onComplete signals from the beginning of the chain up to * the next occurrence of a {@link #publishOn(Scheduler) publishOn}. * <p> * Note that if you are using an eager or blocking * {@link #create(Consumer, FluxSink.OverflowStrategy)} * as the source, it can lead to deadlocks due to requests piling up behind the emitter. * In such case, you should call {@link #subscribeOn(Scheduler, boolean) subscribeOn(scheduler, false)} * instead. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/subscribeon.png" alt=""> * <p> * Typically used for slow publisher e.g., blocking IO, fast consumer(s) scenarios. * * <blockquote><pre> * {@code flux.subscribeOn(Schedulers.single()).subscribe() } * </pre></blockquote> * * <p> * Note that {@link Worker#schedule(Runnable)} raising * {@link java.util.concurrent.RejectedExecutionException} on late * {@link Subscription#request(long)} will be propagated to the request caller. * * @param scheduler a {@link Scheduler} providing the {@link Worker} where to subscribe * * @return a {@link Flux} requesting asynchronously * @see #publishOn(Scheduler) * @see #subscribeOn(Scheduler, boolean) */
public final Flux<T> subscribeOn(Scheduler scheduler) { return subscribeOn(scheduler, true); }
Run subscribe and onSubscribe on a specified Scheduler's Worker. Request will be run on that worker too depending on the requestOnSeparateThread parameter (which defaults to true in the subscribeOn(Scheduler) version). As such, placing this operator anywhere in the chain will also impact the execution context of onNext/onError/onComplete signals from the beginning of the chain up to the next occurrence of a publishOn.

Note that if you are using an eager or blocking create(Consumer<? super FluxSink<Object>>, OverflowStrategy) as the source, it can lead to deadlocks due to requests piling up behind the emitter. Thus this operator has a requestOnSeparateThread parameter, which should be set to false in this case.

Typically used for slow publisher e.g., blocking IO, fast consumer(s) scenarios.

flux.subscribeOn(Schedulers.single()).subscribe()  

Note that Worker.schedule(Runnable) raising RejectedExecutionException on late Subscription.request(long) will be propagated to the request caller.

Params:
  • scheduler – a Scheduler providing the Worker where to subscribe
  • requestOnSeparateThread – whether or not to also perform requests on the worker. true to behave like subscribeOn(Scheduler)
See Also:
Returns:a Flux requesting asynchronously
/** * Run subscribe and onSubscribe on a specified {@link Scheduler}'s {@link Worker}. * Request will be run on that worker too depending on the {@code requestOnSeparateThread} * parameter (which defaults to true in the {@link #subscribeOn(Scheduler)} version). * As such, placing this operator anywhere in the chain will also impact the execution * context of onNext/onError/onComplete signals from the beginning of the chain up to * the next occurrence of a {@link #publishOn(Scheduler) publishOn}. * <p> * Note that if you are using an eager or blocking * {@link Flux#create(Consumer, FluxSink.OverflowStrategy)} * as the source, it can lead to deadlocks due to requests piling up behind the emitter. * Thus this operator has a {@code requestOnSeparateThread} parameter, which should be * set to {@code false} in this case. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/subscribeon.png" alt=""> * <p> * Typically used for slow publisher e.g., blocking IO, fast consumer(s) scenarios. * * <blockquote><pre> * {@code flux.subscribeOn(Schedulers.single()).subscribe() } * </pre></blockquote> * * <p> * Note that {@link Worker#schedule(Runnable)} raising * {@link java.util.concurrent.RejectedExecutionException} on late * {@link Subscription#request(long)} will be propagated to the request caller. * * @param scheduler a {@link Scheduler} providing the {@link Worker} where to subscribe * @param requestOnSeparateThread whether or not to also perform requests on the worker. * {@code true} to behave like {@link #subscribeOn(Scheduler)} * * @return a {@link Flux} requesting asynchronously * @see #publishOn(Scheduler) * @see #subscribeOn(Scheduler) */
public final Flux<T> subscribeOn(Scheduler scheduler, boolean requestOnSeparateThread) { if (this instanceof Callable) { if (this instanceof Fuseable.ScalarCallable) { try { @SuppressWarnings("unchecked") T value = ((Fuseable.ScalarCallable<T>) this).call(); return onAssembly(new FluxSubscribeOnValue<>(value, scheduler)); } catch (Exception e) { //leave FluxSubscribeOnCallable defer error } } @SuppressWarnings("unchecked") Callable<T> c = (Callable<T>)this; return onAssembly(new FluxSubscribeOnCallable<>(c, scheduler)); } return onAssembly(new FluxSubscribeOn<>(this, scheduler, requestOnSeparateThread)); }
Subscribe the given Subscriber to this Flux and return said Subscriber (eg. a FluxProcessor).
flux.subscribeWith(WorkQueueProcessor.create()).subscribe()  
If you need more control over backpressure and the request, use a BaseSubscriber.
Params:
  • subscriber – the Subscriber to subscribe with and return
Type parameters:
  • <E> – the reified type from the input/output subscriber
Returns:the passed Subscriber
/** * Subscribe the given {@link Subscriber} to this {@link Flux} and return said * {@link Subscriber} (eg. a {@link FluxProcessor}). * * <blockquote><pre> * {@code flux.subscribeWith(WorkQueueProcessor.create()).subscribe() } * </pre></blockquote> * * If you need more control over backpressure and the request, use a {@link BaseSubscriber}. * * @param subscriber the {@link Subscriber} to subscribe with and return * @param <E> the reified type from the input/output subscriber * * @return the passed {@link Subscriber} */
public final <E extends Subscriber<? super T>> E subscribeWith(E subscriber) { subscribe(subscriber); return subscriber; }
Switch to an alternative Publisher if this sequence is completed without any data.

Params:
  • alternate – the alternative Publisher if this sequence is empty
Returns:a new Flux that falls back on a Publisher if source is empty
/** * Switch to an alternative {@link Publisher} if this sequence is completed without any data. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/switchifempty.png" alt=""> * <p> * @param alternate the alternative {@link Publisher} if this sequence is empty * * @return a new {@link Flux} that falls back on a {@link Publisher} if source is empty */
public final Flux<T> switchIfEmpty(Publisher<? extends T> alternate) { return onAssembly(new FluxSwitchIfEmpty<>(this, alternate)); }
Switch to a new Publisher generated via a Function whenever this Flux produces an item. As such, the elements from each generated Publisher are emitted in the resulting Flux.

Params:
Type parameters:
  • <V> – the type of the return value of the transformation function
Returns:a new Flux that emits values from an alternative Publisher for each source onNext
/** * Switch to a new {@link Publisher} generated via a {@link Function} whenever * this {@link Flux} produces an item. As such, the elements from each generated * Publisher are emitted in the resulting {@link Flux}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/switchmap.png" alt=""> * * @param fn the {@link Function} to generate a {@link Publisher} for each source value * @param <V> the type of the return value of the transformation function * * @return a new {@link Flux} that emits values from an alternative {@link Publisher} * for each source onNext * */
public final <V> Flux<V> switchMap(Function<? super T, Publisher<? extends V>> fn) { return switchMap(fn, Queues.XS_BUFFER_SIZE); }
Switch to a new Publisher generated via a Function whenever this Flux produces an item. As such, the elements from each generated Publisher are emitted in the resulting Flux.

Params:
  • fn – the Function to generate a Publisher for each source value
  • prefetch – the produced demand for inner sources
Type parameters:
  • <V> – the type of the return value of the transformation function
Returns:a new Flux that emits values from an alternative Publisher for each source onNext
/** * Switch to a new {@link Publisher} generated via a {@link Function} whenever * this {@link Flux} produces an item. As such, the elements from each generated * Publisher are emitted in the resulting {@link Flux}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/switchmap.png" alt=""> * * @param fn the {@link Function} to generate a {@link Publisher} for each source value * @param prefetch the produced demand for inner sources * * @param <V> the type of the return value of the transformation function * * @return a new {@link Flux} that emits values from an alternative {@link Publisher} * for each source onNext */
public final <V> Flux<V> switchMap(Function<? super T, Publisher<? extends V>> fn, int prefetch) { return onAssembly(new FluxSwitchMap<>(this, fn, Queues.unbounded(prefetch), prefetch)); }
Tag this flux with a key/value pair. These can be retrieved as a Set of all tags throughout the publisher chain by using Scannable.tags() (as traversed by Scannable.parents()).
Params:
  • key – a tag key
  • value – a tag value
Returns:the same sequence, but bearing tags
/** * Tag this flux with a key/value pair. These can be retrieved as a {@link Set} of * all tags throughout the publisher chain by using {@link Scannable#tags()} (as * traversed * by {@link Scannable#parents()}). * * @param key a tag key * @param value a tag value * @return the same sequence, but bearing tags */
public final Flux<T> tag(String key, String value) { return FluxName.createOrAppend(this, key, value); }
Take only the first N values from this Flux, if available.

If N is zero, the resulting Flux completes as soon as this Flux signals its first value (which is not not relayed, though).

Note that this operator doesn't manipulate the backpressure requested amount. Rather, it merely lets requests from downstream propagate as is and cancels once N elements have been emitted. As a result, the source could produce a lot of extraneous elements in the meantime. If that behavior is undesirable and you do not own the request from downstream (e.g. prefetching operators), consider using limitRequest(long) instead.

Params:
  • n – the number of items to emit from this Flux
See Also:
Returns:a Flux limited to size N
/** * Take only the first N values from this {@link Flux}, if available. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/take.png" alt=""> * <p> * If N is zero, the resulting {@link Flux} completes as soon as this {@link Flux} * signals its first value (which is not not relayed, though). * <p> * Note that this operator doesn't manipulate the backpressure requested amount. * Rather, it merely lets requests from downstream propagate as is and cancels once * N elements have been emitted. As a result, the source could produce a lot of * extraneous elements in the meantime. If that behavior is undesirable and you do * not own the request from downstream (e.g. prefetching operators), consider * using {@link #limitRequest(long)} instead. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/take0.png" alt=""> * @param n the number of items to emit from this {@link Flux} * * @return a {@link Flux} limited to size N * @see #limitRequest(long) */
public final Flux<T> take(long n) { if (this instanceof Fuseable) { return onAssembly(new FluxTakeFuseable<>(this, n)); } return onAssembly(new FluxTake<>(this, n)); }
Relay values from this Flux until the specified Duration elapses.

If the duration is zero, the resulting Flux completes as soon as this Flux signals its first value (which is not not relayed, though).

Params:
  • timespan – the Duration of the time window during which to emit elements from this Flux
Returns:a Flux limited to elements emitted within a specific duration
/** * Relay values from this {@link Flux} until the specified {@link Duration} elapses. * <p> * If the duration is zero, the resulting {@link Flux} completes as soon as this {@link Flux} * signals its first value (which is not not relayed, though). * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/taketime.png" alt=""> * * @param timespan the {@link Duration} of the time window during which to emit elements * from this {@link Flux} * * @return a {@link Flux} limited to elements emitted within a specific duration */
public final Flux<T> take(Duration timespan) { return take(timespan, Schedulers.parallel()); }
Relay values from this Flux until the specified Duration elapses, as measured on the specified Scheduler.

If the duration is zero, the resulting Flux completes as soon as this Flux signals its first value (which is not not relayed, though).

Params:
  • timespan – the Duration of the time window during which to emit elements from this Flux
  • timer – a time-capable Scheduler instance to run on
Returns:a Flux limited to elements emitted within a specific duration
/** * Relay values from this {@link Flux} until the specified {@link Duration} elapses, * as measured on the specified {@link Scheduler}. * <p> * If the duration is zero, the resulting {@link Flux} completes as soon as this {@link Flux} * signals its first value (which is not not relayed, though). * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/taketime.png" alt=""> * * @param timespan the {@link Duration} of the time window during which to emit elements * from this {@link Flux} * @param timer a time-capable {@link Scheduler} instance to run on * * @return a {@link Flux} limited to elements emitted within a specific duration */
public final Flux<T> take(Duration timespan, Scheduler timer) { if (!timespan.isZero()) { return takeUntilOther(Mono.delay(timespan, timer)); } else { return take(0); } }
Emit the last N values this Flux emitted before its completion.

Params:
  • n – the number of items from this Flux to retain and emit on onComplete
Returns:a terminating Flux sub-sequence
/** * Emit the last N values this {@link Flux} emitted before its completion. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/takelast.png" alt=""> * * @param n the number of items from this {@link Flux} to retain and emit on onComplete * * @return a terminating {@link Flux} sub-sequence * */
public final Flux<T> takeLast(int n) { if(n == 1){ return onAssembly(new FluxTakeLastOne<>(this)); } return onAssembly(new FluxTakeLast<>(this, n)); }
Relay values from this Flux until the given Predicate matches. This includes the matching data (unlike takeWhile).

Params:
  • predicate – the Predicate that stops the taking of values from this Flux when returning true.
Returns:a new Flux limited by the predicate
/** * Relay values from this {@link Flux} until the given {@link Predicate} matches. * This includes the matching data (unlike {@link #takeWhile}). * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/takeuntilp.png" alt=""> * * @param predicate the {@link Predicate} that stops the taking of values from this {@link Flux} * when returning {@literal true}. * * @return a new {@link Flux} limited by the predicate * */
public final Flux<T> takeUntil(Predicate<? super T> predicate) { return onAssembly(new FluxTakeUntil<>(this, predicate)); }
Relay values from this Flux until the given Publisher emits.

Params:
  • other – the companion Publisher that signals when to stop taking values from this Flux
Returns:a new Flux limited by a companion Publisher
/** * Relay values from this {@link Flux} until the given {@link Publisher} emits. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/takeuntil.png" alt=""> * * @param other the companion {@link Publisher} that signals when to stop taking values from this {@link Flux} * * @return a new {@link Flux} limited by a companion {@link Publisher} * */
public final Flux<T> takeUntilOther(Publisher<?> other) { return onAssembly(new FluxTakeUntilOther<>(this, other)); }
Relay values from this Flux while a predicate returns TRUE for the values (checked before each value is delivered). This only includes the matching data (unlike takeUntil).

Params:
  • continuePredicate – the Predicate invoked each onNext returning TRUE to relay a value or FALSE to terminate
Returns:a new Flux taking values from the source while the predicate matches
/** * Relay values from this {@link Flux} while a predicate returns {@literal TRUE} * for the values (checked before each value is delivered). * This only includes the matching data (unlike {@link #takeUntil}). * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/takewhile.png" alt=""> * * @param continuePredicate the {@link Predicate} invoked each onNext returning {@literal TRUE} * to relay a value or {@literal FALSE} to terminate * * @return a new {@link Flux} taking values from the source while the predicate matches */
public final Flux<T> takeWhile(Predicate<? super T> continuePredicate) { return onAssembly(new FluxTakeWhile<>(this, continuePredicate)); }
Return a Mono<Void> that completes when this Flux completes. This will actively ignore the sequence and only replay completion or error signals.

@reactor.discardThis operator discards elements from the source.
Returns:a new Mono representing the termination of this Flux
/** * Return a {@code Mono<Void>} that completes when this {@link Flux} completes. * This will actively ignore the sequence and only replay completion or error signals. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/ignorethen.png" alt=""> * <p> * * @reactor.discard This operator discards elements from the source. * * @return a new {@link Mono} representing the termination of this {@link Flux} */
public final Mono<Void> then() { @SuppressWarnings("unchecked") Mono<Void> then = (Mono<Void>) new MonoIgnoreElements<>(this); return Mono.onAssembly(then); }
Let this Flux complete then play signals from a provided Mono.

In other words ignore element from this Flux and transform its completion signal into the emission and completion signal of a provided Mono<V>. Error signal is replayed in the resulting Mono<V>.

Params:
  • other – a Mono to emit from after termination
Type parameters:
  • <V> – the element type of the supplied Mono
@reactor.discardThis operator discards elements from the source.
Returns:a new Flux that wait for source completion then emits from the supplied Mono
/** * Let this {@link Flux} complete then play signals from a provided {@link Mono}. * <p> * In other words ignore element from this {@link Flux} and transform its completion signal into the * emission and completion signal of a provided {@code Mono<V>}. Error signal is * replayed in the resulting {@code Mono<V>}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/ignorethen1.png" alt=""> * * @reactor.discard This operator discards elements from the source. * * @param other a {@link Mono} to emit from after termination * @param <V> the element type of the supplied Mono * * @return a new {@link Flux} that wait for source completion then emits from the supplied {@link Mono} */
public final <V> Mono<V> then(Mono<V> other) { return Mono.onAssembly(new MonoIgnoreThen<>(new Publisher[] { this }, other)); }
Return a Mono<Void> that waits for this Flux to complete then for a supplied Publisher<Void> to also complete. The second completion signal is replayed, or any error signal that occurs instead.

Params:
  • other – a Publisher to wait for after this Flux's termination
@reactor.discardThis operator discards elements from the source.
Returns:a new Mono completing when both publishers have completed in sequence
/** * Return a {@code Mono<Void>} that waits for this {@link Flux} to complete then * for a supplied {@link Publisher Publisher&lt;Void&gt;} to also complete. The * second completion signal is replayed, or any error signal that occurs instead. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.2.0.M2/src/docs/marble/thenempty.png" * alt=""> * * @reactor.discard This operator discards elements from the source. * * @param other a {@link Publisher} to wait for after this Flux's termination * @return a new {@link Mono} completing when both publishers have completed in * sequence */
public final Mono<Void> thenEmpty(Publisher<Void> other) { return then(Mono.fromDirect(other)); }
Let this Flux complete then play another Publisher.

In other words ignore element from this flux and transform the completion signal into a Publisher<V> that will emit elements from the provided Publisher.

Params:
  • other – a Publisher to emit from after termination
Type parameters:
  • <V> – the element type of the supplied Publisher
@reactor.discardThis operator discards elements from the source.
Returns:a new Flux that emits from the supplied Publisher after this Flux completes.
/** * Let this {@link Flux} complete then play another {@link Publisher}. * <p> * In other words ignore element from this flux and transform the completion signal into a * {@code Publisher<V>} that will emit elements from the provided {@link Publisher}. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.2.0.M2/src/docs/marble/thenmany.png" alt=""> * * @reactor.discard This operator discards elements from the source. * * @param other a {@link Publisher} to emit from after termination * @param <V> the element type of the supplied Publisher * * @return a new {@link Flux} that emits from the supplied {@link Publisher} after * this Flux completes. */
public final <V> Flux<V> thenMany(Publisher<V> other) { if (this instanceof FluxConcatArray) { @SuppressWarnings({ "unchecked" }) FluxConcatArray<T> fluxConcatArray = (FluxConcatArray<T>) this; return fluxConcatArray.concatAdditionalIgnoredLast(other); } @SuppressWarnings("unchecked") Flux<V> concat = (Flux<V>)concat(ignoreElements(), other); return concat; }
Propagate a TimeoutException as soon as no item is emitted within the given Duration from the previous emission (or the subscription for the first item).

Params:
  • timeout – the timeout between two signals from this Flux
Returns:a Flux that can time out on a per-item basis
/** * Propagate a {@link TimeoutException} as soon as no item is emitted within the * given {@link Duration} from the previous emission (or the subscription for the first item). * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/timeouttime.png" alt=""> * * @param timeout the timeout between two signals from this {@link Flux} * * @return a {@link Flux} that can time out on a per-item basis */
public final Flux<T> timeout(Duration timeout) { return timeout(timeout, null, Schedulers.parallel()); }
Switch to a fallback Flux as soon as no item is emitted within the given Duration from the previous emission (or the subscription for the first item).

If the given Publisher is null, signal a TimeoutException instead.

Params:
  • timeout – the timeout between two signals from this Flux
  • fallback – the fallback Publisher to subscribe when a timeout occurs
Returns:a Flux that will fallback to a different Publisher in case of a per-item timeout
/** * Switch to a fallback {@link Flux} as soon as no item is emitted within the * given {@link Duration} from the previous emission (or the subscription for the first item). * <p> * If the given {@link Publisher} is null, signal a {@link TimeoutException} instead. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/timeouttimefallback.png" alt=""> * * @param timeout the timeout between two signals from this {@link Flux} * @param fallback the fallback {@link Publisher} to subscribe when a timeout occurs * * @return a {@link Flux} that will fallback to a different {@link Publisher} in case of a per-item timeout */
public final Flux<T> timeout(Duration timeout, @Nullable Publisher<? extends T> fallback) { return timeout(timeout, fallback, Schedulers.parallel()); }
Propagate a TimeoutException as soon as no item is emitted within the given Duration from the previous emission (or the subscription for the first item), as measured by the specified Scheduler.

Params:
  • timeout – the timeout Duration between two signals from this Flux
  • timer – a time-capable Scheduler instance to run on
Returns:a Flux that can time out on a per-item basis
/** * Propagate a {@link TimeoutException} as soon as no item is emitted within the * given {@link Duration} from the previous emission (or the subscription for the first * item), as measured by the specified {@link Scheduler}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/timeouttime.png" alt=""> * * @param timeout the timeout {@link Duration} between two signals from this {@link Flux} * @param timer a time-capable {@link Scheduler} instance to run on * * @return a {@link Flux} that can time out on a per-item basis */
public final Flux<T> timeout(Duration timeout, Scheduler timer) { return timeout(timeout, null, timer); }
Switch to a fallback Flux as soon as no item is emitted within the given Duration from the previous emission (or the subscription for the first item), as measured on the specified Scheduler.

If the given Publisher is null, signal a TimeoutException instead.

Params:
  • timeout – the timeout Duration between two signals from this Flux
  • fallback – the fallback Publisher to subscribe when a timeout occurs
  • timer – a time-capable Scheduler instance to run on
Returns:a Flux that will fallback to a different Publisher in case of a per-item timeout
/** * Switch to a fallback {@link Flux} as soon as no item is emitted within the * given {@link Duration} from the previous emission (or the subscription for the * first item), as measured on the specified {@link Scheduler}. * <p> * If the given {@link Publisher} is null, signal a {@link TimeoutException} instead. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/timeouttimefallback.png" alt=""> * * @param timeout the timeout {@link Duration} between two signals from this {@link Flux} * @param fallback the fallback {@link Publisher} to subscribe when a timeout occurs * @param timer a time-capable {@link Scheduler} instance to run on * * @return a {@link Flux} that will fallback to a different {@link Publisher} in case of a per-item timeout */
public final Flux<T> timeout(Duration timeout, @Nullable Publisher<? extends T> fallback, Scheduler timer) { final Mono<Long> _timer = Mono.delay(timeout, timer).onErrorReturn(0L); final Function<T, Publisher<Long>> rest = o -> _timer; if(fallback == null) { return timeout(_timer, rest, timeout.toMillis() + "ms"); } return timeout(_timer, rest, fallback); }
Signal a TimeoutException in case the first item from this Flux has not been emitted before the given Publisher emits.

Params:
  • firstTimeout – the companion Publisher that will trigger a timeout if emitting before the first signal from this Flux
Type parameters:
  • <U> – the type of the timeout Publisher
Returns:a Flux that can time out if the first item does not come before a signal from a companion Publisher
/** * Signal a {@link TimeoutException} in case the first item from this {@link Flux} has * not been emitted before the given {@link Publisher} emits. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/timeoutfirst.png" alt=""> * * @param firstTimeout the companion {@link Publisher} that will trigger a timeout if * emitting before the first signal from this {@link Flux} * * @param <U> the type of the timeout Publisher * * @return a {@link Flux} that can time out if the first item does not come before * a signal from a companion {@link Publisher} * */
public final <U> Flux<T> timeout(Publisher<U> firstTimeout) { return timeout(firstTimeout, t -> never()); }
Signal a TimeoutException in case the first item from this Flux has not been emitted before the firstTimeout Publisher emits, and whenever each subsequent elements is not emitted before a Publisher generated from the latest element signals.

Params:
  • firstTimeout – the timeout Publisher that must not emit before the first signal from this Flux
  • nextTimeoutFactory – the timeout Publisher factory for each next item
Type parameters:
  • <U> – the type of the elements of the first timeout Publisher
  • <V> – the type of the elements of the subsequent timeout Publishers
Returns:a Flux that can time out if each element does not come before a signal from a per-item companion Publisher
/** * Signal a {@link TimeoutException} in case the first item from this {@link Flux} has * not been emitted before the {@code firstTimeout} {@link Publisher} emits, and whenever * each subsequent elements is not emitted before a {@link Publisher} generated from * the latest element signals. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/timeoutall.png" alt=""> * * @param firstTimeout the timeout {@link Publisher} that must not emit before the first signal from this {@link Flux} * @param nextTimeoutFactory the timeout {@link Publisher} factory for each next item * * @param <U> the type of the elements of the first timeout Publisher * @param <V> the type of the elements of the subsequent timeout Publishers * * @return a {@link Flux} that can time out if each element does not come before * a signal from a per-item companion {@link Publisher} */
public final <U, V> Flux<T> timeout(Publisher<U> firstTimeout, Function<? super T, ? extends Publisher<V>> nextTimeoutFactory) { return timeout(firstTimeout, nextTimeoutFactory, "first signal from a Publisher"); } private final <U, V> Flux<T> timeout(Publisher<U> firstTimeout, Function<? super T, ? extends Publisher<V>> nextTimeoutFactory, String timeoutDescription) { return onAssembly(new FluxTimeout<>(this, firstTimeout, nextTimeoutFactory, timeoutDescription)); }
Switch to a fallback Publisher in case the first item from this Flux has not been emitted before the firstTimeout Publisher emits, and whenever each subsequent elements is not emitted before a Publisher generated from the latest element signals.

Params:
  • firstTimeout – the timeout Publisher that must not emit before the first signal from this Flux
  • nextTimeoutFactory – the timeout Publisher factory for each next item
  • fallback – the fallback Publisher to subscribe when a timeout occurs
Type parameters:
  • <U> – the type of the elements of the first timeout Publisher
  • <V> – the type of the elements of the subsequent timeout Publishers
Returns:a Flux that can time out if each element does not come before a signal from a per-item companion Publisher
/** * Switch to a fallback {@link Publisher} in case the first item from this {@link Flux} has * not been emitted before the {@code firstTimeout} {@link Publisher} emits, and whenever * each subsequent elements is not emitted before a {@link Publisher} generated from * the latest element signals. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/timeoutallfallback.png" alt=""> * * @param firstTimeout the timeout {@link Publisher} that must not emit before the first signal from this {@link Flux} * @param nextTimeoutFactory the timeout {@link Publisher} factory for each next item * @param fallback the fallback {@link Publisher} to subscribe when a timeout occurs * * @param <U> the type of the elements of the first timeout Publisher * @param <V> the type of the elements of the subsequent timeout Publishers * * @return a {@link Flux} that can time out if each element does not come before * a signal from a per-item companion {@link Publisher} */
public final <U, V> Flux<T> timeout(Publisher<U> firstTimeout, Function<? super T, ? extends Publisher<V>> nextTimeoutFactory, Publisher<? extends T> fallback) { return onAssembly(new FluxTimeout<>(this, firstTimeout, nextTimeoutFactory, fallback)); }
Emit a Tuple2 pair of T1 the current clock time in millis (as a Long measured by the parallel Scheduler) and T2 the emitted data (as a T), for each item from this Flux.

Returns:a timestamped Flux
/** * Emit a {@link reactor.util.function.Tuple2} pair of T1 the current clock time in * millis (as a {@link Long} measured by the {@link Schedulers#parallel() parallel} * Scheduler) and T2 the emitted data (as a {@code T}), for each item from this {@link Flux}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/timestamp.png" alt=""> * * @return a timestamped {@link Flux} */
public final Flux<Tuple2<Long, T>> timestamp() { return timestamp(Schedulers.parallel()); }
Emit a Tuple2 pair of T1 the current clock time in millis (as a Long measured by the provided Scheduler) and T2 the emitted data (as a T), for each item from this Flux.

Params:
  • scheduler – the Scheduler to read time from
Returns:a timestamped Flux
/** * Emit a {@link reactor.util.function.Tuple2} pair of T1 the current clock time in * millis (as a {@link Long} measured by the provided {@link Scheduler}) and T2 * the emitted data (as a {@code T}), for each item from this {@link Flux}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/timestamp.png" alt=""> * * @param scheduler the {@link Scheduler} to read time from * @return a timestamped {@link Flux} */
public final Flux<Tuple2<Long, T>> timestamp(Scheduler scheduler) { Objects.requireNonNull(scheduler, "scheduler"); return map(d -> Tuples.of(scheduler.now(TimeUnit.MILLISECONDS), d)); }
Transform this Flux into a lazy Iterable blocking on Iterator.next() calls.

Note that iterating from within threads marked as "non-blocking only" is illegal and will cause an IllegalStateException to be thrown, but obtaining the Iterable itself within these threads is ok.

Returns:a blocking Iterable
/** * Transform this {@link Flux} into a lazy {@link Iterable} blocking on * {@link Iterator#next()} calls. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/toiterable.png" alt=""> * <p> * Note that iterating from within threads marked as "non-blocking only" is illegal and will * cause an {@link IllegalStateException} to be thrown, but obtaining the {@link Iterable} * itself within these threads is ok. * * @return a blocking {@link Iterable} */
public final Iterable<T> toIterable() { return toIterable(Queues.SMALL_BUFFER_SIZE); }
Transform this Flux into a lazy Iterable blocking on Iterator.next() calls.
Params:
  • batchSize – the bounded capacity to prefetch from this Flux or Integer.MAX_VALUE for unbounded demand

    Note that iterating from within threads marked as "non-blocking only" is illegal and will cause an IllegalStateException to be thrown, but obtaining the Iterable itself within these threads is ok.

Returns:a blocking Iterable
/** * Transform this {@link Flux} into a lazy {@link Iterable} blocking on * {@link Iterator#next()} calls. * * @param batchSize the bounded capacity to prefetch from this {@link Flux} or * {@code Integer.MAX_VALUE} for unbounded demand * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/toiterablen.png" alt=""> * <p> * Note that iterating from within threads marked as "non-blocking only" is illegal and will * cause an {@link IllegalStateException} to be thrown, but obtaining the {@link Iterable} * itself within these threads is ok. * * @return a blocking {@link Iterable} */
public final Iterable<T> toIterable(int batchSize) { return toIterable(batchSize, null); }
Transform this Flux into a lazy Iterable blocking on Iterator.next() calls.

Note that iterating from within threads marked as "non-blocking only" is illegal and will cause an IllegalStateException to be thrown, but obtaining the Iterable itself within these threads is ok.

Params:
  • batchSize – the bounded capacity to prefetch from this Flux or Integer.MAX_VALUE for unbounded demand
  • queueProvider – the supplier of the queue implementation to be used for storing elements emitted faster than the iteration
Returns:a blocking Iterable
/** * Transform this {@link Flux} into a lazy {@link Iterable} blocking on * {@link Iterator#next()} calls. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/toiterablen.png" alt=""> * <p> * Note that iterating from within threads marked as "non-blocking only" is illegal and will * cause an {@link IllegalStateException} to be thrown, but obtaining the {@link Iterable} * itself within these threads is ok. * * @param batchSize the bounded capacity to prefetch from this {@link Flux} or * {@code Integer.MAX_VALUE} for unbounded demand * @param queueProvider the supplier of the queue implementation to be used for storing * elements emitted faster than the iteration * * @return a blocking {@link Iterable} */
public final Iterable<T> toIterable(int batchSize, @Nullable Supplier<Queue<T>> queueProvider) { final Supplier<Queue<T>> provider; if(queueProvider == null){ provider = Queues.get(batchSize); } else{ provider = queueProvider; } return new BlockingIterable<>(this, batchSize, provider); }
Transform this Flux into a lazy Stream blocking for each source onNext call.

Note that iterating from within threads marked as "non-blocking only" is illegal and will cause an IllegalStateException to be thrown, but obtaining the Stream itself or applying lazy intermediate operation on the stream within these threads is ok.

Returns:a Stream of unknown size with onClose attached to Subscription.cancel()
/** * Transform this {@link Flux} into a lazy {@link Stream} blocking for each source * {@link Subscriber#onNext(Object) onNext} call. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/tostream.png" alt=""> * <p> * Note that iterating from within threads marked as "non-blocking only" is illegal and will * cause an {@link IllegalStateException} to be thrown, but obtaining the {@link Stream} * itself or applying lazy intermediate operation on the stream within these threads is ok. * * @return a {@link Stream} of unknown size with onClose attached to {@link Subscription#cancel()} */
public final Stream<T> toStream() { return toStream(Queues.SMALL_BUFFER_SIZE); }
Transform this Flux into a lazy Stream blocking for each source onNext call.
Params:
  • batchSize – the bounded capacity to prefetch from this Flux or Integer.MAX_VALUE for unbounded demand

    Note that iterating from within threads marked as "non-blocking only" is illegal and will cause an IllegalStateException to be thrown, but obtaining the Stream itself or applying lazy intermediate operation on the stream within these threads is ok.

Returns:a Stream of unknown size with onClose attached to Subscription.cancel()
/** * Transform this {@link Flux} into a lazy {@link Stream} blocking for each source * {@link Subscriber#onNext(Object) onNext} call. * * @param batchSize the bounded capacity to prefetch from this {@link Flux} or * {@code Integer.MAX_VALUE} for unbounded demand * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/tostream.png" alt=""> * <p> * Note that iterating from within threads marked as "non-blocking only" is illegal and will * cause an {@link IllegalStateException} to be thrown, but obtaining the {@link Stream} * itself or applying lazy intermediate operation on the stream within these threads is ok. * * @return a {@link Stream} of unknown size with onClose attached to {@link Subscription#cancel()} */
public final Stream<T> toStream(int batchSize) { final Supplier<Queue<T>> provider; provider = Queues.get(batchSize); return new BlockingIterable<>(this, batchSize, provider).stream(); }
Transform this Flux in order to generate a target Flux. Unlike compose(Function), the provided function is executed as part of assembly.

Function<Flux, Flux> applySchedulers = flux -> flux.subscribeOn(Schedulers.elastic())
                                                   .publishOn(Schedulers.parallel());
flux.transform(applySchedulers).map(v -> v * v).subscribe(); 
Params:
  • transformer – the Function to immediately map this Flux into a target Flux instance.
Type parameters:
  • <V> – the item type in the returned Flux
See Also:
Returns:a new Flux
/** * Transform this {@link Flux} in order to generate a target {@link Flux}. Unlike {@link #compose(Function)}, the * provided function is executed as part of assembly. * * <blockquote><pre> * {@code * Function<Flux, Flux> applySchedulers = flux -> flux.subscribeOn(Schedulers.elastic()) * .publishOn(Schedulers.parallel()); * flux.transform(applySchedulers).map(v -> v * v).subscribe();} * </pre></blockquote> * * @param transformer the {@link Function} to immediately map this {@link Flux} into a target {@link Flux} * instance. * @param <V> the item type in the returned {@link Flux} * * @return a new {@link Flux} * @see #compose(Function) for deferred composition of {@link Flux} for each {@link Subscriber} * @see #as for a loose conversion to an arbitrary type */
public final <V> Flux<V> transform(Function<? super Flux<T>, ? extends Publisher<V>> transformer) { return onAssembly(from(transformer.apply(this))); }
Split this Flux sequence into multiple Flux windows containing maxSize elements (or less for the final window) and starting from the first item. Each Flux window will onComplete after maxSize items have been routed.

Params:
  • maxSize – the maximum number of items to emit in the window before closing it
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a Flux of Flux windows based on element count
/** * Split this {@link Flux} sequence into multiple {@link Flux} windows containing * {@code maxSize} elements (or less for the final window) and starting from the first item. * Each {@link Flux} window will onComplete after {@code maxSize} items have been routed. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowsize.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure * upon cancellation or error triggered by a data signal. * * @param maxSize the maximum number of items to emit in the window before closing it * * @return a {@link Flux} of {@link Flux} windows based on element count */
public final Flux<Flux<T>> window(int maxSize) { return onAssembly(new FluxWindow<>(this, maxSize, Queues.get(maxSize))); }
Split this Flux sequence into multiple Flux windows of size maxSize, that each open every skip elements in the source.

When maxSize < skip : dropping windows

When maxSize > skip : overlapping windows

When maxSize == skip : exact windows

Params:
  • maxSize – the maximum number of items to emit in the window before closing it
  • skip – the number of items to count before opening and emitting a new window
@reactor.discardThe overlapping variant DOES NOT discard elements, as they might be part of another still valid window. The exact window and dropping window variants bot discard elements they internally queued for backpressure upon cancellation or error triggered by a data signal. The dropping window variant also discards elements in between windows.
Returns:a Flux of Flux windows based on element count and opened every skipCount
/** * Split this {@link Flux} sequence into multiple {@link Flux} windows of size * {@code maxSize}, that each open every {@code skip} elements in the source. * * <p> * When maxSize < skip : dropping windows * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowsizeskip.png" alt=""> * <p> * When maxSize > skip : overlapping windows * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowsizeskipover.png" alt=""> * <p> * When maxSize == skip : exact windows * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowsize.png" alt=""> * * @reactor.discard The overlapping variant DOES NOT discard elements, as they might be part of another still valid window. * The exact window and dropping window variants bot discard elements they internally queued for backpressure * upon cancellation or error triggered by a data signal. The dropping window variant also discards elements in between windows. * * @param maxSize the maximum number of items to emit in the window before closing it * @param skip the number of items to count before opening and emitting a new window * * @return a {@link Flux} of {@link Flux} windows based on element count and opened every skipCount */
public final Flux<Flux<T>> window(int maxSize, int skip) { return onAssembly(new FluxWindow<>(this, maxSize, skip, Queues.unbounded(Queues.XS_BUFFER_SIZE), Queues.unbounded(Queues.XS_BUFFER_SIZE))); }
Split this Flux sequence into continuous, non-overlapping windows where the window boundary is signalled by another Publisher

Params:
  • boundary – a Publisher to emit any item for a split signal and complete to terminate
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a Flux of Flux windows delimited by a given Publisher
/** * Split this {@link Flux} sequence into continuous, non-overlapping windows * where the window boundary is signalled by another {@link Publisher} * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowboundary.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure * upon cancellation or error triggered by a data signal. * * @param boundary a {@link Publisher} to emit any item for a split signal and complete to terminate * * @return a {@link Flux} of {@link Flux} windows delimited by a given {@link Publisher} */
public final Flux<Flux<T>> window(Publisher<?> boundary) { return onAssembly(new FluxWindowBoundary<>(this, boundary, Queues.unbounded(Queues.XS_BUFFER_SIZE))); }
Split this Flux sequence into continuous, non-overlapping windows that open for a windowingTimespan Duration (as measured on the parallel Scheduler).

Params:
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a Flux of Flux windows continuously opened for a given Duration
/** * Split this {@link Flux} sequence into continuous, non-overlapping windows that open * for a {@code windowingTimespan} {@link Duration} (as measured on the {@link Schedulers#parallel() parallel} * Scheduler). * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowtimespan.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure * upon cancellation or error triggered by a data signal. * * @param windowingTimespan the {@link Duration} to delimit {@link Flux} windows * * @return a {@link Flux} of {@link Flux} windows continuously opened for a given {@link Duration} */
public final Flux<Flux<T>> window(Duration windowingTimespan) { return window(windowingTimespan, Schedulers.parallel()); }
Split this Flux sequence into multiple Flux windows that open for a given windowingTimespan Duration, after which it closes with onComplete. Each window is opened at a regular timeShift interval, starting from the first item. Both durations are measured on the parallel Scheduler.

When windowingTimespan < openWindowEvery : dropping windows

When windowingTimespan > openWindowEvery : overlapping windows

When windowingTimespan == openWindowEvery : exact windows

Params:
  • windowingTimespan – the maximum Flux window Duration
  • openWindowEvery – the period of time at which to create new Flux windows
@reactor.discardThe overlapping variant DOES NOT discard elements, as they might be part of another still valid window. The exact window and dropping window variants bot discard elements they internally queued for backpressure upon cancellation or error triggered by a data signal. The dropping window variant also discards elements in between windows.
Returns:a Flux of Flux windows opened at regular intervals and closed after a Duration
/** * Split this {@link Flux} sequence into multiple {@link Flux} windows that open * for a given {@code windowingTimespan} {@link Duration}, after which it closes with onComplete. * Each window is opened at a regular {@code timeShift} interval, starting from the * first item. * Both durations are measured on the {@link Schedulers#parallel() parallel} Scheduler. * * <p> * When windowingTimespan < openWindowEvery : dropping windows * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowsizeskip.png" alt=""> * <p> * When windowingTimespan > openWindowEvery : overlapping windows * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowsizeskipover.png" alt=""> * <p> * When windowingTimespan == openWindowEvery : exact windows * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowsize.png" alt=""> * * @reactor.discard The overlapping variant DOES NOT discard elements, as they might be part of another still valid window. * The exact window and dropping window variants bot discard elements they internally queued for backpressure * upon cancellation or error triggered by a data signal. The dropping window variant also discards elements in between windows. * * @param windowingTimespan the maximum {@link Flux} window {@link Duration} * @param openWindowEvery the period of time at which to create new {@link Flux} windows * * @return a {@link Flux} of {@link Flux} windows opened at regular intervals and * closed after a {@link Duration} * */
public final Flux<Flux<T>> window(Duration windowingTimespan, Duration openWindowEvery) { return window(windowingTimespan, openWindowEvery, Schedulers.parallel()); }
Split this Flux sequence into continuous, non-overlapping windows that open for a windowingTimespan Duration (as measured on the provided Scheduler).

Params:
  • windowingTimespan – the Duration to delimit Flux windows
  • timer – a time-capable Scheduler instance to run on
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a Flux of Flux windows continuously opened for a given Duration
/** * Split this {@link Flux} sequence into continuous, non-overlapping windows that open * for a {@code windowingTimespan} {@link Duration} (as measured on the provided {@link Scheduler}). * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowtimespan.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure * upon cancellation or error triggered by a data signal. * * @param windowingTimespan the {@link Duration} to delimit {@link Flux} windows * @param timer a time-capable {@link Scheduler} instance to run on * * @return a {@link Flux} of {@link Flux} windows continuously opened for a given {@link Duration} */
public final Flux<Flux<T>> window(Duration windowingTimespan, Scheduler timer) { return window(interval(windowingTimespan, timer)); }
Split this Flux sequence into multiple Flux windows that open for a given windowingTimespan Duration, after which it closes with onComplete. Each window is opened at a regular timeShift interval, starting from the first item. Both durations are measured on the provided Scheduler.

When windowingTimespan < openWindowEvery : dropping windows

When windowingTimespan > openWindowEvery : overlapping windows

When openWindowEvery == openWindowEvery : exact windows

Params:
  • windowingTimespan – the maximum Flux window Duration
  • openWindowEvery – the period of time at which to create new Flux windows
  • timer – a time-capable Scheduler instance to run on
@reactor.discardThe overlapping variant DOES NOT discard elements, as they might be part of another still valid window. The exact window and dropping window variants bot discard elements they internally queued for backpressure upon cancellation or error triggered by a data signal. The dropping window variant also discards elements in between windows.
Returns:a Flux of Flux windows opened at regular intervals and closed after a Duration
/** * Split this {@link Flux} sequence into multiple {@link Flux} windows that open * for a given {@code windowingTimespan} {@link Duration}, after which it closes with onComplete. * Each window is opened at a regular {@code timeShift} interval, starting from the * first item. * Both durations are measured on the provided {@link Scheduler}. * * <p> * When windowingTimespan < openWindowEvery : dropping windows * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowsizeskip.png" alt=""> * <p> * When windowingTimespan > openWindowEvery : overlapping windows * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowsizeskipover.png" alt=""> * <p> * When openWindowEvery == openWindowEvery : exact windows * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowsize.png" alt=""> * * @reactor.discard The overlapping variant DOES NOT discard elements, as they might be part of another still valid window. * The exact window and dropping window variants bot discard elements they internally queued for backpressure * upon cancellation or error triggered by a data signal. The dropping window variant also discards elements in between windows. * * @param windowingTimespan the maximum {@link Flux} window {@link Duration} * @param openWindowEvery the period of time at which to create new {@link Flux} windows * @param timer a time-capable {@link Scheduler} instance to run on * * @return a {@link Flux} of {@link Flux} windows opened at regular intervals and * closed after a {@link Duration} */
public final Flux<Flux<T>> window(Duration windowingTimespan, Duration openWindowEvery, Scheduler timer) { if (openWindowEvery.equals(windowingTimespan)) { return window(windowingTimespan); } return windowWhen(interval(Duration.ZERO, openWindowEvery, timer), aLong -> Mono.delay(windowingTimespan, timer)); }
Split this Flux sequence into multiple Flux windows containing maxSize elements (or less for the final window) and starting from the first item. Each Flux window will onComplete once it contains maxSize elements OR it has been open for the given Duration (as measured on the parallel Scheduler).

Params:
  • maxSize – the maximum number of items to emit in the window before closing it
  • maxTime – the maximum Duration since the window was opened before closing it
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a Flux of Flux windows based on element count and duration
/** * Split this {@link Flux} sequence into multiple {@link Flux} windows containing * {@code maxSize} elements (or less for the final window) and starting from the first item. * Each {@link Flux} window will onComplete once it contains {@code maxSize} elements * OR it has been open for the given {@link Duration} (as measured on the {@link Schedulers#parallel() parallel} * Scheduler). * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowsizetimeout.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure * upon cancellation or error triggered by a data signal. * * @param maxSize the maximum number of items to emit in the window before closing it * @param maxTime the maximum {@link Duration} since the window was opened before closing it * * @return a {@link Flux} of {@link Flux} windows based on element count and duration */
public final Flux<Flux<T>> windowTimeout(int maxSize, Duration maxTime) { return windowTimeout(maxSize, maxTime , Schedulers.parallel()); }
Split this Flux sequence into multiple Flux windows containing maxSize elements (or less for the final window) and starting from the first item. Each Flux window will onComplete once it contains maxSize elements OR it has been open for the given Duration (as measured on the provided Scheduler).

Params:
  • maxSize – the maximum number of items to emit in the window before closing it
  • maxTime – the maximum Duration since the window was opened before closing it
  • timer – a time-capable Scheduler instance to run on
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a Flux of Flux windows based on element count and duration
/** * Split this {@link Flux} sequence into multiple {@link Flux} windows containing * {@code maxSize} elements (or less for the final window) and starting from the first item. * Each {@link Flux} window will onComplete once it contains {@code maxSize} elements * OR it has been open for the given {@link Duration} (as measured on the provided * {@link Scheduler}). * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowsizetimeout.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure * upon cancellation or error triggered by a data signal. * * @param maxSize the maximum number of items to emit in the window before closing it * @param maxTime the maximum {@link Duration} since the window was opened before closing it * @param timer a time-capable {@link Scheduler} instance to run on * * @return a {@link Flux} of {@link Flux} windows based on element count and duration */
public final Flux<Flux<T>> windowTimeout(int maxSize, Duration maxTime, Scheduler timer) { return onAssembly(new FluxWindowTimeout<>(this, maxSize, maxTime.toMillis(), timer)); }
Split this Flux sequence into multiple Flux windows delimited by the given predicate. A new window is opened each time the predicate returns true, at which point the previous window will receive the triggering element then onComplete.

Windows are lazily made available downstream at the point where they receive their first event (an element is pushed, the window errors). This variant shouldn't expose empty windows, as the separators are emitted into the windows they close.

Params:
  • boundaryTrigger – a predicate that triggers the next window when it becomes true.
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a Flux of Flux windows, bounded depending on the predicate.
/** * Split this {@link Flux} sequence into multiple {@link Flux} windows delimited by the * given predicate. A new window is opened each time the predicate returns true, at which * point the previous window will receive the triggering element then onComplete. * <p> * Windows are lazily made available downstream at the point where they receive their * first event (an element is pushed, the window errors). This variant shouldn't * expose empty windows, as the separators are emitted into * the windows they close. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowuntil.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure * upon cancellation or error triggered by a data signal. * * @param boundaryTrigger a predicate that triggers the next window when it becomes true. * @return a {@link Flux} of {@link Flux} windows, bounded depending * on the predicate. */
public final Flux<Flux<T>> windowUntil(Predicate<T> boundaryTrigger) { return windowUntil(boundaryTrigger, false); }
Split this Flux sequence into multiple Flux windows delimited by the given predicate. A new window is opened each time the predicate returns true.

Windows are lazily made available downstream at the point where they receive their first event (an element is pushed, the window completes or errors).

If cutBefore is true, the old window will onComplete and the triggering element will be emitted in the new window, which becomes immediately available. This variant can emit an empty window if the sequence starts with a separator.

Otherwise, the triggering element will be emitted in the old window before it does onComplete, similar to windowUntil(Predicate). This variant shouldn't expose empty windows, as the separators are emitted into the windows they close.

Params:
  • boundaryTrigger – a predicate that triggers the next window when it becomes true.
  • cutBefore – set to true to include the triggering element in the new window rather than the old.
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a Flux of Flux windows, bounded depending on the predicate.
/** * Split this {@link Flux} sequence into multiple {@link Flux} windows delimited by the * given predicate. A new window is opened each time the predicate returns true. * <p> * Windows are lazily made available downstream at the point where they receive their * first event (an element is pushed, the window completes or errors). * <p> * If {@code cutBefore} is true, the old window will onComplete and the triggering * element will be emitted in the new window, which becomes immediately available. * This variant can emit an empty window if the sequence starts with a separator. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowuntilcutbefore.png" alt=""> * <p> * Otherwise, the triggering element will be emitted in the old window before it does * onComplete, similar to {@link #windowUntil(Predicate)}. This variant shouldn't * expose empty windows, as the separators are emitted into the windows they close. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowuntilcutafter.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure * upon cancellation or error triggered by a data signal. * * @param boundaryTrigger a predicate that triggers the next window when it becomes true. * @param cutBefore set to true to include the triggering element in the new window rather than the old. * @return a {@link Flux} of {@link Flux} windows, bounded depending * on the predicate. */
public final Flux<Flux<T>> windowUntil(Predicate<T> boundaryTrigger, boolean cutBefore) { return windowUntil(boundaryTrigger, cutBefore, Queues.SMALL_BUFFER_SIZE); }
Split this Flux sequence into multiple Flux windows delimited by the given predicate and using a prefetch. A new window is opened each time the predicate returns true.

Windows are lazily made available downstream at the point where they receive their first event (an element is pushed, the window completes or errors).

If cutBefore is true, the old window will onComplete and the triggering element will be emitted in the new window. This variant can emit an empty window if the sequence starts with a separator.

Otherwise, the triggering element will be emitted in the old window before it does onComplete, similar to windowUntil(Predicate). This variant shouldn't expose empty windows, as the separators are emitted into the windows they close.

Params:
  • boundaryTrigger – a predicate that triggers the next window when it becomes true.
  • cutBefore – set to true to include the triggering element in the new window rather than the old.
  • prefetch – the request size to use for this Flux.
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal.
Returns:a Flux of Flux windows, bounded depending on the predicate.
/** * Split this {@link Flux} sequence into multiple {@link Flux} windows delimited by the given * predicate and using a prefetch. A new window is opened each time the predicate * returns true. * <p> * Windows are lazily made available downstream at the point where they receive their * first event (an element is pushed, the window completes or errors). * <p> * If {@code cutBefore} is true, the old window will onComplete and the triggering * element will be emitted in the new window. This variant can emit an empty window * if the sequence starts with a separator. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowuntilcutbefore.png" alt=""> * <p> * Otherwise, the triggering element will be emitted in the old window before it does * onComplete, similar to {@link #windowUntil(Predicate)}. This variant shouldn't * expose empty windows, as the separators are emitted into the windows they close. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowuntilcutafter.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure * upon cancellation or error triggered by a data signal. * * @param boundaryTrigger a predicate that triggers the next window when it becomes true. * @param cutBefore set to true to include the triggering element in the new window rather than the old. * @param prefetch the request size to use for this {@link Flux}. * @return a {@link Flux} of {@link Flux} windows, bounded depending * on the predicate. */
public final Flux<Flux<T>> windowUntil(Predicate<T> boundaryTrigger, boolean cutBefore, int prefetch) { return onAssembly(new FluxWindowPredicate<>(this, Queues.unbounded(prefetch), Queues.unbounded(prefetch), prefetch, boundaryTrigger, cutBefore ? FluxBufferPredicate.Mode.UNTIL_CUT_BEFORE : FluxBufferPredicate.Mode.UNTIL)); }
Split this Flux sequence into multiple Flux windows that stay open while a given predicate matches the source elements. Once the predicate returns false, the window closes with an onComplete and the triggering element is discarded.

Windows are lazily made available downstream at the point where they receive their first event (an element is pushed, the window completes or errors). Empty windows can happen when a sequence starts with a separator or contains multiple separators, but a sequence that finishes with a separator won't cause a remainder empty window to be emitted.

Params:
  • inclusionPredicate – a predicate that triggers the next window when it becomes false.
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal, as well as the triggering element(s) (that doesn't match the predicate).
Returns:a Flux of Flux windows, each containing subsequent elements that all passed a predicate.
/** * Split this {@link Flux} sequence into multiple {@link Flux} windows that stay open * while a given predicate matches the source elements. Once the predicate returns * false, the window closes with an onComplete and the triggering element is discarded. * <p> * Windows are lazily made available downstream at the point where they receive their * first event (an element is pushed, the window completes or errors). Empty windows * can happen when a sequence starts with a separator or contains multiple separators, * but a sequence that finishes with a separator won't cause a remainder empty window * to be emitted. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowwhile.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure * upon cancellation or error triggered by a data signal, as well as the triggering element(s) (that doesn't match * the predicate). * * @param inclusionPredicate a predicate that triggers the next window when it becomes false. * @return a {@link Flux} of {@link Flux} windows, each containing * subsequent elements that all passed a predicate. */
public final Flux<Flux<T>> windowWhile(Predicate<T> inclusionPredicate) { return windowWhile(inclusionPredicate, Queues.SMALL_BUFFER_SIZE); }
Split this Flux sequence into multiple Flux windows that stay open while a given predicate matches the source elements. Once the predicate returns false, the window closes with an onComplete and the triggering element is discarded.

Windows are lazily made available downstream at the point where they receive their first event (an element is pushed, the window completes or errors). Empty windows can happen when a sequence starts with a separator or contains multiple separators, but a sequence that finishes with a separator won't cause a remainder empty window to be emitted.

Params:
  • inclusionPredicate – a predicate that triggers the next window when it becomes false.
  • prefetch – the request size to use for this Flux.
@reactor.discardThis operator discards elements it internally queued for backpressure upon cancellation or error triggered by a data signal, as well as the triggering element(s) (that doesn't match the predicate).
Returns:a Flux of Flux windows, each containing subsequent elements that all passed a predicate.
/** * Split this {@link Flux} sequence into multiple {@link Flux} windows that stay open * while a given predicate matches the source elements. Once the predicate returns * false, the window closes with an onComplete and the triggering element is discarded. * <p> * Windows are lazily made available downstream at the point where they receive their * first event (an element is pushed, the window completes or errors). Empty windows * can happen when a sequence starts with a separator or contains multiple separators, * but a sequence that finishes with a separator won't cause a remainder empty window * to be emitted. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowwhile.png" alt=""> * * @reactor.discard This operator discards elements it internally queued for backpressure * upon cancellation or error triggered by a data signal, as well as the triggering element(s) (that doesn't match * the predicate). * * @param inclusionPredicate a predicate that triggers the next window when it becomes false. * @param prefetch the request size to use for this {@link Flux}. * @return a {@link Flux} of {@link Flux} windows, each containing * subsequent elements that all passed a predicate. */
public final Flux<Flux<T>> windowWhile(Predicate<T> inclusionPredicate, int prefetch) { return onAssembly(new FluxWindowPredicate<>(this, Queues.unbounded(prefetch), Queues.unbounded(prefetch), prefetch, inclusionPredicate, FluxBufferPredicate.Mode.WHILE)); }
Split this Flux sequence into potentially overlapping windows controlled by items of a start Publisher and end Publisher derived from the start values.

When Open signal is strictly not overlapping Close signal : dropping windows

When Open signal is strictly more frequent than Close signal : overlapping windows

When Open signal is exactly coordinated with Close signal : exact windows

Params:
  • bucketOpening – a Publisher that opens a new window when it emits any item
  • closeSelector – a Function given an opening signal and returning a Publisher that will close the window when emitting
Type parameters:
  • <U> – the type of the sequence opening windows
  • <V> – the type of the sequence closing windows opened by the bucketOpening Publisher's elements
@reactor.discardThis operator DOES NOT discard elements.
Returns:a Flux of Flux windows opened by signals from a first Publisher and lasting until a selected second Publisher emits
/** * Split this {@link Flux} sequence into potentially overlapping windows controlled by items of a * start {@link Publisher} and end {@link Publisher} derived from the start values. * * <p> * When Open signal is strictly not overlapping Close signal : dropping windows * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowopenclose.png" alt=""> * <p> * When Open signal is strictly more frequent than Close signal : overlapping windows * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowopencloseover.png" alt=""> * <p> * When Open signal is exactly coordinated with Close signal : exact windows * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/windowboundary.png" alt=""> * * @reactor.discard This operator DOES NOT discard elements. * * @param bucketOpening a {@link Publisher} that opens a new window when it emits any item * @param closeSelector a {@link Function} given an opening signal and returning a {@link Publisher} that * will close the window when emitting * * @param <U> the type of the sequence opening windows * @param <V> the type of the sequence closing windows opened by the bucketOpening Publisher's elements * * @return a {@link Flux} of {@link Flux} windows opened by signals from a first * {@link Publisher} and lasting until a selected second {@link Publisher} emits */
public final <U, V> Flux<Flux<T>> windowWhen(Publisher<U> bucketOpening, final Function<? super U, ? extends Publisher<V>> closeSelector) { return onAssembly(new FluxWindowWhen<>(this, bucketOpening, closeSelector, Queues.unbounded(Queues.XS_BUFFER_SIZE))); }
Combine the most recently emitted values from both this Flux and another Publisher through a BiFunction and emits the result.

The operator will drop values from this Flux until the other Publisher produces any value.

If the other Publisher completes without any value, the sequence is completed.

Params:
  • other – the Publisher to combine with
  • resultSelector – the bi-function called with each pair of source and other elements that should return a single value to be emitted
Type parameters:
  • <U> – the other Publisher sequence type
  • <R> – the result type
Returns:a combined Flux gated by another Publisher
/** * Combine the most recently emitted values from both this {@link Flux} and another * {@link Publisher} through a {@link BiFunction} and emits the result. * <p> * The operator will drop values from this {@link Flux} until the other * {@link Publisher} produces any value. * <p> * If the other {@link Publisher} completes without any value, the sequence is completed. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/withlatestfrom.png" alt=""> * * @param other the {@link Publisher} to combine with * @param resultSelector the bi-function called with each pair of source and other * elements that should return a single value to be emitted * * @param <U> the other {@link Publisher} sequence type * @param <R> the result type * * @return a combined {@link Flux} gated by another {@link Publisher} */
public final <U, R> Flux<R> withLatestFrom(Publisher<? extends U> other, BiFunction<? super T, ? super U, ? extends R > resultSelector){ return onAssembly(new FluxWithLatestFrom<>(this, other, resultSelector)); }
Zip this Flux with another Publisher source, that is to say wait for both to emit one element and combine these elements once into a Tuple2. The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios.

Params:
Type parameters:
  • <T2> – type of the value from source2
Returns:a zipped Flux
/** * Zip this {@link Flux} with another {@link Publisher} source, that is to say wait * for both to emit one element and combine these elements once into a {@link Tuple2}. * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zipt.png" alt=""> * <p> * @param source2 The second source {@link Publisher} to zip with this {@link Flux}. * @param <T2> type of the value from source2 * * @return a zipped {@link Flux} * */
public final <T2> Flux<Tuple2<T, T2>> zipWith(Publisher<? extends T2> source2) { return zipWith(source2, tuple2Function()); }
Zip this Flux with another Publisher source, that is to say wait for both to emit one element and combine these elements using a combinator BiFunction The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios.

Params:
  • source2 – The second source Publisher to zip with this Flux.
  • combinator – The aggregate function that will receive a unique value from each source and return the value to signal downstream
Type parameters:
  • <T2> – type of the value from source2
  • <V> – The produced output after transformation by the combinator
Returns:a zipped Flux
/** * Zip this {@link Flux} with another {@link Publisher} source, that is to say wait * for both to emit one element and combine these elements using a {@code combinator} * {@link BiFunction} * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zip.png" alt=""> * <p> * @param source2 The second source {@link Publisher} to zip with this {@link Flux}. * @param combinator The aggregate function that will receive a unique value from each * source and return the value to signal downstream * @param <T2> type of the value from source2 * @param <V> The produced output after transformation by the combinator * * @return a zipped {@link Flux} */
public final <T2, V> Flux<V> zipWith(Publisher<? extends T2> source2, final BiFunction<? super T, ? super T2, ? extends V> combinator) { if (this instanceof FluxZip) { @SuppressWarnings("unchecked") FluxZip<T, V> o = (FluxZip<T, V>) this; Flux<V> result = o.zipAdditionalSource(source2, combinator); if (result != null) { return result; } } return zip(this, source2, combinator); }
Zip this Flux with another Publisher source, that is to say wait for both to emit one element and combine these elements using a combinator BiFunction The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios.

Params:
  • source2 – The second source Publisher to zip with this Flux.
  • prefetch – the request size to use for this Flux and the other Publisher
  • combinator – The aggregate function that will receive a unique value from each source and return the value to signal downstream
Type parameters:
  • <T2> – type of the value from source2
  • <V> – The produced output after transformation by the combinator
Returns:a zipped Flux
/** * Zip this {@link Flux} with another {@link Publisher} source, that is to say wait * for both to emit one element and combine these elements using a {@code combinator} * {@link BiFunction} * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zipp.png" alt=""> * <p> * @param source2 The second source {@link Publisher} to zip with this {@link Flux}. * @param prefetch the request size to use for this {@link Flux} and the other {@link Publisher} * @param combinator The aggregate function that will receive a unique value from each * source and return the value to signal downstream * @param <T2> type of the value from source2 * @param <V> The produced output after transformation by the combinator * * @return a zipped {@link Flux} */
@SuppressWarnings("unchecked") public final <T2, V> Flux<V> zipWith(Publisher<? extends T2> source2, int prefetch, BiFunction<? super T, ? super T2, ? extends V> combinator) { return zip(objects -> combinator.apply((T) objects[0], (T2) objects[1]), prefetch, this, source2); }
Zip this Flux with another Publisher source, that is to say wait for both to emit one element and combine these elements once into a Tuple2. The operator will continue doing so until any of the sources completes. Errors will immediately be forwarded. This "Step-Merge" processing is especially useful in Scatter-Gather scenarios.

Params:
  • source2 – The second source Publisher to zip with this Flux.
  • prefetch – the request size to use for this Flux and the other Publisher
Type parameters:
  • <T2> – type of the value from source2
Returns:a zipped Flux
/** * Zip this {@link Flux} with another {@link Publisher} source, that is to say wait * for both to emit one element and combine these elements once into a {@link Tuple2}. * The operator will continue doing so until any of the sources completes. * Errors will immediately be forwarded. * This "Step-Merge" processing is especially useful in Scatter-Gather scenarios. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zipp.png" alt=""> * <p> * @param source2 The second source {@link Publisher} to zip with this {@link Flux}. * @param prefetch the request size to use for this {@link Flux} and the other {@link Publisher} * @param <T2> type of the value from source2 * * @return a zipped {@link Flux} */
public final <T2> Flux<Tuple2<T, T2>> zipWith(Publisher<? extends T2> source2, int prefetch) { return zipWith(source2, prefetch, tuple2Function()); }
Zip elements from this Flux with the content of an Iterable, that is to say combine one element from each, pairwise, into a Tuple2.

Params:
Type parameters:
  • <T2> – the value type of the other iterable sequence
Returns:a zipped Flux
/** * Zip elements from this {@link Flux} with the content of an {@link Iterable}, that is * to say combine one element from each, pairwise, into a {@link Tuple2}. * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zipwithiterable.png" alt=""> * * @param iterable the {@link Iterable} to zip with * @param <T2> the value type of the other iterable sequence * * @return a zipped {@link Flux} * */
@SuppressWarnings("unchecked") public final <T2> Flux<Tuple2<T, T2>> zipWithIterable(Iterable<? extends T2> iterable) { return zipWithIterable(iterable, tuple2Function()); }
Zip elements from this Flux with the content of an Iterable, that is to say combine one element from each, pairwise, using the given zipper BiFunction.

Params:
Type parameters:
  • <T2> – the value type of the other iterable sequence
  • <V> – the result type
Returns:a zipped Flux
/** * Zip elements from this {@link Flux} with the content of an {@link Iterable}, that is * to say combine one element from each, pairwise, using the given zipper {@link BiFunction}. * * * <p> * <img class="marble" src="https://raw.githubusercontent.com/reactor/reactor-core/v3.1.3.RELEASE/src/docs/marble/zipwithiterable.png" alt=""> * * @param iterable the {@link Iterable} to zip with * @param zipper the {@link BiFunction} pair combinator * * @param <T2> the value type of the other iterable sequence * @param <V> the result type * * @return a zipped {@link Flux} * */
public final <T2, V> Flux<V> zipWithIterable(Iterable<? extends T2> iterable, BiFunction<? super T, ? super T2, ? extends V> zipper) { return onAssembly(new FluxZipIterable<>(this, iterable, zipper)); }
To be used by custom operators: invokes assembly Hooks pointcut given a Flux, potentially returning a new Flux. This is for example useful to activate cross-cutting concerns at assembly time, eg. a generalized checkpoint().
Params:
  • source – the source to apply assembly hooks onto
Type parameters:
  • <T> – the value type
Returns:the source, potentially wrapped with assembly time cross-cutting behavior
/** * To be used by custom operators: invokes assembly {@link Hooks} pointcut given a * {@link Flux}, potentially returning a new {@link Flux}. This is for example useful * to activate cross-cutting concerns at assembly time, eg. a generalized * {@link #checkpoint()}. * * @param <T> the value type * @param source the source to apply assembly hooks onto * * @return the source, potentially wrapped with assembly time cross-cutting behavior */
@SuppressWarnings("unchecked") protected static <T> Flux<T> onAssembly(Flux<T> source) { Function<Publisher, Publisher> hook = Hooks.onEachOperatorHook; if(hook == null) { return source; } return (Flux<T>)hook.apply(source); }
To be used by custom operators: invokes assembly Hooks pointcut given a Flux, potentially returning a new Flux. This is for example useful to activate cross-cutting concerns at assembly time, eg. a generalized checkpoint().
Params:
  • source – the source to apply assembly hooks onto
Type parameters:
  • <T> – the value type
Returns:the source, potentially wrapped with assembly time cross-cutting behavior
/** * To be used by custom operators: invokes assembly {@link Hooks} pointcut given a * {@link Flux}, potentially returning a new {@link Flux}. This is for example useful * to activate cross-cutting concerns at assembly time, eg. a generalized * {@link #checkpoint()}. * * @param <T> the value type * @param source the source to apply assembly hooks onto * * @return the source, potentially wrapped with assembly time cross-cutting behavior */
@SuppressWarnings("unchecked") protected static <T> Flux<T> onLastAssembly(Flux<T> source) { Function<Publisher, Publisher> hook = Hooks.onLastOperatorHook; if(hook == null) { return source; } return (Flux<T>)Objects.requireNonNull(hook.apply(source), "LastOperator hook returned null"); }
To be used by custom operators: invokes assembly Hooks pointcut given a ConnectableFlux, potentially returning a new ConnectableFlux. This is for example useful to activate cross-cutting concerns at assembly time, eg. a generalized checkpoint().
Params:
  • source – the source to apply assembly hooks onto
Type parameters:
  • <T> – the value type
Returns:the source, potentially wrapped with assembly time cross-cutting behavior
/** * To be used by custom operators: invokes assembly {@link Hooks} pointcut given a * {@link ConnectableFlux}, potentially returning a new {@link ConnectableFlux}. This * is for example useful to activate cross-cutting concerns at assembly time, eg. a * generalized {@link #checkpoint()}. * * @param <T> the value type * @param source the source to apply assembly hooks onto * * @return the source, potentially wrapped with assembly time cross-cutting behavior */
@SuppressWarnings("unchecked") protected static <T> ConnectableFlux<T> onAssembly(ConnectableFlux<T> source) { Function<Publisher, Publisher> hook = Hooks.onEachOperatorHook; if(hook == null) { return source; } return (ConnectableFlux<T>)hook.apply(source); } @Override public String toString() { return getClass().getSimpleName(); } final <V> Flux<V> flatMap(Function<? super T, ? extends Publisher<? extends V>> mapper, boolean delayError, int concurrency, int prefetch) { return onAssembly(new FluxFlatMap<>( this, mapper, delayError, concurrency, Queues.get(concurrency), prefetch, Queues.get(prefetch) )); } final <R> Flux<R> flatMapSequential(Function<? super T, ? extends Publisher<? extends R>> mapper, boolean delayError, int maxConcurrency, int prefetch) { return onAssembly(new FluxMergeSequential<>(this, mapper, maxConcurrency, prefetch, delayError ? FluxConcatMap.ErrorMode.END : FluxConcatMap.ErrorMode.IMMEDIATE)); } @SuppressWarnings("unchecked") static <T> Flux<T> doOnSignal(Flux<T> source, @Nullable Consumer<? super Subscription> onSubscribe, @Nullable Consumer<? super T> onNext, @Nullable Consumer<? super Throwable> onError, @Nullable Runnable onComplete, @Nullable Runnable onAfterTerminate, @Nullable LongConsumer onRequest, @Nullable Runnable onCancel) { if (source instanceof Fuseable) { return onAssembly(new FluxPeekFuseable<>(source, onSubscribe, onNext, onError, onComplete, onAfterTerminate, onRequest, onCancel)); } return onAssembly(new FluxPeek<>(source, onSubscribe, onNext, onError, onComplete, onAfterTerminate, onRequest, onCancel)); }
Returns the appropriate Mono instance for a known Supplier Flux.
Params:
  • supplier – the supplier Flux
Returns:the mono representing that Flux
/** * Returns the appropriate Mono instance for a known Supplier Flux. * * @param supplier the supplier Flux * * @return the mono representing that Flux */
static <T> Mono<T> convertToMono(Callable<T> supplier) { if (supplier instanceof Fuseable.ScalarCallable) { Fuseable.ScalarCallable<T> scalarCallable = (Fuseable.ScalarCallable<T>) supplier; T v; try { v = scalarCallable.call(); } catch (Exception e) { return Mono.error(e); } if (v == null) { return Mono.empty(); } return Mono.just(v); } return Mono.onAssembly(new MonoCallable<>(supplier)); } @SafeVarargs static <I> Flux<I> merge(int prefetch, boolean delayError, Publisher<? extends I>... sources) { if (sources.length == 0) { return empty(); } if (sources.length == 1) { return from(sources[0]); } return onAssembly(new FluxMerge<>(sources, delayError, sources.length, Queues.get(sources.length), prefetch, Queues.get(prefetch))); } @SafeVarargs static <I> Flux<I> mergeSequential(int prefetch, boolean delayError, Publisher<? extends I>... sources) { if (sources.length == 0) { return empty(); } if (sources.length == 1) { return from(sources[0]); } return onAssembly(new FluxMergeSequential<>(new FluxArray<>(sources), identityFunction(), sources.length, prefetch, delayError ? FluxConcatMap.ErrorMode.END : FluxConcatMap.ErrorMode.IMMEDIATE)); } static <T> Flux<T> mergeSequential(Publisher<? extends Publisher<? extends T>> sources, boolean delayError, int maxConcurrency, int prefetch) { return onAssembly(new FluxMergeSequential<>(from(sources), identityFunction(), maxConcurrency, prefetch, delayError ? FluxConcatMap.ErrorMode.END : FluxConcatMap.ErrorMode.IMMEDIATE)); } static <I> Flux<I> mergeSequential(Iterable<? extends Publisher<? extends I>> sources, boolean delayError, int maxConcurrency, int prefetch) { return onAssembly(new FluxMergeSequential<>(new FluxIterable<>(sources), identityFunction(), maxConcurrency, prefetch, delayError ? FluxConcatMap.ErrorMode.END : FluxConcatMap.ErrorMode.IMMEDIATE)); } static BooleanSupplier countingBooleanSupplier(BooleanSupplier predicate, long max) { if (max <= 0) { return predicate; } return new BooleanSupplier() { long n; @Override public boolean getAsBoolean() { return n++ < max && predicate.getAsBoolean(); } }; } static <O> Predicate<O> countingPredicate(Predicate<O> predicate, long max) { if (max == 0) { return predicate; } return new Predicate<O>() { long n; @Override public boolean test(O o) { return n++ < max && predicate.test(o); } }; } @SuppressWarnings("unchecked") static <O> Supplier<Set<O>> hashSetSupplier() { return SET_SUPPLIER; } @SuppressWarnings("unchecked") static <O> Supplier<List<O>> listSupplier() { return LIST_SUPPLIER; } @SuppressWarnings("unchecked") static <U, V> BiPredicate<U, V> equalPredicate() { return OBJECT_EQUAL; } @SuppressWarnings("unchecked") static <T> Function<T, T> identityFunction(){ return IDENTITY_FUNCTION; } @SuppressWarnings("unchecked") static <A, B> BiFunction<A, B, Tuple2<A, B>> tuple2Function() { return TUPLE2_BIFUNCTION; }
Unchecked wrap of Publisher as Flux, supporting Fuseable sources
Params:
Type parameters:
  • <I> – input upstream type
Returns:a wrapped Flux
/** * Unchecked wrap of {@link Publisher} as {@link Flux}, supporting {@link Fuseable} sources * * @param source the {@link Publisher} to wrap * @param <I> input upstream type * @return a wrapped {@link Flux} */
@SuppressWarnings("unchecked") static <I> Flux<I> wrap(Publisher<? extends I> source){ if(source instanceof Mono){ if(source instanceof Fuseable){ return new FluxSourceMonoFuseable<>((Mono<I>)source); } return new FluxSourceMono<>((Mono<I>)source); } if(source instanceof Fuseable){ return new FluxSourceFuseable<>(source); } return new FluxSource<>(source); } @SuppressWarnings("rawtypes") static final BiFunction TUPLE2_BIFUNCTION = Tuples::of; @SuppressWarnings("rawtypes") static final Supplier LIST_SUPPLIER = ArrayList::new; @SuppressWarnings("rawtypes") static final Supplier SET_SUPPLIER = HashSet::new; static final BooleanSupplier ALWAYS_BOOLEAN_SUPPLIER = () -> true; static final BiPredicate OBJECT_EQUAL = Object::equals; @SuppressWarnings("rawtypes") static final Function IDENTITY_FUNCTION = Function.identity(); }