Packages

c

org.apache.spark.sql.streaming

DataStreamWriter

final class DataStreamWriter[T] extends AnyRef

Interface used to write a streaming Dataset to external storage systems (e.g. file systems, key-value stores, etc). Use Dataset.writeStream to access this.

Annotations
@Evolving()
Source
DataStreamWriter.scala
Since

2.0.0

Linear Supertypes
AnyRef, Any
Ordering
  1. Alphabetic
  2. By Inheritance
Inherited
  1. DataStreamWriter
  2. AnyRef
  3. Any
  1. Hide All
  2. Show All
Visibility
  1. Public
  2. All

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. def clone(): AnyRef
    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( ... ) @native()
  6. final def eq(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  7. def equals(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  8. def finalize(): Unit
    Attributes
    protected[lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( classOf[java.lang.Throwable] )
  9. def foreach(writer: ForeachWriter[T]): DataStreamWriter[T]

    Sets the output of the streaming query to be processed using the provided writer object.

    Sets the output of the streaming query to be processed using the provided writer object. object. See org.apache.spark.sql.ForeachWriter for more details on the lifecycle and semantics.

    Since

    2.0.0

  10. def foreachBatch(function: VoidFunction2[Dataset[T], Long]): DataStreamWriter[T]

    :: Experimental ::

    :: Experimental ::

    (Java-specific) Sets the output of the streaming query to be processed using the provided function. This is supported only the in the micro-batch execution modes (that is, when the trigger is not continuous). In every micro-batch, the provided function will be called in every micro-batch with (i) the output rows as a Dataset and (ii) the batch identifier. The batchId can be used deduplicate and transactionally write the output (that is, the provided Dataset) to external systems. The output Dataset is guaranteed to exactly same for the same batchId (assuming all operations are deterministic in the query).

    Annotations
    @Evolving()
    Since

    2.4.0

  11. def foreachBatch(function: (Dataset[T], Long) ⇒ Unit): DataStreamWriter[T]

    :: Experimental ::

    :: Experimental ::

    (Scala-specific) Sets the output of the streaming query to be processed using the provided function. This is supported only the in the micro-batch execution modes (that is, when the trigger is not continuous). In every micro-batch, the provided function will be called in every micro-batch with (i) the output rows as a Dataset and (ii) the batch identifier. The batchId can be used deduplicate and transactionally write the output (that is, the provided Dataset) to external systems. The output Dataset is guaranteed to exactly same for the same batchId (assuming all operations are deterministic in the query).

    Annotations
    @Evolving()
    Since

    2.4.0

  12. def format(source: String): DataStreamWriter[T]

    Specifies the underlying output data source.

    Specifies the underlying output data source.

    Since

    2.0.0

  13. final def getClass(): Class[_]
    Definition Classes
    AnyRef → Any
    Annotations
    @native()
  14. def hashCode(): Int
    Definition Classes
    AnyRef → Any
    Annotations
    @native()
  15. final def isInstanceOf[T0]: Boolean
    Definition Classes
    Any
  16. final def ne(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  17. final def notify(): Unit
    Definition Classes
    AnyRef
    Annotations
    @native()
  18. final def notifyAll(): Unit
    Definition Classes
    AnyRef
    Annotations
    @native()
  19. def option(key: String, value: Double): DataStreamWriter[T]

    Adds an output option for the underlying data source.

    Adds an output option for the underlying data source.

    Since

    2.0.0

  20. def option(key: String, value: Long): DataStreamWriter[T]

    Adds an output option for the underlying data source.

    Adds an output option for the underlying data source.

    Since

    2.0.0

  21. def option(key: String, value: Boolean): DataStreamWriter[T]

    Adds an output option for the underlying data source.

    Adds an output option for the underlying data source.

    Since

    2.0.0

  22. def option(key: String, value: String): DataStreamWriter[T]

    Adds an output option for the underlying data source.

    Adds an output option for the underlying data source.

    You can set the following option(s):

    • timeZone (default session local timezone): sets the string that indicates a time zone ID to be used to format timestamps in the JSON/CSV datasources or partition values. The following formats of timeZone are supported:
    • Region-based zone ID: It should have the form 'area/city', such as 'America/Los_Angeles'.
    • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.

    Other short names like 'CST' are not recommended to use because they can be ambiguous. If it isn't set, the current value of the SQL config spark.sql.session.timeZone is used by default.

    Since

    2.0.0

  23. def options(options: Map[String, String]): DataStreamWriter[T]

    Adds output options for the underlying data source.

    Adds output options for the underlying data source.

    You can set the following option(s):

    • timeZone (default session local timezone): sets the string that indicates a time zone ID to be used to format timestamps in the JSON/CSV datasources or partition values. The following formats of timeZone are supported:
    • Region-based zone ID: It should have the form 'area/city', such as 'America/Los_Angeles'.
    • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.

    Other short names like 'CST' are not recommended to use because they can be ambiguous. If it isn't set, the current value of the SQL config spark.sql.session.timeZone is used by default.

    Since

    2.0.0

  24. def options(options: Map[String, String]): DataStreamWriter[T]

    (Scala-specific) Adds output options for the underlying data source.

    (Scala-specific) Adds output options for the underlying data source.

    You can set the following option(s):

    • timeZone (default session local timezone): sets the string that indicates a time zone ID to be used to format timestamps in the JSON/CSV datasources or partition values. The following formats of timeZone are supported:
    • Region-based zone ID: It should have the form 'area/city', such as 'America/Los_Angeles'.
    • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.

    Other short names like 'CST' are not recommended to use because they can be ambiguous. If it isn't set, the current value of the SQL config spark.sql.session.timeZone is used by default.

    Since

    2.0.0

  25. def outputMode(outputMode: String): DataStreamWriter[T]

    Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink.

    Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink.

    • append: only the new rows in the streaming DataFrame/Dataset will be written to the sink.
    • complete: all the rows in the streaming DataFrame/Dataset will be written to the sink every time there are some updates.
    • update: only the rows that were updated in the streaming DataFrame/Dataset will be written to the sink every time there are some updates. If the query doesn't contain aggregations, it will be equivalent to append mode.
    Since

    2.0.0

  26. def outputMode(outputMode: OutputMode): DataStreamWriter[T]

    Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink.

    Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink.

    • OutputMode.Append(): only the new rows in the streaming DataFrame/Dataset will be written to the sink.
    • OutputMode.Complete(): all the rows in the streaming DataFrame/Dataset will be written to the sink every time there are some updates.
    • OutputMode.Update(): only the rows that were updated in the streaming DataFrame/Dataset will be written to the sink every time there are some updates. If the query doesn't contain aggregations, it will be equivalent to OutputMode.Append() mode.
    Since

    2.0.0

  27. def partitionBy(colNames: String*): DataStreamWriter[T]

    Partitions the output by the given columns on the file system.

    Partitions the output by the given columns on the file system. If specified, the output is laid out on the file system similar to Hive's partitioning scheme. As an example, when we partition a dataset by year and then month, the directory layout would look like:

    • year=2016/month=01/
    • year=2016/month=02/

    Partitioning is one of the most widely used techniques to optimize physical data layout. It provides a coarse-grained index for skipping unnecessary data reads when queries have predicates on the partitioned columns. In order for partitioning to work well, the number of distinct values in each column should typically be less than tens of thousands.

    Annotations
    @varargs()
    Since

    2.0.0

  28. def queryName(queryName: String): DataStreamWriter[T]

    Specifies the name of the StreamingQuery that can be started with start().

    Specifies the name of the StreamingQuery that can be started with start(). This name must be unique among all the currently active queries in the associated SQLContext.

    Since

    2.0.0

  29. def start(): StreamingQuery

    Starts the execution of the streaming query, which will continually output results to the given path as new data arrives.

    Starts the execution of the streaming query, which will continually output results to the given path as new data arrives. The returned StreamingQuery object can be used to interact with the stream. Throws a TimeoutException if the following conditions are met:

    • Another run of the same streaming query, that is a streaming query sharing the same checkpoint location, is already active on the same Spark Driver
    • The SQL configuration spark.sql.streaming.stopActiveRunOnRestart is enabled
    • The active run cannot be stopped within the timeout controlled by the SQL configuration spark.sql.streaming.stopTimeout
    Annotations
    @throws( ... )
    Since

    2.0.0

  30. def start(path: String): StreamingQuery

    Starts the execution of the streaming query, which will continually output results to the given path as new data arrives.

    Starts the execution of the streaming query, which will continually output results to the given path as new data arrives. The returned StreamingQuery object can be used to interact with the stream.

    Since

    2.0.0

  31. final def synchronized[T0](arg0: ⇒ T0): T0
    Definition Classes
    AnyRef
  32. def toString(): String
    Definition Classes
    AnyRef → Any
  33. def trigger(trigger: Trigger): DataStreamWriter[T]

    Set the trigger for the stream query.

    Set the trigger for the stream query. The default value is ProcessingTime(0) and it will run the query as fast as possible.

    Scala Example:

    df.writeStream.trigger(ProcessingTime("10 seconds"))
    
    import scala.concurrent.duration._
    df.writeStream.trigger(ProcessingTime(10.seconds))

    Java Example:

    df.writeStream().trigger(ProcessingTime.create("10 seconds"))
    
    import java.util.concurrent.TimeUnit
    df.writeStream().trigger(ProcessingTime.create(10, TimeUnit.SECONDS))
    Since

    2.0.0

  34. final def wait(): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  35. final def wait(arg0: Long, arg1: Int): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  36. final def wait(arg0: Long): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws( ... ) @native()

Inherited from AnyRef

Inherited from Any

Ungrouped