Class Stream

java.lang.Object
org.apache.storm.trident.Stream
All Implemented Interfaces:
ResourceDeclarer<Stream>, IAggregatableStream

public class Stream extends Object implements IAggregatableStream, ResourceDeclarer<Stream>
A Stream represents the core data model in Trident, and can be thought of as a "stream" of tuples that are processed as a series of small batches. A stream is partitioned accross the nodes in the cluster, and operations are applied to a stream in parallel accross each partition.

There are five types of operations that can be performed on streams in Trident

1. **Partiton-Local Operations** - Operations that are applied locally to each partition and do not involve network transfer 2. **Repartitioning Operations** - Operations that change how tuples are partitioned across tasks(thus causing network transfer), but do not change the content of the stream. 3. **Aggregation Operations** - Operations that *may* repartition a stream (thus causing network transfer) 4. **Grouping Operations** - Operations that may repartition a stream on specific fields and group together tuples whose fields values are equal. 5. **Merge and Join Operations** - Operations that combine different streams together.

  • Constructor Details

  • Method Details

    • name

      public Stream name(String name)
      Applies a label to the stream. Naming a stream will append the label to the name of the bolt(s) created by Trident and will be visible in the Storm UI.
      Parameters:
      name - - The label to apply to the stream
    • parallelismHint

      public Stream parallelismHint(int hint)
      Applies a parallelism hint to a stream.
    • setCPULoad

      public Stream setCPULoad(Number load)
      Sets the CPU Load resource for the current operation.
      Specified by:
      setCPULoad in interface ResourceDeclarer<Stream>
      Parameters:
      load - the amount of CPU
      Returns:
      this for chaining
    • setMemoryLoad

      public Stream setMemoryLoad(Number onHeap)
      Sets the Memory Load resources for the current operation. offHeap becomes default.
      Specified by:
      setMemoryLoad in interface ResourceDeclarer<Stream>
      Parameters:
      onHeap - the amount of on heap memory
      Returns:
      this for chaining
    • setMemoryLoad

      public Stream setMemoryLoad(Number onHeap, Number offHeap)
      Sets the Memory Load resources for the current operation.
      Specified by:
      setMemoryLoad in interface ResourceDeclarer<Stream>
      Parameters:
      onHeap - the amount of on heap memory
      offHeap - the amount of off heap memory
      Returns:
      this for chaining
    • addSharedMemory

      public Stream addSharedMemory(SharedMemory request)
      Description copied from interface: ResourceDeclarer
      Add in request for shared memory that this component will use. See SharedOnHeap, SharedOffHeapWithinNode, and SharedOffHeapWithinWorker for convenient ways to create shared memory requests.
      Specified by:
      addSharedMemory in interface ResourceDeclarer<Stream>
      Parameters:
      request - the shared memory request for this component
      Returns:
      this for chaining
    • project

      public Stream project(Fields keepFields)
      Filters out fields from a stream, resulting in a Stream containing only the fields specified by `keepFields`.

      For example, if you had a Stream `mystream` containing the fields `["a", "b", "c","d"]`, calling"

      ```java mystream.project(new Fields("b", "d")) ```

      would produce a stream containing only the fields `["b", "d"]`.

      Parameters:
      keepFields - The fields in the Stream to keep
    • groupBy

      public GroupedStream groupBy(Fields fields)
      ## Grouping Operation.
    • partitionBy

      public Stream partitionBy(Fields fields)
      ## Repartitioning Operation.
    • partition

      public Stream partition(CustomStreamGrouping partitioner)
      ## Repartitioning Operation.
    • partition

      public Stream partition(Grouping grouping)
      ## Repartitioning Operation.

      This method takes in a custom partitioning function that implements CustomStreamGrouping

    • shuffle

      public Stream shuffle()
      ## Repartitioning Operation.

      Use random round robin algorithm to evenly redistribute tuples across all target partitions.

    • localOrShuffle

      public Stream localOrShuffle()
      ## Repartitioning Operation.

      Use random round robin algorithm to evenly redistribute tuples across all target partitions, with a preference for local tasks.

    • global

      public Stream global()
      ## Repartitioning Operation.

      All tuples are sent to the same partition. The same partition is chosen for all batches in the stream.

    • batchGlobal

      public Stream batchGlobal()
      ## Repartitioning Operation.

      All tuples in the batch are sent to the same partition. Different batches in the stream may go to different partitions.

    • broadcast

      public Stream broadcast()
      ## Repartitioning Operation.

      Every tuple is replicated to all target partitions. This can useful during DRPC – for example, if you need to do a stateQuery on every partition of data.

    • identityPartition

      public Stream identityPartition()
      ## Repartitioning Operation.
    • applyAssembly

      public Stream applyAssembly(Assembly assembly)
      Applies an `Assembly` to this `Stream`.
      See Also:
    • each

      public Stream each(Fields inputFields, Function function, Fields functionFields)
      Specified by:
      each in interface IAggregatableStream
    • each

      public Stream each(Function function, Fields functionFields)
    • each

      public Stream each(Fields inputFields, Filter filter)
    • partitionAggregate

      public Stream partitionAggregate(Fields inputFields, Aggregator agg, Fields functionFields)
      Specified by:
      partitionAggregate in interface IAggregatableStream
    • partitionAggregate

      public Stream partitionAggregate(Aggregator agg, Fields functionFields)
    • partitionAggregate

      public Stream partitionAggregate(CombinerAggregator agg, Fields functionFields)
    • partitionAggregate

      public Stream partitionAggregate(Fields inputFields, CombinerAggregator agg, Fields functionFields)
    • partitionAggregate

      public Stream partitionAggregate(ReducerAggregator agg, Fields functionFields)
    • partitionAggregate

      public Stream partitionAggregate(Fields inputFields, ReducerAggregator agg, Fields functionFields)
    • stateQuery

      public Stream stateQuery(TridentState state, Fields inputFields, QueryFunction function, Fields functionFields)
    • stateQuery

      public Stream stateQuery(TridentState state, QueryFunction function, Fields functionFields)
    • partitionPersist

      public TridentState partitionPersist(StateFactory stateFactory, Fields inputFields, StateUpdater updater, Fields functionFields)
    • partitionPersist

      public TridentState partitionPersist(StateSpec stateSpec, Fields inputFields, StateUpdater updater, Fields functionFields)
    • partitionPersist

      public TridentState partitionPersist(StateFactory stateFactory, Fields inputFields, StateUpdater updater)
    • partitionPersist

      public TridentState partitionPersist(StateSpec stateSpec, Fields inputFields, StateUpdater updater)
    • partitionPersist

      public TridentState partitionPersist(StateFactory stateFactory, StateUpdater updater, Fields functionFields)
    • partitionPersist

      public TridentState partitionPersist(StateSpec stateSpec, StateUpdater updater, Fields functionFields)
    • partitionPersist

      public TridentState partitionPersist(StateFactory stateFactory, StateUpdater updater)
    • partitionPersist

      public TridentState partitionPersist(StateSpec stateSpec, StateUpdater updater)
    • filter

      public Stream filter(Filter filter)
      Returns a stream consisting of the elements of this stream that match the given filter.
      Parameters:
      filter - the filter to apply to each trident tuple to determine if it should be included.
      Returns:
      the new stream
    • filter

      public Stream filter(Fields inputFields, Filter filter)
      Returns a stream consisting of the elements of this stream that match the given filter.
      Parameters:
      inputFields - the fields of the input trident tuple to be selected.
      filter - the filter to apply to each trident tuple to determine if it should be included.
      Returns:
      the new stream
    • map

      public Stream map(MapFunction function)
      Returns a stream consisting of the result of applying the given mapping function to the values of this stream.
      Parameters:
      function - a mapping function to be applied to each value in this stream.
      Returns:
      the new stream
    • map

      public Stream map(MapFunction function, Fields outputFields)
      Returns a stream consisting of the result of applying the given mapping function to the values of this stream. This method replaces old output fields with new output fields, achieving T -> V conversion.
      Parameters:
      function - a mapping function to be applied to each value in this stream.
      outputFields - new output fields
      Returns:
      the new stream
    • flatMap

      public Stream flatMap(FlatMapFunction function)
      Returns a stream consisting of the results of replacing each value of this stream with the contents produced by applying the provided mapping function to each value. This has the effect of applying a one-to-many transformation to the values of the stream, and then flattening the resulting elements into a new stream.
      Parameters:
      function - a mapping function to be applied to each value in this stream which produces new values.
      Returns:
      the new stream
    • flatMap

      public Stream flatMap(FlatMapFunction function, Fields outputFields)
      Returns a stream consisting of the results of replacing each value of this stream with the contents produced by applying the provided mapping function to each value. This has the effect of applying a one-to-many transformation to the values of the stream, and then flattening the resulting elements into a new stream. This method replaces old output fields with new output fields, achieving T -> V conversion.
      Parameters:
      function - a mapping function to be applied to each value in this stream which produces new values.
      outputFields - new output fields
      Returns:
      the new stream
    • peek

      public Stream peek(Consumer action)
      Returns a stream consisting of the trident tuples of this stream, additionally performing the provided action on each trident tuple as they are consumed from the resulting stream. This is mostly useful for debugging to see the tuples as they flow past a certain point in a pipeline.
      Parameters:
      action - the action to perform on the trident tuple as they are consumed from the stream
      Returns:
      the new stream
    • chainedAgg

      public ChainedAggregatorDeclarer chainedAgg()
    • minBy

      public Stream minBy(String inputFieldName)
      This aggregator operation computes the minimum of tuples by the given inputFieldName and it is assumed that its value is an instance of Comparable. If the value of tuple with field inputFieldName is not an instance of Comparable then it throws ClassCastException
      Parameters:
      inputFieldName - input field name
      Returns:
      the new stream with this operation.
    • minBy

      public <T> Stream minBy(String inputFieldName, Comparator<T> comparator)
      This aggregator operation computes the minimum of tuples by the given inputFieldName in a stream by using the given comparator. If the value of tuple with field inputFieldName is not an instance of T then it throws ClassCastException
      Parameters:
      inputFieldName - input field name
      comparator - comparator used in for finding minimum of two tuple values of inputFieldName.
      Returns:
      the new stream with this operation.
    • min

      public Stream min(Comparator<TridentTuple> comparator)
      This aggregator operation computes the minimum of tuples in a stream by using the given comparator with TridentTuples.
      Parameters:
      comparator - comparator used in for finding minimum of two tuple values.
      Returns:
      the new stream with this operation.
    • maxBy

      public Stream maxBy(String inputFieldName)
      This aggregator operation computes the maximum of tuples by the given inputFieldName and it is assumed that its value is an instance of Comparable. If the value of tuple with field inputFieldName is not an instance of Comparable then it throws ClassCastException
      Parameters:
      inputFieldName - input field name
      Returns:
      the new stream with this operation.
    • maxBy

      public <T> Stream maxBy(String inputFieldName, Comparator<T> comparator)
      This aggregator operation computes the maximum of tuples by the given inputFieldName in a stream by using the given comparator. If the value of tuple with field inputFieldName is not an instance of T then it throws ClassCastException
      Parameters:
      inputFieldName - input field name
      comparator - comparator used in for finding maximum of two tuple values of inputFieldName.
      Returns:
      the new stream with this operation.
    • max

      public Stream max(Comparator<TridentTuple> comparator)
      This aggregator operation computes the maximum of tuples in a stream by using the given comparator with TridentTuples.
      Parameters:
      comparator - comparator used in for finding maximum of two tuple values.
      Returns:
      the new stream with this operation.
    • aggregate

      public Stream aggregate(Aggregator agg, Fields functionFields)
    • aggregate

      public Stream aggregate(Fields inputFields, Aggregator agg, Fields functionFields)
    • aggregate

      public Stream aggregate(CombinerAggregator agg, Fields functionFields)
    • aggregate

      public Stream aggregate(Fields inputFields, CombinerAggregator agg, Fields functionFields)
    • aggregate

      public Stream aggregate(ReducerAggregator agg, Fields functionFields)
    • aggregate

      public Stream aggregate(Fields inputFields, ReducerAggregator agg, Fields functionFields)
    • tumblingWindow

      public Stream tumblingWindow(int windowCount, WindowsStoreFactory windowStoreFactory, Fields inputFields, Aggregator aggregator, Fields functionFields)
      Returns a stream of tuples which are aggregated results of a tumbling window with every windowCount of tuples.
      Parameters:
      windowCount - represents number of tuples in the window
      windowStoreFactory - intermediary tuple store for storing windowing tuples
      inputFields - projected fields for aggregator
      aggregator - aggregator to run on the window of tuples to compute the result and emit to the stream.
      functionFields - fields of values to emit with aggregation.
      Returns:
      the new stream with this operation.
    • tumblingWindow

      public Stream tumblingWindow(BaseWindowedBolt.Duration windowDuration, WindowsStoreFactory windowStoreFactory, Fields inputFields, Aggregator aggregator, Fields functionFields)
      Returns a stream of tuples which are aggregated results of a window that tumbles at duration of windowDuration.
      Parameters:
      windowDuration - represents tumbling window duration configuration
      windowStoreFactory - intermediary tuple store for storing windowing tuples
      inputFields - projected fields for aggregator
      aggregator - aggregator to run on the window of tuples to compute the result and emit to the stream.
      functionFields - fields of values to emit with aggregation.
      Returns:
      the new stream with this operation.
    • slidingWindow

      public Stream slidingWindow(int windowCount, int slideCount, WindowsStoreFactory windowStoreFactory, Fields inputFields, Aggregator aggregator, Fields functionFields)
      Returns a stream of tuples which are aggregated results of a sliding window with every windowCount of tuples and slides the window after slideCount.
      Parameters:
      windowCount - represents tuples count of a window
      slideCount - the number of tuples after which the window slides
      windowStoreFactory - intermediary tuple store for storing windowing tuples
      inputFields - projected fields for aggregator
      aggregator - aggregator to run on the window of tuples to compute the result and emit to the stream.
      functionFields - fields of values to emit with aggregation.
      Returns:
      the new stream with this operation.
    • slidingWindow

      public Stream slidingWindow(BaseWindowedBolt.Duration windowDuration, BaseWindowedBolt.Duration slidingInterval, WindowsStoreFactory windowStoreFactory, Fields inputFields, Aggregator aggregator, Fields functionFields)
      Returns a stream of tuples which are aggregated results of a window which slides at duration of slidingInterval and completes a window at windowDuration.
      Parameters:
      windowDuration - represents window duration configuration
      slidingInterval - the time duration after which the window slides
      windowStoreFactory - intermediary tuple store for storing windowing tuples
      inputFields - projected fields for aggregator
      aggregator - aggregator to run on the window of tuples to compute the result and emit to the stream.
      functionFields - fields of values to emit with aggregation.
      Returns:
      the new stream with this operation.
    • window

      public Stream window(WindowConfig windowConfig, Fields inputFields, Aggregator aggregator, Fields functionFields)
      Returns a stream of aggregated results based on the given window configuration which uses inmemory windowing tuple store.
      Parameters:
      windowConfig - window configuration like window length and slide length.
      inputFields - input fields
      aggregator - aggregator to run on the window of tuples to compute the result and emit to the stream.
      functionFields - fields of values to emit with aggregation.
      Returns:
      the new stream with this operation.
    • window

      public Stream window(WindowConfig windowConfig, WindowsStoreFactory windowStoreFactory, Fields inputFields, Aggregator aggregator, Fields functionFields)
      Returns stream of aggregated results based on the given window configuration.
      Parameters:
      windowConfig - window configuration like window length and slide length.
      windowStoreFactory - intermediary tuple store for storing tuples for windowing
      inputFields - input fields
      aggregator - aggregator to run on the window of tuples to compute the result and emit to the stream.
      functionFields - fields of values to emit with aggregation.
      Returns:
      the new stream with this operation.
    • persistentAggregate

      public TridentState persistentAggregate(StateFactory stateFactory, CombinerAggregator agg, Fields functionFields)
    • persistentAggregate

      public TridentState persistentAggregate(StateSpec spec, CombinerAggregator agg, Fields functionFields)
    • persistentAggregate

      public TridentState persistentAggregate(StateFactory stateFactory, Fields inputFields, CombinerAggregator agg, Fields functionFields)
    • persistentAggregate

      public TridentState persistentAggregate(StateSpec spec, Fields inputFields, CombinerAggregator agg, Fields functionFields)
    • persistentAggregate

      public TridentState persistentAggregate(StateFactory stateFactory, ReducerAggregator agg, Fields functionFields)
    • persistentAggregate

      public TridentState persistentAggregate(StateSpec spec, ReducerAggregator agg, Fields functionFields)
    • persistentAggregate

      public TridentState persistentAggregate(StateFactory stateFactory, Fields inputFields, ReducerAggregator agg, Fields functionFields)
    • persistentAggregate

      public TridentState persistentAggregate(StateSpec spec, Fields inputFields, ReducerAggregator agg, Fields functionFields)
    • toStream

      public Stream toStream()
      Specified by:
      toStream in interface IAggregatableStream
    • getName

      public String getName()
      Returns the label applied to the stream.
      Returns:
      the label applied to the stream.
    • getOutputFields

      public Fields getOutputFields()
      Specified by:
      getOutputFields in interface IAggregatableStream