Class DStream<T>

Object
org.apache.spark.streaming.dstream.DStream<T>
All Implemented Interfaces:
Serializable, org.apache.spark.internal.Logging
Direct Known Subclasses:
InputDStream, MapWithStateDStream

public abstract class DStream<T> extends Object implements Serializable, org.apache.spark.internal.Logging
A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous sequence of RDDs (of the same type) representing a continuous stream of data (see org.apache.spark.rdd.RDD in the Spark core documentation for more details on RDDs). DStreams can either be created from live data (such as, data from TCP sockets, Kafka, etc.) using a StreamingContext or it can be generated by transforming existing DStreams using operations such as map, window and reduceByKeyAndWindow. While a Spark Streaming program is running, each DStream periodically generates a RDD, either from live data or by transforming the RDD generated by a parent DStream.

This class contains the basic operations available on all DStreams, such as map, filter and window. In addition, PairDStreamFunctions contains operations available only on DStreams of key-value pairs, such as groupByKeyAndWindow and join. These operations are automatically available on any DStream of pairs (e.g., DStream[(Int, Int)] through implicit conversions.

A DStream internally is characterized by a few basic properties: - A list of other DStreams that the DStream depends on - A time interval at which the DStream generates an RDD - A function that is used to generate an RDD after each time interval

See Also:
  • Nested Class Summary

    Nested classes/interfaces inherited from interface org.apache.spark.internal.Logging

    org.apache.spark.internal.Logging.LogStringContext, org.apache.spark.internal.Logging.SparkShellLoggingFilter
  • Constructor Summary

    Constructors
    Constructor
    Description
    DStream(StreamingContext ssc, scala.reflect.ClassTag<T> evidence$1)
     
  • Method Summary

    Modifier and Type
    Method
    Description
    Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER)
    checkpoint(Duration interval)
    Enable periodic checkpointing of RDDs of this DStream
    abstract scala.Option<RDD<T>>
    compute(Time validTime)
    Method that generates an RDD for the given time
    Return the StreamingContext associated with this DStream
    Return a new DStream in which each RDD has a single element generated by counting each RDD of this DStream.
    DStream<scala.Tuple2<T,Object>>
    countByValue(int numPartitions, scala.math.Ordering<T> ord)
    Return a new DStream in which each RDD contains the counts of each distinct value in each RDD of this DStream.
    DStream<scala.Tuple2<T,Object>>
    countByValueAndWindow(Duration windowDuration, Duration slideDuration, int numPartitions, scala.math.Ordering<T> ord)
    Return a new DStream in which each RDD contains the count of distinct elements in RDDs in a sliding window over this DStream.
    countByWindow(Duration windowDuration, Duration slideDuration)
    Return a new DStream in which each RDD has a single element generated by counting the number of elements in a sliding window over this DStream.
    abstract scala.collection.immutable.List<DStream<?>>
    List of parent DStreams on which this DStream depends on
    filter(scala.Function1<T,Object> filterFunc)
    Return a new DStream containing only the elements that satisfy a predicate.
    <U> DStream<U>
    flatMap(scala.Function1<T,scala.collection.IterableOnce<U>> flatMapFunc, scala.reflect.ClassTag<U> evidence$3)
    Return a new DStream by applying a function to all elements of this DStream, and then flattening the results
    void
    foreachRDD(scala.Function1<RDD<T>,scala.runtime.BoxedUnit> foreachFunc)
    Apply a function to each RDD in this DStream.
    void
    foreachRDD(scala.Function2<RDD<T>,Time,scala.runtime.BoxedUnit> foreachFunc)
    Apply a function to each RDD in this DStream.
    Return a new DStream in which each RDD is generated by applying glom() to each RDD of this DStream.
    <U> DStream<U>
    map(scala.Function1<T,U> mapFunc, scala.reflect.ClassTag<U> evidence$2)
    Return a new DStream by applying a function to all elements of this DStream.
    <U> DStream<U>
    mapPartitions(scala.Function1<scala.collection.Iterator<T>,scala.collection.Iterator<U>> mapPartFunc, boolean preservePartitioning, scala.reflect.ClassTag<U> evidence$4)
    Return a new DStream in which each RDD is generated by applying mapPartitions() to each RDDs of this DStream.
    Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER)
    Persist the RDDs of this DStream with the given storage level
    void
    Print the first ten elements of each RDD generated in this DStream.
    void
    print(int num)
    Print the first num elements of each RDD generated in this DStream.
    reduce(scala.Function2<T,T,T> reduceFunc)
    Return a new DStream in which each RDD has a single element generated by reducing each RDD of this DStream.
    reduceByWindow(scala.Function2<T,T,T> reduceFunc, Duration windowDuration, Duration slideDuration)
    Return a new DStream in which each RDD has a single element generated by reducing all elements in a sliding window over this DStream.
    reduceByWindow(scala.Function2<T,T,T> reduceFunc, scala.Function2<T,T,T> invReduceFunc, Duration windowDuration, Duration slideDuration)
    Return a new DStream in which each RDD has a single element generated by reducing all elements in a sliding window over this DStream.
    repartition(int numPartitions)
    Return a new DStream with an increased or decreased level of parallelism.
    void
    saveAsObjectFiles(String prefix, String suffix)
    Save each RDD in this DStream as a Sequence file of serialized objects.
    void
    saveAsTextFiles(String prefix, String suffix)
    Save each RDD in this DStream as at text file, using string representation of elements.
    scala.collection.immutable.Seq<RDD<T>>
    slice(org.apache.spark.streaming.Interval interval)
    Return all the RDDs defined by the Interval object (both end times included)
    scala.collection.immutable.Seq<RDD<T>>
    slice(Time fromTime, Time toTime)
    Return all the RDDs between 'fromTime' to 'toTime' (both included)
    abstract Duration
    Time interval after which the DStream generates an RDD
    static <K, V> PairDStreamFunctions<K,V>
    toPairDStreamFunctions(DStream<scala.Tuple2<K,V>> stream, scala.reflect.ClassTag<K> kt, scala.reflect.ClassTag<V> vt, scala.math.Ordering<K> ord)
     
    <U> DStream<U>
    transform(scala.Function1<RDD<T>,RDD<U>> transformFunc, scala.reflect.ClassTag<U> evidence$5)
    Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream.
    <U> DStream<U>
    transform(scala.Function2<RDD<T>,Time,RDD<U>> transformFunc, scala.reflect.ClassTag<U> evidence$6)
    Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream.
    <U, V> DStream<V>
    transformWith(DStream<U> other, scala.Function2<RDD<T>,RDD<U>,RDD<V>> transformFunc, scala.reflect.ClassTag<U> evidence$7, scala.reflect.ClassTag<V> evidence$8)
    Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream and 'other' DStream.
    <U, V> DStream<V>
    transformWith(DStream<U> other, scala.Function3<RDD<T>,RDD<U>,Time,RDD<V>> transformFunc, scala.reflect.ClassTag<U> evidence$9, scala.reflect.ClassTag<V> evidence$10)
    Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream and 'other' DStream.
    union(DStream<T> that)
    Return a new DStream by unifying data of another DStream with this DStream.
    window(Duration windowDuration)
    Return a new DStream in which each RDD contains all the elements in seen in a sliding window of time over this DStream.
    window(Duration windowDuration, Duration slideDuration)
    Return a new DStream in which each RDD contains all the elements in seen in a sliding window of time over this DStream.

    Methods inherited from class java.lang.Object

    equals, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait

    Methods inherited from interface org.apache.spark.internal.Logging

    initializeForcefully, initializeLogIfNecessary, initializeLogIfNecessary, initializeLogIfNecessary$default$2, isTraceEnabled, log, logDebug, logDebug, logDebug, logDebug, logError, logError, logError, logError, logInfo, logInfo, logInfo, logInfo, logName, LogStringContext, logTrace, logTrace, logTrace, logTrace, logWarning, logWarning, logWarning, logWarning, org$apache$spark$internal$Logging$$log_, org$apache$spark$internal$Logging$$log__$eq, withLogContext
  • Constructor Details

    • DStream

      public DStream(StreamingContext ssc, scala.reflect.ClassTag<T> evidence$1)
  • Method Details

    • toPairDStreamFunctions

      public static <K, V> PairDStreamFunctions<K,V> toPairDStreamFunctions(DStream<scala.Tuple2<K,V>> stream, scala.reflect.ClassTag<K> kt, scala.reflect.ClassTag<V> vt, scala.math.Ordering<K> ord)
    • slideDuration

      public abstract Duration slideDuration()
      Time interval after which the DStream generates an RDD
    • dependencies

      public abstract scala.collection.immutable.List<DStream<?>> dependencies()
      List of parent DStreams on which this DStream depends on
    • compute

      public abstract scala.Option<RDD<T>> compute(Time validTime)
      Method that generates an RDD for the given time
    • context

      public StreamingContext context()
      Return the StreamingContext associated with this DStream
    • persist

      public DStream<T> persist(StorageLevel level)
      Persist the RDDs of this DStream with the given storage level
    • persist

      public DStream<T> persist()
      Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER)
    • cache

      public DStream<T> cache()
      Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER)
    • checkpoint

      public DStream<T> checkpoint(Duration interval)
      Enable periodic checkpointing of RDDs of this DStream
      Parameters:
      interval - Time interval after which generated RDD will be checkpointed
      Returns:
      (undocumented)
    • map

      public <U> DStream<U> map(scala.Function1<T,U> mapFunc, scala.reflect.ClassTag<U> evidence$2)
      Return a new DStream by applying a function to all elements of this DStream.
    • flatMap

      public <U> DStream<U> flatMap(scala.Function1<T,scala.collection.IterableOnce<U>> flatMapFunc, scala.reflect.ClassTag<U> evidence$3)
      Return a new DStream by applying a function to all elements of this DStream, and then flattening the results
      Parameters:
      flatMapFunc - (undocumented)
      evidence$3 - (undocumented)
      Returns:
      (undocumented)
    • filter

      public DStream<T> filter(scala.Function1<T,Object> filterFunc)
      Return a new DStream containing only the elements that satisfy a predicate.
    • glom

      public DStream<Object> glom()
      Return a new DStream in which each RDD is generated by applying glom() to each RDD of this DStream. Applying glom() to an RDD coalesces all elements within each partition into an array.
      Returns:
      (undocumented)
    • repartition

      public DStream<T> repartition(int numPartitions)
      Return a new DStream with an increased or decreased level of parallelism. Each RDD in the returned DStream has exactly numPartitions partitions.
      Parameters:
      numPartitions - (undocumented)
      Returns:
      (undocumented)
    • mapPartitions

      public <U> DStream<U> mapPartitions(scala.Function1<scala.collection.Iterator<T>,scala.collection.Iterator<U>> mapPartFunc, boolean preservePartitioning, scala.reflect.ClassTag<U> evidence$4)
      Return a new DStream in which each RDD is generated by applying mapPartitions() to each RDDs of this DStream. Applying mapPartitions() to an RDD applies a function to each partition of the RDD.
      Parameters:
      mapPartFunc - (undocumented)
      preservePartitioning - (undocumented)
      evidence$4 - (undocumented)
      Returns:
      (undocumented)
    • reduce

      public DStream<T> reduce(scala.Function2<T,T,T> reduceFunc)
      Return a new DStream in which each RDD has a single element generated by reducing each RDD of this DStream.
      Parameters:
      reduceFunc - (undocumented)
      Returns:
      (undocumented)
    • count

      public DStream<Object> count()
      Return a new DStream in which each RDD has a single element generated by counting each RDD of this DStream.
      Returns:
      (undocumented)
    • countByValue

      public DStream<scala.Tuple2<T,Object>> countByValue(int numPartitions, scala.math.Ordering<T> ord)
      Return a new DStream in which each RDD contains the counts of each distinct value in each RDD of this DStream. Hash partitioning is used to generate the RDDs with numPartitions partitions (Spark's default number of partitions if numPartitions not specified).
      Parameters:
      numPartitions - (undocumented)
      ord - (undocumented)
      Returns:
      (undocumented)
    • foreachRDD

      public void foreachRDD(scala.Function1<RDD<T>,scala.runtime.BoxedUnit> foreachFunc)
      Apply a function to each RDD in this DStream. This is an output operator, so 'this' DStream will be registered as an output stream and therefore materialized.
      Parameters:
      foreachFunc - (undocumented)
    • foreachRDD

      public void foreachRDD(scala.Function2<RDD<T>,Time,scala.runtime.BoxedUnit> foreachFunc)
      Apply a function to each RDD in this DStream. This is an output operator, so 'this' DStream will be registered as an output stream and therefore materialized.
      Parameters:
      foreachFunc - (undocumented)
    • transform

      public <U> DStream<U> transform(scala.Function1<RDD<T>,RDD<U>> transformFunc, scala.reflect.ClassTag<U> evidence$5)
      Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream.
      Parameters:
      transformFunc - (undocumented)
      evidence$5 - (undocumented)
      Returns:
      (undocumented)
    • transform

      public <U> DStream<U> transform(scala.Function2<RDD<T>,Time,RDD<U>> transformFunc, scala.reflect.ClassTag<U> evidence$6)
      Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream.
      Parameters:
      transformFunc - (undocumented)
      evidence$6 - (undocumented)
      Returns:
      (undocumented)
    • transformWith

      public <U, V> DStream<V> transformWith(DStream<U> other, scala.Function2<RDD<T>,RDD<U>,RDD<V>> transformFunc, scala.reflect.ClassTag<U> evidence$7, scala.reflect.ClassTag<V> evidence$8)
      Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream and 'other' DStream.
      Parameters:
      other - (undocumented)
      transformFunc - (undocumented)
      evidence$7 - (undocumented)
      evidence$8 - (undocumented)
      Returns:
      (undocumented)
    • transformWith

      public <U, V> DStream<V> transformWith(DStream<U> other, scala.Function3<RDD<T>,RDD<U>,Time,RDD<V>> transformFunc, scala.reflect.ClassTag<U> evidence$9, scala.reflect.ClassTag<V> evidence$10)
      Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream and 'other' DStream.
      Parameters:
      other - (undocumented)
      transformFunc - (undocumented)
      evidence$9 - (undocumented)
      evidence$10 - (undocumented)
      Returns:
      (undocumented)
    • print

      public void print()
      Print the first ten elements of each RDD generated in this DStream. This is an output operator, so this DStream will be registered as an output stream and there materialized.
    • print

      public void print(int num)
      Print the first num elements of each RDD generated in this DStream. This is an output operator, so this DStream will be registered as an output stream and there materialized.
      Parameters:
      num - (undocumented)
    • window

      public DStream<T> window(Duration windowDuration)
      Return a new DStream in which each RDD contains all the elements in seen in a sliding window of time over this DStream. The new DStream generates RDDs with the same interval as this DStream.
      Parameters:
      windowDuration - width of the window; must be a multiple of this DStream's interval.
      Returns:
      (undocumented)
    • window

      public DStream<T> window(Duration windowDuration, Duration slideDuration)
      Return a new DStream in which each RDD contains all the elements in seen in a sliding window of time over this DStream.
      Parameters:
      windowDuration - width of the window; must be a multiple of this DStream's batching interval
      slideDuration - sliding interval of the window (i.e., the interval after which the new DStream will generate RDDs); must be a multiple of this DStream's batching interval
      Returns:
      (undocumented)
    • reduceByWindow

      public DStream<T> reduceByWindow(scala.Function2<T,T,T> reduceFunc, Duration windowDuration, Duration slideDuration)
      Return a new DStream in which each RDD has a single element generated by reducing all elements in a sliding window over this DStream.
      Parameters:
      reduceFunc - associative and commutative reduce function
      windowDuration - width of the window; must be a multiple of this DStream's batching interval
      slideDuration - sliding interval of the window (i.e., the interval after which the new DStream will generate RDDs); must be a multiple of this DStream's batching interval
      Returns:
      (undocumented)
    • reduceByWindow

      public DStream<T> reduceByWindow(scala.Function2<T,T,T> reduceFunc, scala.Function2<T,T,T> invReduceFunc, Duration windowDuration, Duration slideDuration)
      Return a new DStream in which each RDD has a single element generated by reducing all elements in a sliding window over this DStream. However, the reduction is done incrementally using the old window's reduced value : 1. reduce the new values that entered the window (e.g., adding new counts) 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) This is more efficient than reduceByWindow without "inverse reduce" function. However, it is applicable to only "invertible reduce functions".
      Parameters:
      reduceFunc - associative and commutative reduce function
      invReduceFunc - inverse reduce function; such that for all y, invertible x: invReduceFunc(reduceFunc(x, y), x) = y
      windowDuration - width of the window; must be a multiple of this DStream's batching interval
      slideDuration - sliding interval of the window (i.e., the interval after which the new DStream will generate RDDs); must be a multiple of this DStream's batching interval
      Returns:
      (undocumented)
    • countByWindow

      public DStream<Object> countByWindow(Duration windowDuration, Duration slideDuration)
      Return a new DStream in which each RDD has a single element generated by counting the number of elements in a sliding window over this DStream. Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
      Parameters:
      windowDuration - width of the window; must be a multiple of this DStream's batching interval
      slideDuration - sliding interval of the window (i.e., the interval after which the new DStream will generate RDDs); must be a multiple of this DStream's batching interval
      Returns:
      (undocumented)
    • countByValueAndWindow

      public DStream<scala.Tuple2<T,Object>> countByValueAndWindow(Duration windowDuration, Duration slideDuration, int numPartitions, scala.math.Ordering<T> ord)
      Return a new DStream in which each RDD contains the count of distinct elements in RDDs in a sliding window over this DStream. Hash partitioning is used to generate the RDDs with numPartitions partitions (Spark's default number of partitions if numPartitions not specified).
      Parameters:
      windowDuration - width of the window; must be a multiple of this DStream's batching interval
      slideDuration - sliding interval of the window (i.e., the interval after which the new DStream will generate RDDs); must be a multiple of this DStream's batching interval
      numPartitions - number of partitions of each RDD in the new DStream.
      ord - (undocumented)
      Returns:
      (undocumented)
    • union

      public DStream<T> union(DStream<T> that)
      Return a new DStream by unifying data of another DStream with this DStream.
      Parameters:
      that - Another DStream having the same slideDuration as this DStream.
      Returns:
      (undocumented)
    • slice

      public scala.collection.immutable.Seq<RDD<T>> slice(org.apache.spark.streaming.Interval interval)
      Return all the RDDs defined by the Interval object (both end times included)
      Parameters:
      interval - (undocumented)
      Returns:
      (undocumented)
    • slice

      public scala.collection.immutable.Seq<RDD<T>> slice(Time fromTime, Time toTime)
      Return all the RDDs between 'fromTime' to 'toTime' (both included)
      Parameters:
      fromTime - (undocumented)
      toTime - (undocumented)
      Returns:
      (undocumented)
    • saveAsObjectFiles

      public void saveAsObjectFiles(String prefix, String suffix)
      Save each RDD in this DStream as a Sequence file of serialized objects. The file name at each batch interval is generated based on prefix and suffix: "prefix-TIME_IN_MS.suffix".
      Parameters:
      prefix - (undocumented)
      suffix - (undocumented)
    • saveAsTextFiles

      public void saveAsTextFiles(String prefix, String suffix)
      Save each RDD in this DStream as at text file, using string representation of elements. The file name at each batch interval is generated based on prefix and suffix: "prefix-TIME_IN_MS.suffix".
      Parameters:
      prefix - (undocumented)
      suffix - (undocumented)