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
- Alphabetic
- By Inheritance
- DStream
- Logging
- Serializable
- Serializable
- AnyRef
- Any
- Hide All
- Show All
- Public
- All
Instance Constructors
- new DStream(ssc: StreamingContext)(implicit arg0: ClassTag[T])
Abstract Value Members
-
abstract
def
compute(validTime: Time): Option[RDD[T]]
Method that generates an RDD for the given time
-
abstract
def
dependencies: List[DStream[_]]
List of parent DStreams on which this DStream depends on
-
abstract
def
slideDuration: Duration
Time interval after which the DStream generates an RDD
Concrete Value Members
-
final
def
!=(arg0: Any): Boolean
- Definition Classes
- AnyRef → Any
-
final
def
##(): Int
- Definition Classes
- AnyRef → Any
-
final
def
==(arg0: Any): Boolean
- Definition Classes
- AnyRef → Any
-
final
def
asInstanceOf[T0]: T0
- Definition Classes
- Any
-
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[org.apache.spark.streaming]
-
def
cache(): DStream[T]
Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER)
-
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
-
def
clone(): AnyRef
- Attributes
- protected[java.lang]
- Definition Classes
- AnyRef
- Annotations
- @native() @throws( ... )
-
def
context: StreamingContext
Return the StreamingContext associated with this DStream
-
def
count(): DStream[Long]
Return a new DStream in which each RDD has a single element generated by counting each RDD of this DStream.
-
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 ifnumPartitions
not specified). -
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 ifnumPartitions
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.
-
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
-
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 generatedthis
will be displayed.- body
RDD creation code to execute with certain local properties.
- Attributes
- protected[org.apache.spark.streaming]
-
final
def
eq(arg0: AnyRef): Boolean
- Definition Classes
- AnyRef
-
def
equals(arg0: Any): Boolean
- Definition Classes
- AnyRef → Any
-
def
filter(filterFunc: (T) ⇒ Boolean): DStream[T]
Return a new DStream containing only the elements that satisfy a predicate.
-
def
finalize(): Unit
- Attributes
- protected[java.lang]
- Definition Classes
- AnyRef
- Annotations
- @throws( classOf[java.lang.Throwable] )
-
def
flatMap[U](flatMapFunc: (T) ⇒ TraversableOnce[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
-
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.
-
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.
-
final
def
getClass(): Class[_]
- Definition Classes
- AnyRef → Any
- Annotations
- @native()
-
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.
-
def
hashCode(): Int
- Definition Classes
- AnyRef → Any
- Annotations
- @native()
-
def
initializeLogIfNecessary(isInterpreter: Boolean, silent: Boolean = false): Boolean
- Attributes
- protected
- Definition Classes
- Logging
-
def
initializeLogIfNecessary(isInterpreter: Boolean): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
final
def
isInstanceOf[T0]: Boolean
- Definition Classes
- Any
-
def
isTraceEnabled(): Boolean
- Attributes
- protected
- Definition Classes
- Logging
-
def
log: Logger
- Attributes
- protected
- Definition Classes
- Logging
-
def
logDebug(msg: ⇒ String, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logDebug(msg: ⇒ String): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logError(msg: ⇒ String, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logError(msg: ⇒ String): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logInfo(msg: ⇒ String, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logInfo(msg: ⇒ String): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logName: String
- Attributes
- protected
- Definition Classes
- Logging
-
def
logTrace(msg: ⇒ String, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logTrace(msg: ⇒ String): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logWarning(msg: ⇒ String, throwable: Throwable): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
def
logWarning(msg: ⇒ String): Unit
- Attributes
- protected
- Definition Classes
- Logging
-
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.
-
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.
-
final
def
ne(arg0: AnyRef): Boolean
- Definition Classes
- AnyRef
-
final
def
notify(): Unit
- Definition Classes
- AnyRef
- Annotations
- @native()
-
final
def
notifyAll(): Unit
- Definition Classes
- AnyRef
- Annotations
- @native()
-
def
persist(): DStream[T]
Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER)
-
def
persist(level: StorageLevel): DStream[T]
Persist the RDDs of this DStream with the given storage level
-
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.
-
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.
-
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.
-
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 :
- 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
-
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
-
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.
-
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
andsuffix
: "prefix-TIME_IN_MS.suffix". -
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
andsuffix
: "prefix-TIME_IN_MS.suffix". -
def
slice(fromTime: Time, toTime: Time): Seq[RDD[T]]
Return all the RDDs between 'fromTime' to 'toTime' (both included)
-
def
slice(interval: Interval): Seq[RDD[T]]
Return all the RDDs defined by the Interval object (both end times included)
-
final
def
synchronized[T0](arg0: ⇒ T0): T0
- Definition Classes
- AnyRef
-
def
toString(): String
- Definition Classes
- AnyRef → Any
-
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.
-
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.
-
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.
-
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.
-
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.
-
final
def
wait(): Unit
- Definition Classes
- AnyRef
- Annotations
- @throws( ... )
-
final
def
wait(arg0: Long, arg1: Int): Unit
- Definition Classes
- AnyRef
- Annotations
- @throws( ... )
-
final
def
wait(arg0: Long): Unit
- Definition Classes
- AnyRef
- Annotations
- @native() @throws( ... )
-
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
-
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.