public abstract class DStream<T> extends java.lang.Object implements scala.Serializable, Logging
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.
DStreams 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
Constructor and Description |
---|
DStream(StreamingContext ssc,
scala.reflect.ClassTag<T> evidence$1) |
Modifier and Type | Method and Description |
---|---|
protected scala.Option<java.lang.String> |
baseScope()
The base scope associated with the operation that created this DStream.
|
DStream<T> |
cache()
Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER)
|
DStream<T> |
checkpoint(Duration interval)
Enable periodic checkpointing of RDDs of this DStream
|
org.apache.spark.streaming.dstream.DStreamCheckpointData<T> |
checkpointData() |
Duration |
checkpointDuration() |
abstract scala.Option<RDD<T>> |
compute(Time validTime)
Method that generates a RDD for the given time
|
StreamingContext |
context()
Return the StreamingContext associated with this DStream
|
DStream<java.lang.Object> |
count()
Return a new DStream in which each RDD has a single element generated by counting each RDD
of this DStream.
|
DStream<scala.Tuple2<T,java.lang.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,java.lang.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.
|
DStream<java.lang.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.
|
protected <U> U |
createRDDWithLocalProperties(Time time,
scala.Function0<U> body)
Wrap a body of code such that the call site and operation scope
information are passed to the RDDs created in this body properly.
|
org.apache.spark.util.CallSite |
creationSite() |
abstract scala.collection.immutable.List<DStream<?>> |
dependencies()
List of parent DStreams on which this DStream depends on
|
DStream<T> |
filter(scala.Function1<T,java.lang.Object> filterFunc)
Return a new DStream containing only the elements that satisfy a predicate.
|
<U> DStream<U> |
flatMap(scala.Function1<T,scala.collection.Traversable<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 |
foreach(scala.Function1<RDD<T>,scala.runtime.BoxedUnit> foreachFunc)
Deprecated.
As of 0.9.0, replaced by
foreachRDD . |
void |
foreach(scala.Function2<RDD<T>,Time,scala.runtime.BoxedUnit> foreachFunc)
Deprecated.
As of 0.9.0, replaced by
foreachRDD . |
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.
|
scala.collection.mutable.HashMap<Time,RDD<T>> |
generatedRDDs() |
DStream<java.lang.Object> |
glom()
Return a new DStream in which each RDD is generated by applying glom() to each RDD of
this DStream.
|
org.apache.spark.streaming.DStreamGraph |
graph() |
<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.
|
boolean |
mustCheckpoint() |
DStream<T> |
persist()
Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER)
|
DStream<T> |
persist(StorageLevel level)
Persist the RDDs of this DStream with the given storage level
|
void |
print()
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.
|
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.
|
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.
|
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.
|
Duration |
rememberDuration() |
DStream<T> |
repartition(int numPartitions)
Return a new DStream with an increased or decreased level of parallelism.
|
void |
saveAsObjectFiles(java.lang.String prefix,
java.lang.String suffix)
Save each RDD in this DStream as a Sequence file of serialized objects.
|
void |
saveAsTextFiles(java.lang.String prefix,
java.lang.String suffix)
Save each RDD in this DStream as at text file, using string representation
of elements.
|
scala.collection.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.Seq<RDD<T>> |
slice(Time fromTime,
Time toTime)
Return all the RDDs between 'fromTime' to 'toTime' (both included)
|
abstract Duration |
slideDuration()
Time interval after which the DStream generates a RDD
|
StreamingContext |
ssc() |
StorageLevel |
storageLevel() |
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.
|
DStream<T> |
union(DStream<T> that)
Return a new DStream by unifying data of another DStream with this DStream.
|
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.
|
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.
|
Time |
zeroTime() |
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
initializeIfNecessary, initializeLogging, isTraceEnabled, log_, log, logDebug, logDebug, logError, logError, logInfo, logInfo, logName, logTrace, logTrace, logWarning, logWarning
public DStream(StreamingContext ssc, scala.reflect.ClassTag<T> evidence$1)
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)
public StreamingContext ssc()
public abstract Duration slideDuration()
public abstract scala.collection.immutable.List<DStream<?>> dependencies()
public abstract scala.Option<RDD<T>> compute(Time validTime)
public Time zeroTime()
public Duration rememberDuration()
public StorageLevel storageLevel()
public boolean mustCheckpoint()
public Duration checkpointDuration()
public org.apache.spark.streaming.dstream.DStreamCheckpointData<T> checkpointData()
public org.apache.spark.streaming.DStreamGraph graph()
public StreamingContext context()
public org.apache.spark.util.CallSite creationSite()
protected scala.Option<java.lang.String> baseScope()
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.
public DStream<T> persist(StorageLevel level)
public DStream<T> persist()
public DStream<T> cache()
public DStream<T> checkpoint(Duration interval)
interval
- Time interval after which generated RDD will be checkpointedprotected <U> U createRDDWithLocalProperties(Time time, scala.Function0<U> body)
time
- (undocumented)body
- (undocumented)public <U> DStream<U> map(scala.Function1<T,U> mapFunc, scala.reflect.ClassTag<U> evidence$2)
public <U> DStream<U> flatMap(scala.Function1<T,scala.collection.Traversable<U>> flatMapFunc, scala.reflect.ClassTag<U> evidence$3)
flatMapFunc
- (undocumented)evidence$3
- (undocumented)public DStream<T> filter(scala.Function1<T,java.lang.Object> filterFunc)
public DStream<java.lang.Object> glom()
public DStream<T> repartition(int numPartitions)
numPartitions
- (undocumented)public <U> DStream<U> mapPartitions(scala.Function1<scala.collection.Iterator<T>,scala.collection.Iterator<U>> mapPartFunc, boolean preservePartitioning, scala.reflect.ClassTag<U> evidence$4)
mapPartFunc
- (undocumented)preservePartitioning
- (undocumented)evidence$4
- (undocumented)public DStream<T> reduce(scala.Function2<T,T,T> reduceFunc)
reduceFunc
- (undocumented)public DStream<java.lang.Object> count()
public DStream<scala.Tuple2<T,java.lang.Object>> countByValue(int numPartitions, scala.math.Ordering<T> ord)
numPartitions
partitions (Spark's default number of partitions if
numPartitions
not specified).numPartitions
- (undocumented)ord
- (undocumented)public void foreach(scala.Function1<RDD<T>,scala.runtime.BoxedUnit> foreachFunc)
foreachRDD
.foreachFunc
- (undocumented)public void foreach(scala.Function2<RDD<T>,Time,scala.runtime.BoxedUnit> foreachFunc)
foreachRDD
.foreachFunc
- (undocumented)public void foreachRDD(scala.Function1<RDD<T>,scala.runtime.BoxedUnit> foreachFunc)
foreachFunc
- (undocumented)public void foreachRDD(scala.Function2<RDD<T>,Time,scala.runtime.BoxedUnit> foreachFunc)
foreachFunc
- (undocumented)public <U> DStream<U> transform(scala.Function1<RDD<T>,RDD<U>> transformFunc, scala.reflect.ClassTag<U> evidence$5)
transformFunc
- (undocumented)evidence$5
- (undocumented)public <U> DStream<U> transform(scala.Function2<RDD<T>,Time,RDD<U>> transformFunc, scala.reflect.ClassTag<U> evidence$6)
transformFunc
- (undocumented)evidence$6
- (undocumented)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)
other
- (undocumented)transformFunc
- (undocumented)evidence$7
- (undocumented)evidence$8
- (undocumented)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)
other
- (undocumented)transformFunc
- (undocumented)evidence$9
- (undocumented)evidence$10
- (undocumented)public void print()
public void print(int num)
num
- (undocumented)public DStream<T> window(Duration windowDuration)
windowDuration
- width of the window; must be a multiple of this DStream's interval.public DStream<T> window(Duration windowDuration, Duration slideDuration)
windowDuration
- width of the window; must be a multiple of this DStream's
batching intervalslideDuration
- 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 intervalpublic DStream<T> reduceByWindow(scala.Function2<T,T,T> reduceFunc, Duration windowDuration, Duration slideDuration)
reduceFunc
- associative reduce functionwindowDuration
- width of the window; must be a multiple of this DStream's
batching intervalslideDuration
- 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 intervalpublic DStream<T> reduceByWindow(scala.Function2<T,T,T> reduceFunc, scala.Function2<T,T,T> invReduceFunc, Duration windowDuration, Duration slideDuration)
reduceFunc
- associative reduce functioninvReduceFunc
- inverse reduce functionwindowDuration
- width of the window; must be a multiple of this DStream's
batching intervalslideDuration
- 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 intervalpublic DStream<java.lang.Object> countByWindow(Duration windowDuration, Duration slideDuration)
windowDuration
- width of the window; must be a multiple of this DStream's
batching intervalslideDuration
- 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 intervalpublic DStream<scala.Tuple2<T,java.lang.Object>> countByValueAndWindow(Duration windowDuration, Duration slideDuration, int numPartitions, scala.math.Ordering<T> ord)
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 intervalslideDuration
- 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 intervalnumPartitions
- number of partitions of each RDD in the new DStream.ord
- (undocumented)public DStream<T> union(DStream<T> that)
that
- Another DStream having the same slideDuration as this DStream.public scala.collection.Seq<RDD<T>> slice(org.apache.spark.streaming.Interval interval)
interval
- (undocumented)public scala.collection.Seq<RDD<T>> slice(Time fromTime, Time toTime)
fromTime
- (undocumented)toTime
- (undocumented)public void saveAsObjectFiles(java.lang.String prefix, java.lang.String suffix)
prefix
and
suffix
: "prefix-TIME_IN_MS.suffix".prefix
- (undocumented)suffix
- (undocumented)public void saveAsTextFiles(java.lang.String prefix, java.lang.String suffix)
prefix
and suffix
: "prefix-TIME_IN_MS.suffix".prefix
- (undocumented)suffix
- (undocumented)