Packages

abstract class DStream[T] extends Serializable with 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 org.apache.spark.streaming.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, org.apache.spark.streaming.dstream.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
Source
DStream.scala
Linear Supertypes
Logging, Serializable, AnyRef, Any
Ordering
  1. Alphabetic
  2. By Inheritance
Inherited
  1. DStream
  2. Logging
  3. Serializable
  4. AnyRef
  5. Any
  1. Hide All
  2. Show All
Visibility
  1. Public
  2. Protected

Instance Constructors

  1. new DStream(ssc: StreamingContext)(implicit arg0: ClassTag[T])

Type Members

  1. implicit class LogStringContext extends AnyRef
    Definition Classes
    Logging

Abstract Value Members

  1. abstract def compute(validTime: Time): Option[RDD[T]]

    Method that generates an RDD for the given time

  2. abstract def dependencies: List[DStream[_]]

    List of parent DStreams on which this DStream depends on

  3. abstract def slideDuration: Duration

    Time interval after which the DStream generates an RDD

Concrete Value Members

  1. final def !=(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  2. final def ##: Int
    Definition Classes
    AnyRef → Any
  3. final def ==(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  4. final def asInstanceOf[T0]: T0
    Definition Classes
    Any
  5. val baseScope: Option[String]

    The base scope associated with the operation that created this DStream.

    The base scope associated with the operation that created this DStream.

    This is the medium through which we pass the DStream operation name (e.g. updatedStateByKey) to the RDDs created by this DStream. Note that we never use this scope directly in RDDs. Instead, we instantiate a new scope during each call to compute based on this one.

    This is not defined if the DStream is created outside of one of the public DStream operations.

    Attributes
    protected[streaming]
  6. def cache(): DStream[T]

    Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER)

  7. def checkpoint(interval: Duration): DStream[T]

    Enable periodic checkpointing of RDDs of this DStream

    Enable periodic checkpointing of RDDs of this DStream

    interval

    Time interval after which generated RDD will be checkpointed

  8. def clone(): AnyRef
    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.CloneNotSupportedException]) @IntrinsicCandidate() @native()
  9. def context: StreamingContext

    Return the StreamingContext associated with this DStream

  10. def count(): DStream[Long]

    Return a new DStream in which each RDD has a single element generated by counting each RDD of this DStream.

  11. def countByValue(numPartitions: Int = ssc.sc.defaultParallelism)(implicit ord: Ordering[T] = null): DStream[(T, Long)]

    Return a new DStream in which each RDD contains the counts of each distinct value in each RDD of this DStream.

    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).

  12. def countByValueAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int = ssc.sc.defaultParallelism)(implicit ord: Ordering[T] = null): DStream[(T, Long)]

    Return a new DStream in which each RDD contains the count of distinct elements in RDDs in a sliding window over this DStream.

    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).

    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.

  13. def countByWindow(windowDuration: Duration, slideDuration: Duration): DStream[Long]

    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.

    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.

    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

  14. def createRDDWithLocalProperties[U](time: Time, displayInnerRDDOps: Boolean)(body: => U): U

    Wrap a body of code such that the call site and operation scope information are passed to the RDDs created in this body properly.

    Wrap a body of code such that the call site and operation scope information are passed to the RDDs created in this body properly.

    time

    Current batch time that should be embedded in the scope names

    displayInnerRDDOps

    Whether the detailed callsites and scopes of the inner RDDs generated by body will be displayed in the UI; only the scope and callsite of the DStream operation that generated this will be displayed.

    body

    RDD creation code to execute with certain local properties.

    Attributes
    protected[streaming]
  15. final def eq(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  16. def equals(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef → Any
  17. def filter(filterFunc: (T) => Boolean): DStream[T]

    Return a new DStream containing only the elements that satisfy a predicate.

  18. def flatMap[U](flatMapFunc: (T) => IterableOnce[U])(implicit arg0: ClassTag[U]): DStream[U]

    Return a new DStream by applying a function to all elements of this DStream, and then flattening the results

  19. def foreachRDD(foreachFunc: (RDD[T], Time) => Unit): Unit

    Apply a function to each RDD in this DStream.

    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.

  20. def foreachRDD(foreachFunc: (RDD[T]) => Unit): Unit

    Apply a function to each RDD in this DStream.

    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.

  21. final def getClass(): Class[_ <: AnyRef]
    Definition Classes
    AnyRef → Any
    Annotations
    @IntrinsicCandidate() @native()
  22. def glom(): DStream[Array[T]]

    Return a new DStream in which each RDD is generated by applying glom() to each RDD of this DStream.

    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.

  23. def hashCode(): Int
    Definition Classes
    AnyRef → Any
    Annotations
    @IntrinsicCandidate() @native()
  24. def initializeLogIfNecessary(isInterpreter: Boolean, silent: Boolean): Boolean
    Attributes
    protected
    Definition Classes
    Logging
  25. def initializeLogIfNecessary(isInterpreter: Boolean): Unit
    Attributes
    protected
    Definition Classes
    Logging
  26. final def isInstanceOf[T0]: Boolean
    Definition Classes
    Any
  27. def isTraceEnabled(): Boolean
    Attributes
    protected
    Definition Classes
    Logging
  28. def log: Logger
    Attributes
    protected
    Definition Classes
    Logging
  29. def logDebug(msg: => String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  30. def logDebug(entry: LogEntry, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  31. def logDebug(entry: LogEntry): Unit
    Attributes
    protected
    Definition Classes
    Logging
  32. def logDebug(msg: => String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  33. def logError(msg: => String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  34. def logError(entry: LogEntry, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  35. def logError(entry: LogEntry): Unit
    Attributes
    protected
    Definition Classes
    Logging
  36. def logError(msg: => String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  37. def logInfo(msg: => String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  38. def logInfo(entry: LogEntry, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  39. def logInfo(entry: LogEntry): Unit
    Attributes
    protected
    Definition Classes
    Logging
  40. def logInfo(msg: => String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  41. def logName: String
    Attributes
    protected
    Definition Classes
    Logging
  42. def logTrace(msg: => String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  43. def logTrace(entry: LogEntry, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  44. def logTrace(entry: LogEntry): Unit
    Attributes
    protected
    Definition Classes
    Logging
  45. def logTrace(msg: => String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  46. def logWarning(msg: => String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  47. def logWarning(entry: LogEntry, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  48. def logWarning(entry: LogEntry): Unit
    Attributes
    protected
    Definition Classes
    Logging
  49. def logWarning(msg: => String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  50. def map[U](mapFunc: (T) => U)(implicit arg0: ClassTag[U]): DStream[U]

    Return a new DStream by applying a function to all elements of this DStream.

  51. def mapPartitions[U](mapPartFunc: (Iterator[T]) => Iterator[U], preservePartitioning: Boolean = false)(implicit arg0: ClassTag[U]): DStream[U]

    Return a new DStream in which each RDD is generated by applying mapPartitions() to each RDDs of this DStream.

    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.

  52. final def ne(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  53. final def notify(): Unit
    Definition Classes
    AnyRef
    Annotations
    @IntrinsicCandidate() @native()
  54. final def notifyAll(): Unit
    Definition Classes
    AnyRef
    Annotations
    @IntrinsicCandidate() @native()
  55. def persist(): DStream[T]

    Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER)

  56. def persist(level: StorageLevel): DStream[T]

    Persist the RDDs of this DStream with the given storage level

  57. def print(num: Int): Unit

    Print the first num elements of each RDD generated in this DStream.

    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.

  58. def print(): Unit

    Print the first ten elements of each RDD generated in this DStream.

    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.

  59. def reduce(reduceFunc: (T, T) => T): DStream[T]

    Return a new DStream in which each RDD has a single element generated by reducing each RDD of this DStream.

  60. def reduceByWindow(reduceFunc: (T, T) => T, invReduceFunc: (T, T) => T, windowDuration: Duration, slideDuration: Duration): DStream[T]

    Return a new DStream in which each RDD has a single element generated by reducing all elements in a sliding window over this DStream.

    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".
    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

  61. def reduceByWindow(reduceFunc: (T, T) => T, windowDuration: Duration, slideDuration: Duration): DStream[T]

    Return a new DStream in which each RDD has a single element generated by reducing all elements in a sliding window over this DStream.

    Return a new DStream in which each RDD has a single element generated by reducing all elements in a sliding window over this DStream.

    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

  62. def repartition(numPartitions: Int): DStream[T]

    Return a new DStream with an increased or decreased level of parallelism.

    Return a new DStream with an increased or decreased level of parallelism. Each RDD in the returned DStream has exactly numPartitions partitions.

  63. def saveAsObjectFiles(prefix: String, suffix: String = ""): Unit

    Save each RDD in this DStream as a Sequence file of serialized objects.

    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".

  64. def saveAsTextFiles(prefix: String, suffix: String = ""): Unit

    Save each RDD in this DStream as at text file, using string representation of elements.

    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".

  65. def slice(fromTime: Time, toTime: Time): Seq[RDD[T]]

    Return all the RDDs between 'fromTime' to 'toTime' (both included)

  66. def slice(interval: Interval): Seq[RDD[T]]

    Return all the RDDs defined by the Interval object (both end times included)

  67. final def synchronized[T0](arg0: => T0): T0
    Definition Classes
    AnyRef
  68. def toString(): String
    Definition Classes
    AnyRef → Any
  69. def transform[U](transformFunc: (RDD[T], Time) => RDD[U])(implicit arg0: ClassTag[U]): DStream[U]

    Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream.

  70. def transform[U](transformFunc: (RDD[T]) => RDD[U])(implicit arg0: ClassTag[U]): DStream[U]

    Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream.

  71. def transformWith[U, V](other: DStream[U], transformFunc: (RDD[T], RDD[U], Time) => RDD[V])(implicit arg0: ClassTag[U], arg1: ClassTag[V]): DStream[V]

    Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream and 'other' DStream.

  72. def transformWith[U, V](other: DStream[U], transformFunc: (RDD[T], RDD[U]) => RDD[V])(implicit arg0: ClassTag[U], arg1: ClassTag[V]): DStream[V]

    Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream and 'other' DStream.

  73. def union(that: DStream[T]): DStream[T]

    Return a new DStream by unifying data of another DStream with this DStream.

    Return a new DStream by unifying data of another DStream with this DStream.

    that

    Another DStream having the same slideDuration as this DStream.

  74. final def wait(arg0: Long, arg1: Int): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.InterruptedException])
  75. final def wait(arg0: Long): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.InterruptedException]) @native()
  76. final def wait(): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.InterruptedException])
  77. def window(windowDuration: Duration, slideDuration: Duration): DStream[T]

    Return a new DStream in which each RDD contains all the elements in seen in a sliding window of time over this DStream.

    Return a new DStream in which each RDD contains all the elements in seen in a sliding window of time over this DStream.

    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

  78. def window(windowDuration: Duration): DStream[T]

    Return a new DStream in which each RDD contains all the elements in seen in a sliding window of time over this DStream.

    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.

    windowDuration

    width of the window; must be a multiple of this DStream's interval.

  79. def withLogContext(context: HashMap[String, String])(body: => Unit): Unit
    Attributes
    protected
    Definition Classes
    Logging

Deprecated Value Members

  1. def finalize(): Unit
    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.Throwable]) @Deprecated
    Deprecated

    (Since version 9)

Inherited from Logging

Inherited from Serializable

Inherited from AnyRef

Inherited from Any

Ungrouped