Packages

class StreamingContext extends Logging

Main entry point for Spark Streaming functionality. It provides methods used to create org.apache.spark.streaming.dstream.DStreams from various input sources. It can be either created by providing a Spark master URL and an appName, or from a org.apache.spark.SparkConf configuration (see core Spark documentation), or from an existing org.apache.spark.SparkContext. The associated SparkContext can be accessed using context.sparkContext. After creating and transforming DStreams, the streaming computation can be started and stopped using context.start() and context.stop(), respectively. context.awaitTermination() allows the current thread to wait for the termination of the context by stop() or by an exception.

Annotations
@deprecated
Deprecated

(Since version Spark 3.4.0) DStream is deprecated. Migrate to Structured Streaming.

Source
StreamingContext.scala
Linear Supertypes
Logging, AnyRef, Any
Ordering
  1. Alphabetic
  2. By Inheritance
Inherited
  1. StreamingContext
  2. Logging
  3. AnyRef
  4. Any
  1. Hide All
  2. Show All
Visibility
  1. Public
  2. Protected

Instance Constructors

  1. new StreamingContext(path: String, sparkContext: SparkContext)

    Recreate a StreamingContext from a checkpoint file using an existing SparkContext.

    Recreate a StreamingContext from a checkpoint file using an existing SparkContext.

    path

    Path to the directory that was specified as the checkpoint directory

    sparkContext

    Existing SparkContext

  2. new StreamingContext(path: String)

    Recreate a StreamingContext from a checkpoint file.

    Recreate a StreamingContext from a checkpoint file.

    path

    Path to the directory that was specified as the checkpoint directory

  3. new StreamingContext(path: String, hadoopConf: Configuration)

    Recreate a StreamingContext from a checkpoint file.

    Recreate a StreamingContext from a checkpoint file.

    path

    Path to the directory that was specified as the checkpoint directory

    hadoopConf

    Optional, configuration object if necessary for reading from HDFS compatible filesystems

  4. new StreamingContext(master: String, appName: String, batchDuration: Duration, sparkHome: String = null, jars: Seq[String] = Nil, environment: Map[String, String] = Map())

    Create a StreamingContext by providing the details necessary for creating a new SparkContext.

    Create a StreamingContext by providing the details necessary for creating a new SparkContext.

    master

    cluster URL to connect to (e.g. spark://host:port, local[4]).

    appName

    a name for your job, to display on the cluster web UI

    batchDuration

    the time interval at which streaming data will be divided into batches

  5. new StreamingContext(conf: SparkConf, batchDuration: Duration)

    Create a StreamingContext by providing the configuration necessary for a new SparkContext.

    Create a StreamingContext by providing the configuration necessary for a new SparkContext.

    conf

    a org.apache.spark.SparkConf object specifying Spark parameters

    batchDuration

    the time interval at which streaming data will be divided into batches

  6. new StreamingContext(sparkContext: SparkContext, batchDuration: Duration)

    Create a StreamingContext using an existing SparkContext.

    Create a StreamingContext using an existing SparkContext.

    sparkContext

    existing SparkContext

    batchDuration

    the time interval at which streaming data will be divided into batches

Type Members

  1. implicit class LogStringContext extends AnyRef
    Definition Classes
    Logging

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. def addStreamingListener(streamingListener: StreamingListener): Unit

    Add a org.apache.spark.streaming.scheduler.StreamingListener object for receiving system events related to streaming.

  5. final def asInstanceOf[T0]: T0
    Definition Classes
    Any
  6. def awaitTermination(): Unit

    Wait for the execution to stop.

    Wait for the execution to stop. Any exceptions that occurs during the execution will be thrown in this thread.

  7. def awaitTerminationOrTimeout(timeout: Long): Boolean

    Wait for the execution to stop.

    Wait for the execution to stop. Any exceptions that occurs during the execution will be thrown in this thread.

    timeout

    time to wait in milliseconds

    returns

    true if it's stopped; or throw the reported error during the execution; or false if the waiting time elapsed before returning from the method.

  8. def binaryRecordsStream(directory: String, recordLength: Int): DStream[Array[Byte]]

    Create an input stream that monitors a Hadoop-compatible filesystem for new files and reads them as flat binary files, assuming a fixed length per record, generating one byte array per record.

    Create an input stream that monitors a Hadoop-compatible filesystem for new files and reads them as flat binary files, assuming a fixed length per record, generating one byte array per record. Files must be written to the monitored directory by "moving" them from another location within the same file system. File names starting with . are ignored.

    directory

    HDFS directory to monitor for new file

    recordLength

    length of each record in bytes

    Note

    We ensure that the byte array for each record in the resulting RDDs of the DStream has the provided record length.

  9. def checkpoint(directory: String): Unit

    Set the context to periodically checkpoint the DStream operations for driver fault-tolerance.

    Set the context to periodically checkpoint the DStream operations for driver fault-tolerance.

    directory

    HDFS-compatible directory where the checkpoint data will be reliably stored. Note that this must be a fault-tolerant file system like HDFS.

  10. def clone(): AnyRef
    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.CloneNotSupportedException]) @IntrinsicCandidate() @native()
  11. final def eq(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  12. def equals(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef → Any
  13. def fileStream[K, V, F <: InputFormat[K, V]](directory: String, filter: (Path) => Boolean, newFilesOnly: Boolean, conf: Configuration)(implicit arg0: ClassTag[K], arg1: ClassTag[V], arg2: ClassTag[F]): InputDStream[(K, V)]

    Create an input stream that monitors a Hadoop-compatible filesystem for new files and reads them using the given key-value types and input format.

    Create an input stream that monitors a Hadoop-compatible filesystem for new files and reads them using the given key-value types and input format. Files must be written to the monitored directory by "moving" them from another location within the same file system. File names starting with . are ignored.

    K

    Key type for reading HDFS file

    V

    Value type for reading HDFS file

    F

    Input format for reading HDFS file

    directory

    HDFS directory to monitor for new file

    filter

    Function to filter paths to process

    newFilesOnly

    Should process only new files and ignore existing files in the directory

    conf

    Hadoop configuration

  14. def fileStream[K, V, F <: InputFormat[K, V]](directory: String, filter: (Path) => Boolean, newFilesOnly: Boolean)(implicit arg0: ClassTag[K], arg1: ClassTag[V], arg2: ClassTag[F]): InputDStream[(K, V)]

    Create an input stream that monitors a Hadoop-compatible filesystem for new files and reads them using the given key-value types and input format.

    Create an input stream that monitors a Hadoop-compatible filesystem for new files and reads them using the given key-value types and input format. Files must be written to the monitored directory by "moving" them from another location within the same file system.

    K

    Key type for reading HDFS file

    V

    Value type for reading HDFS file

    F

    Input format for reading HDFS file

    directory

    HDFS directory to monitor for new file

    filter

    Function to filter paths to process

    newFilesOnly

    Should process only new files and ignore existing files in the directory

  15. def fileStream[K, V, F <: InputFormat[K, V]](directory: String)(implicit arg0: ClassTag[K], arg1: ClassTag[V], arg2: ClassTag[F]): InputDStream[(K, V)]

    Create an input stream that monitors a Hadoop-compatible filesystem for new files and reads them using the given key-value types and input format.

    Create an input stream that monitors a Hadoop-compatible filesystem for new files and reads them using the given key-value types and input format. Files must be written to the monitored directory by "moving" them from another location within the same file system. File names starting with . are ignored.

    K

    Key type for reading HDFS file

    V

    Value type for reading HDFS file

    F

    Input format for reading HDFS file

    directory

    HDFS directory to monitor for new file

  16. final def getClass(): Class[_ <: AnyRef]
    Definition Classes
    AnyRef → Any
    Annotations
    @IntrinsicCandidate() @native()
  17. def getState(): StreamingContextState

    :: DeveloperApi ::

    :: DeveloperApi ::

    Return the current state of the context. The context can be in three possible states -

    • StreamingContextState.INITIALIZED - The context has been created, but not started yet. Input DStreams, transformations and output operations can be created on the context.
    • StreamingContextState.ACTIVE - The context has been started, and not stopped. Input DStreams, transformations and output operations cannot be created on the context.
    • StreamingContextState.STOPPED - The context has been stopped and cannot be used any more.
    Annotations
    @DeveloperApi()
  18. def hashCode(): Int
    Definition Classes
    AnyRef → Any
    Annotations
    @IntrinsicCandidate() @native()
  19. def initializeLogIfNecessary(isInterpreter: Boolean, silent: Boolean): Boolean
    Attributes
    protected
    Definition Classes
    Logging
  20. def initializeLogIfNecessary(isInterpreter: Boolean): Unit
    Attributes
    protected
    Definition Classes
    Logging
  21. final def isInstanceOf[T0]: Boolean
    Definition Classes
    Any
  22. def isTraceEnabled(): Boolean
    Attributes
    protected
    Definition Classes
    Logging
  23. def log: Logger
    Attributes
    protected
    Definition Classes
    Logging
  24. def logDebug(msg: => String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  25. def logDebug(entry: LogEntry, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  26. def logDebug(entry: LogEntry): Unit
    Attributes
    protected
    Definition Classes
    Logging
  27. def logDebug(msg: => String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  28. def logError(msg: => String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  29. def logError(entry: LogEntry, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  30. def logError(entry: LogEntry): Unit
    Attributes
    protected
    Definition Classes
    Logging
  31. def logError(msg: => String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  32. def logInfo(msg: => String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  33. def logInfo(entry: LogEntry, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  34. def logInfo(entry: LogEntry): Unit
    Attributes
    protected
    Definition Classes
    Logging
  35. def logInfo(msg: => String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  36. def logName: String
    Attributes
    protected
    Definition Classes
    Logging
  37. def logTrace(msg: => String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  38. def logTrace(entry: LogEntry, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  39. def logTrace(entry: LogEntry): Unit
    Attributes
    protected
    Definition Classes
    Logging
  40. def logTrace(msg: => String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  41. def logWarning(msg: => String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  42. def logWarning(entry: LogEntry, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  43. def logWarning(entry: LogEntry): Unit
    Attributes
    protected
    Definition Classes
    Logging
  44. def logWarning(msg: => String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  45. final def ne(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  46. final def notify(): Unit
    Definition Classes
    AnyRef
    Annotations
    @IntrinsicCandidate() @native()
  47. final def notifyAll(): Unit
    Definition Classes
    AnyRef
    Annotations
    @IntrinsicCandidate() @native()
  48. def queueStream[T](queue: Queue[RDD[T]], oneAtATime: Boolean, defaultRDD: RDD[T])(implicit arg0: ClassTag[T]): InputDStream[T]

    Create an input stream from a queue of RDDs.

    Create an input stream from a queue of RDDs. In each batch, it will process either one or all of the RDDs returned by the queue.

    T

    Type of objects in the RDD

    queue

    Queue of RDDs. Modifications to this data structure must be synchronized.

    oneAtATime

    Whether only one RDD should be consumed from the queue in every interval

    defaultRDD

    Default RDD is returned by the DStream when the queue is empty. Set as null if no RDD should be returned when empty

    Note

    Arbitrary RDDs can be added to queueStream, there is no way to recover data of those RDDs, so queueStream doesn't support checkpointing.

  49. def queueStream[T](queue: Queue[RDD[T]], oneAtATime: Boolean = true)(implicit arg0: ClassTag[T]): InputDStream[T]

    Create an input stream from a queue of RDDs.

    Create an input stream from a queue of RDDs. In each batch, it will process either one or all of the RDDs returned by the queue.

    T

    Type of objects in the RDD

    queue

    Queue of RDDs. Modifications to this data structure must be synchronized.

    oneAtATime

    Whether only one RDD should be consumed from the queue in every interval

    Note

    Arbitrary RDDs can be added to queueStream, there is no way to recover data of those RDDs, so queueStream doesn't support checkpointing.

  50. def rawSocketStream[T](hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2)(implicit arg0: ClassTag[T]): ReceiverInputDStream[T]

    Create an input stream from network source hostname:port, where data is received as serialized blocks (serialized using the Spark's serializer) that can be directly pushed into the block manager without deserializing them.

    Create an input stream from network source hostname:port, where data is received as serialized blocks (serialized using the Spark's serializer) that can be directly pushed into the block manager without deserializing them. This is the most efficient way to receive data.

    T

    Type of the objects in the received blocks

    hostname

    Hostname to connect to for receiving data

    port

    Port to connect to for receiving data

    storageLevel

    Storage level to use for storing the received objects (default: StorageLevel.MEMORY_AND_DISK_SER_2)

  51. def receiverStream[T](receiver: Receiver[T])(implicit arg0: ClassTag[T]): ReceiverInputDStream[T]

    Create an input stream with any arbitrary user implemented receiver.

    Create an input stream with any arbitrary user implemented receiver. Find more details at https://spark.apache.org/docs/latest/streaming-custom-receivers.html

    receiver

    Custom implementation of Receiver

  52. def remember(duration: Duration): Unit

    Set each DStream in this context to remember RDDs it generated in the last given duration.

    Set each DStream in this context to remember RDDs it generated in the last given duration. DStreams remember RDDs only for a limited duration of time and release them for garbage collection. This method allows the developer to specify how long to remember the RDDs ( if the developer wishes to query old data outside the DStream computation).

    duration

    Minimum duration that each DStream should remember its RDDs

  53. def removeStreamingListener(streamingListener: StreamingListener): Unit
  54. def socketStream[T](hostname: String, port: Int, converter: (InputStream) => Iterator[T], storageLevel: StorageLevel)(implicit arg0: ClassTag[T]): ReceiverInputDStream[T]

    Creates an input stream from TCP source hostname:port.

    Creates an input stream from TCP source hostname:port. Data is received using a TCP socket and the receive bytes it interpreted as object using the given converter.

    T

    Type of the objects received (after converting bytes to objects)

    hostname

    Hostname to connect to for receiving data

    port

    Port to connect to for receiving data

    converter

    Function to convert the byte stream to objects

    storageLevel

    Storage level to use for storing the received objects

  55. def socketTextStream(hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2): ReceiverInputDStream[String]

    Creates an input stream from TCP source hostname:port.

    Creates an input stream from TCP source hostname:port. Data is received using a TCP socket and the receive bytes is interpreted as UTF8 encoded \n delimited lines.

    hostname

    Hostname to connect to for receiving data

    port

    Port to connect to for receiving data

    storageLevel

    Storage level to use for storing the received objects (default: StorageLevel.MEMORY_AND_DISK_SER_2)

    See also

    socketStream

  56. def sparkContext: SparkContext

    Return the associated Spark context

  57. def start(): Unit

    Start the execution of the streams.

    Start the execution of the streams.

    Exceptions thrown

    IllegalStateException if the StreamingContext is already stopped.

  58. def stop(stopSparkContext: Boolean, stopGracefully: Boolean): Unit

    Stop the execution of the streams, with option of ensuring all received data has been processed.

    Stop the execution of the streams, with option of ensuring all received data has been processed.

    stopSparkContext

    if true, stops the associated SparkContext. The underlying SparkContext will be stopped regardless of whether this StreamingContext has been started.

    stopGracefully

    if true, stops gracefully by waiting for the processing of all received data to be completed

  59. def stop(stopSparkContext: Boolean = conf.getBoolean("spark.streaming.stopSparkContextByDefault", true)): Unit

    Stop the execution of the streams immediately (does not wait for all received data to be processed).

    Stop the execution of the streams immediately (does not wait for all received data to be processed). By default, if stopSparkContext is not specified, the underlying SparkContext will also be stopped. This implicit behavior can be configured using the SparkConf configuration spark.streaming.stopSparkContextByDefault.

    stopSparkContext

    If true, stops the associated SparkContext. The underlying SparkContext will be stopped regardless of whether this StreamingContext has been started.

  60. final def synchronized[T0](arg0: => T0): T0
    Definition Classes
    AnyRef
  61. def textFileStream(directory: String): DStream[String]

    Create an input stream that monitors a Hadoop-compatible filesystem for new files and reads them as text files (using key as LongWritable, value as Text and input format as TextInputFormat).

    Create an input stream that monitors a Hadoop-compatible filesystem for new files and reads them as text files (using key as LongWritable, value as Text and input format as TextInputFormat). Files must be written to the monitored directory by "moving" them from another location within the same file system. File names starting with . are ignored. The text files must be encoded as UTF-8.

    directory

    HDFS directory to monitor for new file

  62. def toString(): String
    Definition Classes
    AnyRef → Any
  63. def transform[T](dstreams: Seq[DStream[_]], transformFunc: (Seq[RDD[_]], Time) => RDD[T])(implicit arg0: ClassTag[T]): DStream[T]

    Create a new DStream in which each RDD is generated by applying a function on RDDs of the DStreams.

  64. def union[T](streams: Seq[DStream[T]])(implicit arg0: ClassTag[T]): DStream[T]

    Create a unified DStream from multiple DStreams of the same type and same slide duration.

  65. final def wait(arg0: Long, arg1: Int): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.InterruptedException])
  66. final def wait(arg0: Long): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.InterruptedException]) @native()
  67. final def wait(): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws(classOf[java.lang.InterruptedException])
  68. 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 AnyRef

Inherited from Any

Ungrouped