org.apache.spark.streaming.api.java
Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER)
Enable periodic checkpointing of RDDs of this DStream.
Enable periodic checkpointing of RDDs of this DStream.
Time interval after which generated RDD will be checkpointed
Return a new DStream by applying 'cogroup' between RDDs of this
DStream and other
DStream.
Return a new DStream by applying 'cogroup' between RDDs of this
DStream and other
DStream.
Hash partitioning is used to generate the RDDs with numPartitions
partitions.
Return a new DStream by applying 'cogroup' between RDDs of this
DStream and other
DStream.
Return a new DStream by applying 'cogroup' between RDDs of this
DStream and other
DStream.
Hash partitioning is used to generate the RDDs with numPartitions
partitions.
Return a new DStream by applying 'cogroup' between RDDs of this
DStream and other
DStream.
Return a new DStream by applying 'cogroup' between RDDs of this
DStream and other
DStream.
Hash partitioning is used to generate the RDDs with Spark's default number
of partitions.
Combine elements of each key in DStream's RDDs using custom function.
Combine elements of each key in DStream's RDDs using custom function. This is similar to the combineByKey for RDDs. Please refer to combineByKey in org.apache.spark.rdd.PairRDDFunctions for more information.
Combine elements of each key in DStream's RDDs using custom function.
Combine elements of each key in DStream's RDDs using custom function. This is similar to the combineByKey for RDDs. Please refer to combineByKey in org.apache.spark.rdd.PairRDDFunctions for more information.
Method that generates an RDD for the given Duration
Return the org.apache.spark.streaming.StreamingContext associated with this DStream
Return the org.apache.spark.streaming.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.
Return a new DStream in which each RDD has a single element generated by counting 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.
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.
number of partitions of each RDD in the new DStream.
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 Spark's default number of partitions.
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.
width of the window; must be a multiple of this DStream's batching interval
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
number of partitions of each RDD in the new DStream.
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 Spark's default number of partitions.
width of the window; must be a multiple of this DStream's batching interval
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
Return a new DStream in which each RDD has a single element generated by counting the number of elements in a 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 window over this DStream. windowDuration and slideDuration are as defined in the window() operation. This is equivalent to window(windowDuration, slideDuration).count()
Return a new DStream containing only the elements that satisfy a predicate.
Return a new DStream by applying a function to all elements of this DStream, and then flattening the results
Return a new DStream by applying a function to all elements of this DStream, and then flattening the results
Return a new DStream by applying a function to all elements of this DStream, and then flattening the results
Return a new DStream by applying a function to all elements of this DStream, and then flattening the results
Return a new DStream by applying a flatmap function to the value of each key-value pairs in 'this' DStream without changing the key.
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.
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.
Return a new DStream by applying 'full outer join' between RDDs of this
DStream and
other
DStream.
Return a new DStream by applying 'full outer join' between RDDs of this
DStream and
other
DStream. The supplied org.apache.spark.Partitioner is used to control
the partitioning of each RDD.
Return a new DStream by applying 'full outer join' between RDDs of this
DStream and
other
DStream.
Return a new DStream by applying 'full outer join' between RDDs of this
DStream and
other
DStream. Hash partitioning is used to generate the RDDs with numPartitions
partitions.
Return a new DStream by applying 'full outer join' between RDDs of this
DStream and
other
DStream.
Return a new DStream by applying 'full outer join' between RDDs of this
DStream and
other
DStream. Hash partitioning is used to generate the RDDs with Spark's default
number of partitions.
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.
Return a new DStream by applying groupByKey
on each RDD of this
DStream.
Return a new DStream by applying groupByKey
on each RDD of this
DStream.
Therefore, the values for each key in this
DStream's RDDs are grouped into a
single sequence to generate the RDDs of the new DStream. org.apache.spark.Partitioner
is used to control the partitioning of each RDD.
Return a new DStream by applying groupByKey
to each RDD.
Return a new DStream by applying groupByKey
to each RDD. Hash partitioning is used to
generate the RDDs with numPartitions
partitions.
Return a new DStream by applying groupByKey
to each RDD.
Return a new DStream by applying groupByKey
to each RDD. Hash partitioning is used to
generate the RDDs with Spark's default number of partitions.
Return a new DStream by applying groupByKey
over a sliding window on this
DStream.
Return a new DStream by applying groupByKey
over a sliding window on this
DStream.
Similar to DStream.groupByKey()
, but applies it over a sliding window.
width of the window; must be a multiple of this DStream's batching interval
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
Partitioner for controlling the partitioning of each RDD in the new DStream.
Return a new DStream by applying groupByKey
over a sliding window on this
DStream.
Return a new DStream by applying groupByKey
over a sliding window on this
DStream.
Similar to DStream.groupByKey()
, but applies it over a sliding window.
Hash partitioning is used to generate the RDDs with numPartitions
partitions.
width of the window; must be a multiple of this DStream's batching interval
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
Number of partitions of each RDD in the new DStream.
Return a new DStream by applying groupByKey
over a sliding window.
Return a new DStream by applying groupByKey
over a sliding window. Similar to
DStream.groupByKey()
, but applies it over a sliding window. Hash partitioning is used to
generate the RDDs with Spark's default number of partitions.
width of the window; must be a multiple of this DStream's batching interval
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
Return a new DStream by applying groupByKey
over a sliding window.
Return a new DStream by applying groupByKey
over a sliding window. This is similar to
DStream.groupByKey()
but applies it over a sliding window. The new DStream generates RDDs
with the same interval as this DStream. Hash partitioning is used to generate the RDDs with
Spark's default number of partitions.
width of the window; must be a multiple of this DStream's batching interval
Return a new DStream by applying 'join' between RDDs of this
DStream and other
DStream.
Return a new DStream by applying 'join' between RDDs of this
DStream and other
DStream.
The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD.
Return a new DStream by applying 'join' between RDDs of this
DStream and other
DStream.
Return a new DStream by applying 'join' between RDDs of this
DStream and other
DStream.
Hash partitioning is used to generate the RDDs with numPartitions
partitions.
Return a new DStream by applying 'join' between RDDs of this
DStream and other
DStream.
Return a new DStream by applying 'join' between RDDs of this
DStream and other
DStream.
Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
Return a new DStream by applying 'left outer join' between RDDs of this
DStream and
other
DStream.
Return a new DStream by applying 'left outer join' between RDDs of this
DStream and
other
DStream. The supplied org.apache.spark.Partitioner is used to control
the partitioning of each RDD.
Return a new DStream by applying 'left outer join' between RDDs of this
DStream and
other
DStream.
Return a new DStream by applying 'left outer join' between RDDs of this
DStream and
other
DStream. Hash partitioning is used to generate the RDDs with numPartitions
partitions.
Return a new DStream by applying 'left outer join' between RDDs of this
DStream and
other
DStream.
Return a new DStream by applying 'left outer join' between RDDs of this
DStream and
other
DStream. Hash partitioning is used to generate the RDDs with Spark's default
number of partitions.
Return a new DStream by applying a function to all elements of this DStream.
Return a new DStream by applying a function to all elements of this DStream.
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.
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.
Return a new DStream by applying a function to all elements of this DStream.
Return a new DStream by applying a function to all elements of this DStream.
Return a new DStream by applying a map function to the value of each key-value pairs in 'this' DStream without changing the key.
:: Experimental ::
Return a JavaMapWithStateDStream by applying a function to every key-value element of
this
stream, while maintaining some state data for each unique key.
:: Experimental ::
Return a JavaMapWithStateDStream by applying a function to every key-value element of
this
stream, while maintaining some state data for each unique key. The mapping function
and other specification (e.g. partitioners, timeouts, initial state data, etc.) of this
transformation can be specified using StateSpec
class. The state data is accessible in
as a parameter of type State
in the mapping function.
Example of using mapWithState
:
// A mapping function that maintains an integer state and return a string Function3<String, Optional<Integer>, State<Integer>, String> mappingFunction = new Function3<String, Optional<Integer>, State<Integer>, String>() { @Override public Optional<String> call(Optional<Integer> value, State<Integer> state) { // Use state.exists(), state.get(), state.update() and state.remove() // to manage state, and return the necessary string } }; JavaMapWithStateDStream<String, Integer, Integer, String> mapWithStateDStream = keyValueDStream.mapWithState(StateSpec.function(mappingFunc));
Class type of the state data
Class type of the mapped data
Specification of this transformation
Persist the RDDs of this DStream with the given storage level
Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER)
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.
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.
Return a new DStream in which each RDD has a single element generated by reducing each RDD of this DStream.
Return a new DStream in which each RDD has a single element generated by reducing each RDD of this DStream.
Return a new DStream by applying reduceByKey
to each RDD.
Return a new DStream by applying reduceByKey
to each RDD. The values for each key are
merged using the supplied reduce function. org.apache.spark.Partitioner is used to control
the partitioning of each RDD.
Return a new DStream by applying reduceByKey
to each RDD.
Return a new DStream by applying reduceByKey
to each RDD. The values for each key are
merged using the supplied reduce function. Hash partitioning is used to generate the RDDs
with numPartitions
partitions.
Return a new DStream by applying reduceByKey
to each RDD.
Return a new DStream by applying reduceByKey
to each RDD. The values for each key are
merged using the associative and commutative reduce function. Hash partitioning is used to
generate the RDDs with Spark's default number of partitions.
Return a new DStream by applying incremental reduceByKey
over a sliding window.
Return a new DStream by applying incremental reduceByKey
over a sliding window.
The reduced value of over a new window is calculated using the old window's reduce value :
associative and commutative reduce function
inverse function
width of the window; must be a multiple of this DStream's batching interval
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
Partitioner for controlling the partitioning of each RDD in the new DStream.
function to filter expired key-value pairs; only pairs that satisfy the function are retained set this to null if you do not want to filter
Return a new DStream by applying incremental reduceByKey
over a sliding window.
Return a new DStream by applying incremental reduceByKey
over a sliding window.
The reduced value of over a new window is calculated using the old window's reduce value :
numPartitions
partitions.associative and commutative reduce function
inverse function
width of the window; must be a multiple of this DStream's batching interval
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
number of partitions of each RDD in the new DStream.
function to filter expired key-value pairs; only pairs that satisfy the function are retained set this to null if you do not want to filter
Return a new DStream by reducing over a using incremental computation.
Return a new DStream by reducing over a using incremental computation. The reduced value of over a new window is calculated using the old window's reduce value :
associative and commutative reduce function
inverse function; such that for all y, invertible x:
invReduceFunc(reduceFunc(x, y), x) = y
width of the window; must be a multiple of this DStream's batching interval
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
Return a new DStream by applying reduceByKey
over a sliding window.
Return a new DStream by applying reduceByKey
over a sliding window. Similar to
DStream.reduceByKey()
, but applies it over a sliding window.
associative rand commutative educe function
width of the window; must be a multiple of this DStream's batching interval
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
Partitioner for controlling the partitioning of each RDD in the new DStream.
Return a new DStream by applying reduceByKey
over a sliding window.
Return a new DStream by applying reduceByKey
over a sliding window. This is similar to
DStream.reduceByKey()
but applies it over a sliding window. Hash partitioning is used to
generate the RDDs with numPartitions
partitions.
associative and commutative reduce function
width of the window; must be a multiple of this DStream's batching interval
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
Number of partitions of each RDD in the new DStream.
Return a new DStream by applying reduceByKey
over a sliding window.
Return a new DStream by applying reduceByKey
over a sliding window. This is similar to
DStream.reduceByKey()
but applies it over a sliding window. Hash partitioning is used to
generate the RDDs with Spark's default number of partitions.
associative and commutative reduce function
width of the window; must be a multiple of this DStream's batching interval
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
Create a new DStream by applying reduceByKey
over a sliding window on this
DStream.
Create a new DStream by applying reduceByKey
over a sliding window on this
DStream.
Similar to DStream.reduceByKey()
, but applies it over a sliding window. The new DStream
generates RDDs with the same interval as this DStream. Hash partitioning is used to generate
the RDDs with Spark's default number of partitions.
associative and commutative reduce function
width of the window; must be a multiple of this DStream's batching interval
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 :
associative and commutative reduce function
inverse reduce function; such that for all y, invertible x:
invReduceFunc(reduceFunc(x, y), x) = y
width of the window; must be a multiple of this DStream's batching interval
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
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.
associative and commutative reduce function
width of the window; must be a multiple of this DStream's batching interval
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
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.
Return a new DStream by applying 'right outer join' between RDDs of this
DStream and
other
DStream.
Return a new DStream by applying 'right outer join' between RDDs of this
DStream and
other
DStream. The supplied org.apache.spark.Partitioner is used to control
the partitioning of each RDD.
Return a new DStream by applying 'right outer join' between RDDs of this
DStream and
other
DStream.
Return a new DStream by applying 'right outer join' between RDDs of this
DStream and
other
DStream. Hash partitioning is used to generate the RDDs with numPartitions
partitions.
Return a new DStream by applying 'right outer join' between RDDs of this
DStream and
other
DStream.
Return a new DStream by applying 'right outer join' between RDDs of this
DStream and
other
DStream. Hash partitioning is used to generate the RDDs with Spark's default
number of partitions.
Save each RDD in this
DStream as a Hadoop file.
Save each RDD in this
DStream as a Hadoop file. The file name at each batch interval is
generated based on prefix
and suffix
: "prefix-TIME_IN_MS.suffix".
Save each RDD in this
DStream as a Hadoop file.
Save each RDD in this
DStream as a Hadoop file. The file name at each batch interval is
generated based on prefix
and suffix
: "prefix-TIME_IN_MS.suffix".
Save each RDD in this
DStream as a Hadoop file.
Save each RDD in this
DStream as a Hadoop file. The file name at each batch interval is
generated based on prefix
and suffix
: "prefix-TIME_IN_MS.suffix".
Save each RDD in this
DStream as a Hadoop file.
Save each RDD in this
DStream as a Hadoop file. The file name at each batch interval is
generated based on prefix
and suffix
: "prefix-TIME_IN_MS.suffix".
Save each RDD in this
DStream as a Hadoop file.
Save each RDD in this
DStream as a Hadoop file. The file name at each batch interval is
generated based on prefix
and suffix
: "prefix-TIME_IN_MS.suffix".
Save each RDD in this
DStream as a Hadoop file.
Save each RDD in this
DStream as a Hadoop file. The file name at each batch interval is
generated based on prefix
and suffix
: "prefix-TIME_IN_MS.suffix".
Return all the RDDs between 'fromDuration' to 'toDuration' (both included)
Return all the RDDs between 'fromDuration' to 'toDuration' (both included)
Convert to a JavaDStream
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream.
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream.
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream.
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream.
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream.
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream.
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream.
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream.
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream and 'other' DStream.
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream and 'other' DStream.
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream and 'other' DStream.
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream and 'other' DStream.
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream and 'other' DStream.
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream and 'other' DStream.
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream and 'other' DStream.
Return a new DStream in which each RDD is generated by applying a function on each RDD of 'this' DStream and 'other' DStream.
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.
Another DStream having the same interval (i.e., slideDuration) as this DStream.
Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values of the key.
Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values of the key. org.apache.spark.Partitioner is used to control the partitioning of each RDD.
State type
State update function. If this
function returns None, then
corresponding state key-value pair will be eliminated.
Partitioner for controlling the partitioning of each RDD in the new DStream.
initial state value of each key.
Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values of the key.
Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values of the key. org.apache.spark.Partitioner is used to control the partitioning of each RDD.
State type
State update function. If this
function returns None, then
corresponding state key-value pair will be eliminated.
Partitioner for controlling the partitioning of each RDD in the new DStream.
Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values of each key.
Return a new "state" DStream where the state for each key is updated by applying
the given function on the previous state of the key and the new values of each key.
Hash partitioning is used to generate the RDDs with numPartitions
partitions.
State type
State update function. If this
function returns None, then
corresponding state key-value pair will be eliminated.
Number of partitions of each RDD in the new DStream.
Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values of each key.
Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values of each key. Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
State type
State update function. If this
function returns None, then
corresponding state key-value pair will be eliminated.
Return a new DStream which is computed based on windowed batches of this DStream.
Return a new DStream which is computed based on windowed batches of this DStream.
duration (i.e., width) of the window; must be a multiple of this DStream's interval
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 interval
Return a new DStream which is computed based on windowed batches of this DStream.
Return a new DStream which is computed based on windowed batches of this DStream. The new DStream generates RDDs with the same interval as this DStream.
width of the window; must be a multiple of this DStream's interval.
A Java-friendly interface to a DStream of key-value pairs, which provides extra methods like
reduceByKey
andjoin
.