Interface StreamStageWithKey<T,K>

Type Parameters:
T - type of the stream items
K - type of the key
All Superinterfaces:
GeneralStageWithKey<T,K>

public interface StreamStageWithKey<T,K> extends GeneralStageWithKey<T,K>
An intermediate step while constructing a pipeline transform that involves a grouping key, such as windowed group-and-aggregate. Some transforms use the grouping key only to partition the stream (such as mapUsingIMap(java.lang.String, com.hazelcast.function.BiFunctionEx<? super T, ? super V, ? extends R>)).
Since:
Jet 3.0
  • Method Details

    • window

      @Nonnull StageWithKeyAndWindow<T,K> window(@Nonnull WindowDefinition wDef)
      Adds the definition of the window to use in the group-and-aggregate pipeline stage being constructed.
    • mapStateful

      @Nonnull <S, R> StreamStage<R> mapStateful(long ttl, @Nonnull SupplierEx<? extends S> createFn, @Nonnull TriFunction<? super S,? super K,? super T,? extends R> mapFn, @Nonnull TriFunction<? super S,? super K,? super Long,? extends R> onEvictFn)
      Attaches a stage that performs a stateful mapping operation. createFn returns the object that holds the state. Jet passes this object along with each input item to mapFn, which can update the object's state. For each grouping key there's a separate state object. The state object will be included in the state snapshot, so it survives job restarts. For this reason it must be serializable.

      If the given ttl is greater than zero, Jet will consider the state object stale if its time-to-live has expired. The state object for a given key has a timestamp attached to it: the top timestamp of any event with that key seen so far. Whenever the watermark advances, Jet discards all state objects with a timestamp less than wm - ttl. Just before discarding the state object, Jet calls onEvictFn on it. The function can return an output item that will be emitted, or null if it doesn't need to emit an item. If TTL is used, Jet also drops late events; otherwise, all events are processed.

      This sample takes a stream of pairs (serverId, latency) representing the latencies of serving individual requests and keeps track, separately for each server, of the total latency accumulated over individual sessions — bursts of server activity separated by quiet periods of one minute or more. For each input item it outputs the accumulated latency so far and when a session ends, it outputs a special entry that reports the total latency for that session.

      
       StreamStage<Entry<String, Long>> latencies = null;
       StreamStage<Entry<String, Long>> cumulativeLatencies = latencies
               .groupingKey(Entry::getKey)
               .mapStateful(
                       MINUTES.toMillis(1),
                       LongAccumulator::new,
                       (sum, key, entry) -> {
                           sum.add(entry.getValue());
                           return entry(key, sum.get());
                       },
                       (sum, key, time) -> entry(String.format(
                               "%s:totalForSession:%d", key, time), sum.get())
               );
       

      The given functions must be stateless and cooperative.

      Type Parameters:
      S - type of the state object
      R - type of the result
      Parameters:
      ttl - time-to-live for each state object, disabled if zero or less
      createFn - function that returns the state object
      mapFn - function that receives the state object and the input item and outputs the result item. It may modify the state object.
      onEvictFn - function that Jet calls when evicting a state object
    • mapStateful

      @Nonnull <S, R> StreamStage<R> mapStateful(@Nonnull SupplierEx<? extends S> createFn, @Nonnull TriFunction<? super S,? super K,? super T,? extends R> mapFn)
      Description copied from interface: GeneralStageWithKey
      Attaches a stage that performs a stateful mapping operation. createFn returns the object that holds the state. Jet passes this object along with each input item and its key to mapFn, which can update the object's state. For each grouping key there's a separate state object. The state object will be included in the state snapshot, so it survives job restarts. For this reason it must be serializable. If you want to return the state variable from mapFn, then the return value must be a copy of state variable to avoid situations in which the result of mapFn is modified after being emitted or where the state is modified by downstream processors.

      If you want to return the state variable from mapFn, then the return value must be a copy of state variable to avoid situations in which the result of mapFn is modified after being emitted or where the state is modified by downstream processors.

      This sample takes a stream of pairs (serverId, latency) representing the latencies of serving individual requests and outputs the cumulative latency of all handled requests so far, for each server separately:

      
       GeneralStage<Entry<String, Long>> latencies;
       GeneralStage<Entry<String, Long>> cumulativeLatencies = latencies
               .groupingKey(Entry::getKey)
               .mapStateful(
                       LongAccumulator::new,
                       (sum, key, entry) -> {
                           sum.add(entry.getValue());
                           return entry(key, sum.get());
                       }
               );
       
      This code has the same result as latencies.groupingKey(Entry::getKey).rollingAggregate(summing()).

      This operation is subject to memory limits. See InstanceConfig.setMaxProcessorAccumulatedRecords(long) for more information.

      Specified by:
      mapStateful in interface GeneralStageWithKey<T,K>
      Type Parameters:
      S - type of the state object
      R - type of the result
      Parameters:
      createFn - function that returns the state object
      mapFn - function that receives the state object and the input item and outputs the result item. It may modify the state object. It must be stateless and cooperative.
    • filterStateful

      @Nonnull <S> StreamStage<T> filterStateful(long ttl, @Nonnull SupplierEx<? extends S> createFn, @Nonnull BiPredicateEx<? super S,? super T> filterFn)
      Attaches a stage that performs a stateful filtering operation. createFn returns the object that holds the state. Jet passes this object along with each input item to filterFn, which can update the object's state. For each grouping key there's a separate state object. The state object will be included in the state snapshot, so it survives job restarts. For this reason it must be serializable.

      If the given ttl is greater than zero, Jet will consider the state object stale if its time-to-live has expired. The state object has a timestamp attached to it: the top timestamp of any event with the same key seen so far. Upon seeing another event, Jet compares the state timestamp with the current watermark. If it is less than wm - ttl, it discards the state object and creates a new one before processing the event. If TTL is used, Jet also drops late events; otherwise, all events are processed.

      This sample receives a stream of pairs (serverId, requestLatency) that represent the latencies of individual requests served by a cluster of servers. It emits the record-breaking (worst so far) latencies for each server independently and resets the score after one minute of inactivity on a given server.

      
       StreamStage<Entry<String, Long>> latencies;
       StreamStage<Entry<String, Long>> topLatencies = latencies
               .groupingKey(Entry::getKey)
               .filterStateful(
                       MINUTES.toMillis(1),
                       LongAccumulator::new,
                       (topLatencyState, entry) -> {
                           long currLatency = entry.getValue();
                           long topLatency = topLatencyState.get();
                           topLatencyState.set(Math.max(currLatency, topLatency));
                           return currLatency > topLatency;
                       }
               );
       

      The given functions must be stateless and cooperative.

      Type Parameters:
      S - type of the state object
      Parameters:
      ttl - time-to-live for each state object, disabled if zero or less
      createFn - function that returns the state object
      filterFn - predicate that receives the state object and the input item and outputs a boolean value. It may modify the state object.
    • filterStateful

      @Nonnull default <S> StreamStage<T> filterStateful(@Nonnull SupplierEx<? extends S> createFn, @Nonnull BiPredicateEx<? super S,? super T> filterFn)
      Description copied from interface: GeneralStageWithKey
      Attaches a stage that performs a stateful filtering operation. createFn returns the object that holds the state. Jet passes this object along with each input item and its key to filterFn, which can update the object's state. For each grouping key there's a separate state object. The state object will be included in the state snapshot, so it survives job restarts. For this reason it must be serializable.

      This sample groups a stream of strings by length and decimates each group (throws out every 10th string of each length):

      
       GeneralStage<String> decimated = input
               .groupingKey(String::length)
               .filterStateful(
                       LongAccumulator::new,
                       (counter, item) -> {
                           counter.add(1);
                           return counter.get() % 10 != 0;
                       }
               );
       

      The given functions must be stateless and cooperative.

      This operation is subject to memory limits. See InstanceConfig.setMaxProcessorAccumulatedRecords(long) for more information.

      Specified by:
      filterStateful in interface GeneralStageWithKey<T,K>
      Type Parameters:
      S - type of the state object
      Parameters:
      createFn - function that returns the state object
      filterFn - predicate that receives the state object and the input item and outputs a boolean value. It may modify the state object.
    • flatMapStateful

      @Nonnull <S, R> StreamStage<R> flatMapStateful(long ttl, @Nonnull SupplierEx<? extends S> createFn, @Nonnull TriFunction<? super S,? super K,? super T,? extends Traverser<R>> flatMapFn, @Nonnull TriFunction<? super S,? super K,? super Long,? extends Traverser<R>> onEvictFn)
      Attaches a stage that performs a stateful flat-mapping operation. createFn returns the object that holds the state. Jet passes this object along with each input item to flatMapFn, which can update the object's state. For each grouping key there's a separate state object. The state object will be included in the state snapshot, so it survives job restarts. For this reason it must be serializable.

      If the given ttl is greater than zero, Jet will consider the state object stale if its time-to-live has expired. The state object for a given key has a timestamp attached to it: the top timestamp of any event with that key seen so far. Whenever the watermark advances, Jet discards all state objects with a timestamp less than wm - ttl. Just before discarding the state object, Jet calls onEvictFn on it. The function returns a traverser over the items it wants to emit, or it can return an empty traverser. If TTL is used, Jet also drops late events; otherwise, all events are processed.

      This sample groups a stream of strings by length and inserts punctuation (a special string) after every 10th string in each group, or after one minute elapses without further input for a given key:

      
       StreamStage<String> punctuated = input
               .groupingKey(String::length)
               .flatMapStateful(
                       MINUTES.toMillis(1),
                       LongAccumulator::new,
                       (counter, key, item) -> {
                           counter.add(1);
                           return counter.get() % 10 == 0
                                   ? Traversers.traverseItems("punctuation" + key, item)
                                   : Traversers.singleton(item);
                       },
                       (counter, key, wm) -> Traversers.singleton("punctuation" + key)
               );
       

      The given functions must be stateless and cooperative.

      Type Parameters:
      S - type of the state object
      R - type of the result
      Parameters:
      ttl - time-to-live for each state object, disabled if zero or less
      createFn - function that returns the state object
      flatMapFn - function that receives the state object and the input item and outputs the result items. It may modify the state object.
      onEvictFn - function that Jet calls when evicting a state object
    • flatMapStateful

      @Nonnull <S, R> StreamStage<R> flatMapStateful(@Nonnull SupplierEx<? extends S> createFn, @Nonnull TriFunction<? super S,? super K,? super T,? extends Traverser<R>> flatMapFn)
      Description copied from interface: GeneralStageWithKey
      Attaches a stage that performs a stateful flat-mapping operation. createFn returns the object that holds the state. Jet passes this object along with each input item and its key to flatMapFn, which can update the object's state. For each grouping key there's a separate state object. The state object will be included in the state snapshot, so it survives job restarts. For this reason it must be serializable. If you want to return the state variable from flatMapStateful, then the return value must be a copy of state variable to avoid situations in which the result of mapFn is modified after being emitted or where the state is modified by downstream processors.

      If you want to return the state variable from flatMapFn, then the return value must be a copy of state variable to avoid situations in which the result of mapFn is modified after being emitted or where the state is modified by downstream processors.

      This sample groups a stream of strings by length and inserts punctuation (a special string) after every 10th string in each group:

      
       GeneralStage<String> punctuated = input
               .groupingKey(String::length)
               .flatMapStateful(
                       LongAccumulator::new,
                       (counter, key, item) -> {
                           counter.add(1);
                           return counter.get() % 10 == 0
                                   ? Traversers.traverseItems("punctuation" + key, item)
                                   : Traversers.singleton(item);
                       }
               );
       

      The given functions must be stateless and cooperative.

      This operation is subject to memory limits. See InstanceConfig.setMaxProcessorAccumulatedRecords(long) for more information.

      Specified by:
      flatMapStateful in interface GeneralStageWithKey<T,K>
      Type Parameters:
      S - type of the state object
      R - type of the result
      Parameters:
      createFn - function that returns the state object
      flatMapFn - function that receives the state object and the input item and outputs the result items. It may modify the state object. It must not return null traverser, but can return an empty traverser.
    • rollingAggregate

      @Nonnull default <A, R> StreamStage<Map.Entry<K,R>> rollingAggregate(@Nonnull AggregateOperation1<? super T,A,? extends R> aggrOp)
      Description copied from interface: GeneralStageWithKey
      Attaches a rolling aggregation stage. This is a special case of stateful mapping that uses an AggregateOperation. It passes each input item to the accumulator and outputs the current result of aggregation (as returned by the export primitive).

      Sample usage:

      
       StreamStage<Entry<Color, Long>> aggregated = items
               .groupingKey(Item::getColor)
               .rollingAggregate(AggregateOperations.counting());
       
      For example, if your input is {2, 7, 8, -5}, the output will be {2, 9, 17, 12}.

      This stage is fault-tolerant and saves its state to the snapshot.

      This operation is subject to memory limits. See InstanceConfig.setMaxProcessorAccumulatedRecords(long) for more information.

      Specified by:
      rollingAggregate in interface GeneralStageWithKey<T,K>
      R - type of the aggregate operation result
      Parameters:
      aggrOp - the aggregate operation to perform
      Returns:
      the newly attached stage
    • rollingAggregate

      @Nonnull default <A, R> StreamStage<Map.Entry<K,R>> rollingAggregate(long ttl, @Nonnull AggregateOperation1<? super T,A,? extends R> aggrOp)
      Attaches a rolling aggregation stage. This is a special case of stateful mapping that uses an AggregateOperation. It passes each input item to the accumulator and outputs the current result of aggregation (as returned by the export primitive).

      This sample takes a stream of items and gives rolling counts of items of each color:

      
       StreamStage<Entry<Color, Long>> aggregated = items
               .groupingKey(Item::getColor)
               .rollingAggregate(AggregateOperations.counting());
       

      If the given ttl is greater than zero, Jet will consider the accumulator object stale if its time-to-live has expired. The accumulator object has a timestamp attached to it: the top timestamp of any event with the same key seen so far. Upon seeing another event, Jet compares the accumulator timestamp with the current watermark. If it is less than wm - ttl, it discards the accumulator object and creates a new one before processing the event.

      This stage is fault-tolerant and saves its state to the snapshot.

      Type Parameters:
      R - type of the aggregate operation result
      Parameters:
      aggrOp - the aggregate operation to perform
      Returns:
      the newly attached stage
    • mapUsingIMap

      @Nonnull default <V, R> StreamStage<R> mapUsingIMap(@Nonnull String mapName, @Nonnull BiFunctionEx<? super T,? super V,? extends R> mapFn)
      Description copied from interface: GeneralStageWithKey
      Attaches a mapping stage where for each item a lookup in the IMap with the supplied name using the grouping key is performed and the result of the lookup is merged with the item and emitted.

      If the result of the mapping is null, it emits nothing. Therefore this stage can be used to implement filtering semantics as well.

      The mapping logic is equivalent to:

      
       V value = map.get(groupingKey);
       return mapFn.apply(item, value);
       
      Sample usage:
      
       items.groupingKey(Item::getDetailId)
            .mapUsingIMap(
                    "enriching-map",
                    (Item item, ItemDetail detail) -> item.setDetail(detail)
            );
       
      This stage is similar to stageWithoutKey.mapUsingIMap(), but here Jet knows the key and uses it to partition and distribute the input in order to achieve data locality. The value it fetches from the IMap is stored on the cluster member where the processing takes place. However, if the map doesn't use the default partitioning strategy, the data locality will be broken.
      Specified by:
      mapUsingIMap in interface GeneralStageWithKey<T,K>
      Type Parameters:
      V - type of the value in the IMap
      R - type of the output item
      Parameters:
      mapName - name of the IMap
      mapFn - the mapping function. It must be stateless and cooperative.
      Returns:
      the newly attached stage
    • mapUsingIMap

      @Nonnull default <V, R> StreamStage<R> mapUsingIMap(@Nonnull IMap<K,V> iMap, @Nonnull BiFunctionEx<? super T,? super V,? extends R> mapFn)
      Description copied from interface: GeneralStageWithKey
      Attaches a mapping stage where for each item a lookup in the supplied IMap using the grouping key is performed and the result of the lookup is merged with the item and emitted.

      If the result of the mapping is null, it emits nothing. Therefore this stage can be used to implement filtering semantics as well.

      The mapping logic is equivalent to:

      
       V value = map.get(groupingKey);
       return mapFn.apply(item, value);
       
      Sample usage:
      
       items.groupingKey(Item::getDetailId)
            .mapUsingIMap(enrichingMap, (item, detail) -> item.setDetail(detail));
       
      This stage is similar to stageWithoutKey.mapUsingIMap(), but here Jet knows the key and uses it to partition and distribute the input in order to achieve data locality. The value it fetches from the IMap is stored on the cluster member where the processing takes place. However, if the map doesn't use the default partitioning strategy, data locality will be broken.
      Specified by:
      mapUsingIMap in interface GeneralStageWithKey<T,K>
      Type Parameters:
      V - type of the value in the IMap
      R - type of the output item
      Parameters:
      iMap - the IMap to use as the service
      mapFn - the mapping function. It must be stateless and cooperative.
      Returns:
      the newly attached stage
    • mapUsingService

      @Nonnull <S, R> StreamStage<R> mapUsingService(@Nonnull ServiceFactory<?,S> serviceFactory, @Nonnull TriFunction<? super S,? super K,? super T,? extends R> mapFn)
      Description copied from interface: GeneralStageWithKey
      Attaches a mapping stage which applies the given function to each input item independently and emits the function's result as the output item. The mapping function receives another parameter, the service object, which Jet will create using the supplied serviceFactory. If the mapping result is null, it emits nothing. Therefore this stage can be used to implement filtering semantics as well.

      Jet uses the key-extracting function specified on this stage for partitioning: all the items with the same key will see the same service instance (but note that the same instance serves many keys). One case where this is useful is fetching data from an external system because you can use a near-cache without duplicating the cached data.

      Sample usage:

      
       items.groupingKey(Item::getDetailId)
            .mapUsingService(
                ServiceFactories.sharedService(ctx -> new ItemDetailRegistry()),
                (reg, key, item) -> item.setDetail(reg.fetchDetail(key))
            );
       

      Interaction with fault-tolerant unbounded jobs

      If you use this stage in a fault-tolerant unbounded job, keep in mind that any state the service object maintains doesn't participate in Jet's fault tolerance protocol. If the state is local, it will be lost after a job restart; if it is saved to some durable storage, the state of that storage won't be rewound to the last checkpoint, so you'll perform duplicate updates.
      Specified by:
      mapUsingService in interface GeneralStageWithKey<T,K>
      Type Parameters:
      S - type of service object
      R - the result type of the mapping function
      Parameters:
      serviceFactory - the service factory
      mapFn - a mapping function. It must be stateless. It must be cooperative, if the service is cooperative.
      Returns:
      the newly attached stage
    • mapUsingServiceAsync

      @Nonnull default <S, R> StreamStage<R> mapUsingServiceAsync(@Nonnull ServiceFactory<?,S> serviceFactory, @Nonnull TriFunction<? super S,? super K,? super T,CompletableFuture<R>> mapAsyncFn)
      Description copied from interface: GeneralStageWithKey
      Asynchronous version of GeneralStageWithKey.mapUsingService(com.hazelcast.jet.pipeline.ServiceFactory<?, S>, com.hazelcast.jet.function.TriFunction<? super S, ? super K, ? super T, ? extends R>): the mapAsyncFn returns a CompletableFuture<R> instead of just R.

      Uses default values for some extra parameters, so the maximum number of concurrent async operations per processor will be limited to 4 and whether or not the order of input items should be preserved will be true.

      The function can return a null future or the future can return a null result: in both cases it will act just like a filter.

      Sample usage:

      
       items.groupingKey(Item::getDetailId)
            .mapUsingServiceAsync(
                ServiceFactories.sharedService(ctx -> new ItemDetailRegistry()),
                (reg, key, item) -> reg.fetchDetailAsync(key).thenApply(item::setDetail)
            );
       
      The latency of the async call will add to the total latency of the output.
      Specified by:
      mapUsingServiceAsync in interface GeneralStageWithKey<T,K>
      Type Parameters:
      S - type of service object
      R - the future's result type of the mapping function
      Parameters:
      serviceFactory - the service factory
      mapAsyncFn - a mapping function. Can map to null (return a null future). It must be stateless and cooperative.
      Returns:
      the newly attached stage
    • mapUsingServiceAsync

      @Nonnull <S, R> StreamStage<R> mapUsingServiceAsync(@Nonnull ServiceFactory<?,S> serviceFactory, int maxConcurrentOps, boolean preserveOrder, @Nonnull TriFunction<? super S,? super K,? super T,CompletableFuture<R>> mapAsyncFn)
      Description copied from interface: GeneralStageWithKey
      Asynchronous version of GeneralStageWithKey.mapUsingService(com.hazelcast.jet.pipeline.ServiceFactory<?, S>, com.hazelcast.jet.function.TriFunction<? super S, ? super K, ? super T, ? extends R>): the mapAsyncFn returns a CompletableFuture<R> instead of just R.

      The function can return a null future or the future can return a null result: in both cases it will act just like a filter.

      Sample usage:

      
       items.groupingKey(Item::getDetailId)
            .mapUsingServiceAsync(
                ServiceFactories.sharedService(ctx -> new ItemDetailRegistry()),
                16,
                true,
                (reg, key, item) -> reg.fetchDetailAsync(key).thenApply(item::setDetail)
            );
       
      The latency of the async call will add to the total latency of the output.
      Specified by:
      mapUsingServiceAsync in interface GeneralStageWithKey<T,K>
      Type Parameters:
      S - type of service object
      R - the future's result type of the mapping function
      Parameters:
      serviceFactory - the service factory
      maxConcurrentOps - maximum number of concurrent async operations per processor
      preserveOrder - whether the async responses are ordered or not
      mapAsyncFn - a mapping function. Can map to null (return a null future). It must be stateless and cooperative.
      Returns:
      the newly attached stage
    • mapUsingServiceAsyncBatched

      @Nonnull <S, R> StreamStage<R> mapUsingServiceAsyncBatched(@Nonnull ServiceFactory<?,S> serviceFactory, int maxBatchSize, @Nonnull BiFunctionEx<? super S,? super List<T>,? extends CompletableFuture<List<R>>> mapAsyncFn)
      Description copied from interface: GeneralStageWithKey
      Batched version of GeneralStageWithKey.mapUsingServiceAsync(com.hazelcast.jet.pipeline.ServiceFactory<?, S>, com.hazelcast.jet.function.TriFunction<? super S, ? super K, ? super T, java.util.concurrent.CompletableFuture<R>>): mapAsyncFn takes a list of input items and returns a CompletableFuture<List<R>>. The size of list is limited by the given maxBatchSize.

      This transform can perform filtering by putting null elements into the output list.

      The latency of the async call will add to the total latency of the output.

      This sample takes a stream of stock items and sets the detail field on them by performing batched lookups from a registry. The max size of the items to lookup is specified as 100:

      
       items.groupingKey(Item::getDetailId)
            .mapUsingServiceAsyncBatched(
                ServiceFactories.sharedService(ctx -> new ItemDetailRegistry()),
                100,
                (reg, itemList) -> reg
                    .fetchDetailsAsync(itemList.stream().map(Item::getDetailId).collect(Collectors.toList()))
                    .thenApply(details -> {
                        for (int i = 0; i < itemList.size(); i++) {
                            itemList.get(i).setDetail(details.get(i));
                        }
                        return itemList;
                    })
            );
       
      Specified by:
      mapUsingServiceAsyncBatched in interface GeneralStageWithKey<T,K>
      Type Parameters:
      S - type of service object
      R - the future result type of the mapping function
      Parameters:
      serviceFactory - the service factory
      maxBatchSize - max size of the input list
      mapAsyncFn - a mapping function. It must be stateless. It must be cooperative, if the service is cooperative.
      Returns:
      the newly attached stage
    • mapUsingServiceAsyncBatched

      @Nonnull <S, R> StreamStage<R> mapUsingServiceAsyncBatched(@Nonnull ServiceFactory<?,S> serviceFactory, int maxBatchSize, @Nonnull TriFunction<? super S,? super List<K>,? super List<T>,? extends CompletableFuture<List<R>>> mapAsyncFn)
      Description copied from interface: GeneralStageWithKey
      Batched version of GeneralStageWithKey.mapUsingServiceAsync(com.hazelcast.jet.pipeline.ServiceFactory<?, S>, com.hazelcast.jet.function.TriFunction<? super S, ? super K, ? super T, java.util.concurrent.CompletableFuture<R>>): mapAsyncFn takes a list of input items (and a list of their corresponding keys) and returns a CompletableFuture<List<R>>. The sizes of the input lists are identical and are limited by the given maxBatchSize. The key at index N corresponds to the input item at index N.

      The number of in-flight batches being completed asynchronously is limited to 2 and this mapping operation always preserves the order of input elements.

      This transform can perform filtering by putting null elements into the output list.

      The latency of the async call will add to the total latency of the output.

      This sample takes a stream of stock items and sets the detail field on them by performing batched lookups from a registry. The max size of the items to lookup is specified as 100:

      
       items.groupingKey(Item::getDetailId)
            .mapUsingServiceAsyncBatched(
                ServiceFactories.sharedService(ctx -> new ItemDetailRegistry()),
                100,
                (reg, keyList, itemList) -> reg.fetchDetailsAsync(keyList).thenApply(details -> {
                    for (int i = 0; i < itemList.size(); i++) {
                        itemList.get(i).setDetail(details.get(i));
                    }
                    return itemList;
                })
            );
       
      Specified by:
      mapUsingServiceAsyncBatched in interface GeneralStageWithKey<T,K>
      Type Parameters:
      S - type of service object
      R - the future result type of the mapping function
      Parameters:
      serviceFactory - the service factory
      maxBatchSize - max size of the input list
      mapAsyncFn - a mapping function. It must be stateless and cooperative.
      Returns:
      the newly attached stage
    • filterUsingService

      @Nonnull <S> StreamStage<T> filterUsingService(@Nonnull ServiceFactory<?,S> serviceFactory, @Nonnull TriPredicate<? super S,? super K,? super T> filterFn)
      Description copied from interface: GeneralStageWithKey
      Attaches a filtering stage which applies the provided predicate function to each input item to decide whether to pass the item to the output or to discard it. The predicate function receives another parameter, the service object, which Jet will create using the supplied serviceFactory.

      The number of in-flight batches being completed asynchronously is limited to 2 and this mapping operation always preserves the order of input elements.

      Jet uses the key-extracting function specified on this stage for partitioning: all the items with the same key will see the same service instance (but note that the same instance serves many keys). One case where this is useful is fetching data from an external system because you can use a near-cache without duplicating the cached data.

      Sample usage:

      
       items.groupingKey(Item::getDetailId)
            .filterUsingService(
                ServiceFactories.sharedService(ctx -> new ItemDetailRegistry()),
                (reg, key, item) -> reg.fetchDetail(key).contains("blade")
            );
       

      Interaction with fault-tolerant unbounded jobs

      If you use this stage in a fault-tolerant unbounded job, keep in mind that any state the service object maintains doesn't participate in Jet's fault tolerance protocol. If the state is local, it will be lost after a job restart; if it is saved to some durable storage, the state of that storage won't be rewound to the last checkpoint, so you'll perform duplicate updates.
      Specified by:
      filterUsingService in interface GeneralStageWithKey<T,K>
      Type Parameters:
      S - type of service object
      Parameters:
      serviceFactory - the service factory
      filterFn - a filter predicate function. It must be stateless. It must be cooperative, if the service is cooperative.
      Returns:
      the newly attached stage
    • flatMapUsingService

      @Nonnull <S, R> StreamStage<R> flatMapUsingService(@Nonnull ServiceFactory<?,S> serviceFactory, @Nonnull TriFunction<? super S,? super K,? super T,? extends Traverser<R>> flatMapFn)
      Description copied from interface: GeneralStageWithKey
      Attaches a flat-mapping stage which applies the supplied function to each input item independently and emits all the items from the Traverser it returns as the output items. The traverser must be null-terminated. The mapping function receives another parameter, the service object, which Jet will create using the supplied serviceFactory.

      Jet uses the key-extracting function specified on this stage for partitioning: all the items with the same key will see the same service instance (but note that the same instance serves many keys). One case where this is useful is fetching data from an external system because you can use a near-cache without duplicating the cached data.

      Sample usage:

      
       StreamStage<Part> parts = products
           .groupingKey(Product::getId)
           .flatMapUsingService(
               ServiceFactories.sharedService(ctx -> new PartRegistry()),
               (registry, productId, product) -> Traversers.traverseIterable(
                       registry.fetchParts(productId))
           );
       

      Interaction with fault-tolerant unbounded jobs

      If you use this stage in a fault-tolerant unbounded job, keep in mind that any state the service object maintains doesn't participate in Jet's fault tolerance protocol. If the state is local, it will be lost after a job restart; if it is saved to some durable storage, the state of that storage won't be rewound to the last checkpoint, so you'll perform duplicate updates.
      Specified by:
      flatMapUsingService in interface GeneralStageWithKey<T,K>
      Type Parameters:
      S - type of service object
      R - type of the output items
      Parameters:
      serviceFactory - the service factory
      flatMapFn - a flatmapping function. It must not return null traverser, but can return an empty traverser. It must be stateless. It must be cooperative, if the service is cooperative.
      Returns:
      the newly attached stage
    • customTransform

      @Nonnull default <R> StreamStage<R> customTransform(@Nonnull String stageName, @Nonnull SupplierEx<Processor> procSupplier)
      Description copied from interface: GeneralStageWithKey
      Attaches a stage with a custom transform based on the provided supplier of Core API Processors. The inbound edge will be distributed and partitioned using the key function assigned to this stage.

      Note that the type parameter of the returned stage is inferred from the call site and not propagated from the processor that will produce the result, so there is no actual type safety provided.

      Specified by:
      customTransform in interface GeneralStageWithKey<T,K>
      Type Parameters:
      R - the type of the output items
      Parameters:
      stageName - a human-readable name for the custom stage
      procSupplier - the supplier of processors
    • customTransform

      @Nonnull default <R> StreamStage<R> customTransform(@Nonnull String stageName, @Nonnull ProcessorSupplier procSupplier)
      Description copied from interface: GeneralStageWithKey
      Attaches a stage with a custom transform based on the provided supplier of Core API Processors. The inbound edge will be distributed and partitioned using the key function assigned to this stage.

      Note that the type parameter of the returned stage is inferred from the call site and not propagated from the processor that will produce the result, so there is no actual type safety provided.

      Specified by:
      customTransform in interface GeneralStageWithKey<T,K>
      Type Parameters:
      R - the type of the output items
      Parameters:
      stageName - a human-readable name for the custom stage
      procSupplier - the supplier of processors
    • customTransform

      @Nonnull <R> StreamStage<R> customTransform(@Nonnull String stageName, @Nonnull ProcessorMetaSupplier procSupplier)
      Description copied from interface: GeneralStageWithKey
      Attaches a stage with a custom transform based on the provided supplier of Core API Processors. The inbound edge will be distributed and partitioned using the key function assigned to this stage.

      Note that the type parameter of the returned stage is inferred from the call site and not propagated from the processor that will produce the result, so there is no actual type safety provided.

      Specified by:
      customTransform in interface GeneralStageWithKey<T,K>
      Type Parameters:
      R - the type of the output items
      Parameters:
      stageName - a human-readable name for the custom stage
      procSupplier - the supplier of processors