Hot-keys on this page
r m x p toggle line displays
j k next/prev highlighted chunk
0 (zero) top of page
1 (one) first highlighted chunk
# # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with # this work for additional information regarding copyright ownership. # The ASF licenses this file to You under the Apache License, Version 2.0 # (the "License"); you may not use this file except in compliance with # the License. You may obtain a copy of the License at # # http://www.apache.org/licenses/LICENSE-2.0 # # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. #
""" A handle to a query that is executing continuously in the background as new data arrives. All these methods are thread-safe.
.. versionadded:: 2.0.0
Notes ----- This API is evolving. """
def id(self): """Returns the unique id of this query that persists across restarts from checkpoint data. That is, this id is generated when a query is started for the first time, and will be the same every time it is restarted from checkpoint data. There can only be one query with the same id active in a Spark cluster. Also see, `runId`. """
def runId(self): """Returns the unique id of this query that does not persist across restarts. That is, every query that is started (or restarted from checkpoint) will have a different runId. """ return self._jsq.runId().toString()
def name(self): """Returns the user-specified name of the query, or null if not specified. This name can be specified in the `org.apache.spark.sql.streaming.DataStreamWriter` as `dataframe.writeStream.queryName("query").start()`. This name, if set, must be unique across all active queries. """
def isActive(self): """Whether this streaming query is currently active or not. """
"""Waits for the termination of `this` query, either by :func:`query.stop()` or by an exception. If the query has terminated with an exception, then the exception will be thrown. If `timeout` is set, it returns whether the query has terminated or not within the `timeout` seconds.
If the query has terminated, then all subsequent calls to this method will either return immediately (if the query was terminated by :func:`stop()`), or throw the exception immediately (if the query has terminated with exception).
throws :class:`StreamingQueryException`, if `this` query has terminated with an exception """ else: return self._jsq.awaitTermination()
def status(self): """ Returns the current status of the query. """
def recentProgress(self): """Returns an array of the most recent [[StreamingQueryProgress]] updates for this query. The number of progress updates retained for each stream is configured by Spark session configuration `spark.sql.streaming.numRecentProgressUpdates`. """
def lastProgress(self): """ Returns the most recent :class:`StreamingQueryProgress` update of this streaming query or None if there were no progress updates
.. versionadded:: 2.1.0
Returns ------- dict """ else:
"""Blocks until all available data in the source has been processed and committed to the sink. This method is intended for testing.
.. versionadded:: 2.0.0
Notes ----- In the case of continually arriving data, this method may block forever. Additionally, this method is only guaranteed to block until data that has been synchronously appended data to a stream source prior to invocation. (i.e. `getOffset` must immediately reflect the addition). """
def stop(self): """Stop this streaming query. """
"""Prints the (logical and physical) plans to the console for debugging purpose.
.. versionadded:: 2.1.0
Parameters ---------- extended : bool, optional default ``False``. If ``False``, prints only the physical plan.
Examples -------- >>> sq = sdf.writeStream.format('memory').queryName('query_explain').start() >>> sq.processAllAvailable() # Wait a bit to generate the runtime plans. >>> sq.explain() == Physical Plan == ... >>> sq.explain(True) == Parsed Logical Plan == ... == Analyzed Logical Plan == ... == Optimized Logical Plan == ... == Physical Plan == ... >>> sq.stop() """ # Cannot call `_jsq.explain(...)` because it will print in the JVM process. # We should print it in the Python process.
""" .. versionadded:: 2.1.0
Returns ------- :class:`StreamingQueryException` the StreamingQueryException if the query was terminated by an exception, or None. """ else:
"""A class to manage all the :class:`StreamingQuery` StreamingQueries active.
.. versionadded:: 2.0.0
Notes ----- This API is evolving. """
def active(self): """Returns a list of active queries associated with this SQLContext
.. versionadded:: 2.0.0
Examples -------- >>> sq = sdf.writeStream.format('memory').queryName('this_query').start() >>> sqm = spark.streams >>> # get the list of active streaming queries >>> [q.name for q in sqm.active] ['this_query'] >>> sq.stop() """
"""Returns an active query from this SQLContext or throws exception if an active query with this name doesn't exist.
.. versionadded:: 2.0.0
Examples -------- >>> sq = sdf.writeStream.format('memory').queryName('this_query').start() >>> sq.name 'this_query' >>> sq = spark.streams.get(sq.id) >>> sq.isActive True >>> sq = sqlContext.streams.get(sq.id) >>> sq.isActive True >>> sq.stop() """
"""Wait until any of the queries on the associated SQLContext has terminated since the creation of the context, or since :func:`resetTerminated()` was called. If any query was terminated with an exception, then the exception will be thrown. If `timeout` is set, it returns whether the query has terminated or not within the `timeout` seconds.
If a query has terminated, then subsequent calls to :func:`awaitAnyTermination()` will either return immediately (if the query was terminated by :func:`query.stop()`), or throw the exception immediately (if the query was terminated with exception). Use :func:`resetTerminated()` to clear past terminations and wait for new terminations.
In the case where multiple queries have terminated since :func:`resetTermination()` was called, if any query has terminated with exception, then :func:`awaitAnyTermination()` will throw any of the exception. For correctly documenting exceptions across multiple queries, users need to stop all of them after any of them terminates with exception, and then check the `query.exception()` for each query.
throws :class:`StreamingQueryException`, if `this` query has terminated with an exception """ else: return self._jsqm.awaitAnyTermination()
"""Forget about past terminated queries so that :func:`awaitAnyTermination()` can be used again to wait for new terminations.
.. versionadded:: 2.0.0
Examples -------- >>> spark.streams.resetTerminated() """
""" Interface used to load a streaming :class:`DataFrame <pyspark.sql.DataFrame>` from external storage systems (e.g. file systems, key-value stores, etc). Use :attr:`SparkSession.readStream <pyspark.sql.SparkSession.readStream>` to access this.
.. versionadded:: 2.0.0
Notes ----- This API is evolving. """
"""Specifies the input data source format.
.. versionadded:: 2.0.0
Parameters ---------- source : str name of the data source, e.g. 'json', 'parquet'.
Notes ----- This API is evolving.
Examples -------- >>> s = spark.readStream.format("text") """
"""Specifies the input schema.
Some data sources (e.g. JSON) can infer the input schema automatically from data. By specifying the schema here, the underlying data source can skip the schema inference step, and thus speed up data loading.
.. versionadded:: 2.0.0
Parameters ---------- schema : :class:`pyspark.sql.types.StructType` or str a :class:`pyspark.sql.types.StructType` object or a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``).
Notes ----- This API is evolving.
Examples -------- >>> s = spark.readStream.schema(sdf_schema) >>> s = spark.readStream.schema("col0 INT, col1 DOUBLE") """ else: raise TypeError("schema should be StructType or string")
"""Adds an input option for the underlying data source.
.. versionadded:: 2.0.0
Notes ----- This API is evolving.
Examples -------- >>> s = spark.readStream.option("x", 1) """
"""Adds input options for the underlying data source.
.. versionadded:: 2.0.0
Notes ----- This API is evolving.
Examples -------- >>> s = spark.readStream.options(x="1", y=2) """
"""Loads a data stream from a data source and returns it as a :class:`DataFrame <pyspark.sql.DataFrame>`.
.. versionadded:: 2.0.0
Parameters ---------- path : str, optional optional string for file-system backed data sources. format : str, optional optional string for format of the data source. Default to 'parquet'. schema : :class:`pyspark.sql.types.StructType` or str, optional optional :class:`pyspark.sql.types.StructType` for the input schema or a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``). **options : dict all other string options
Notes ----- This API is evolving.
Examples -------- >>> json_sdf = spark.readStream.format("json") \\ ... .schema(sdf_schema) \\ ... .load(tempfile.mkdtemp()) >>> json_sdf.isStreaming True >>> json_sdf.schema == sdf_schema True """ raise ValueError("If the path is provided for stream, it needs to be a " + "non-empty string. List of paths are not supported.") else:
allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None, allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None, multiLine=None, allowUnquotedControlChars=None, lineSep=None, locale=None, dropFieldIfAllNull=None, encoding=None, pathGlobFilter=None, recursiveFileLookup=None, allowNonNumericNumbers=None): """ Loads a JSON file stream and returns the results as a :class:`DataFrame`.
`JSON Lines <http://jsonlines.org/>`_ (newline-delimited JSON) is supported by default. For JSON (one record per file), set the ``multiLine`` parameter to ``true``.
If the ``schema`` parameter is not specified, this function goes through the input once to determine the input schema.
.. versionadded:: 2.0.0
Parameters ---------- path : str string represents path to the JSON dataset, or RDD of Strings storing JSON objects. schema : :class:`pyspark.sql.types.StructType` or str, optional an optional :class:`pyspark.sql.types.StructType` for the input schema or a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``).
Other Parameters ---------------- Extra options For the extra options, refer to `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_ in the version you use.
.. # noqa
Notes ----- This API is evolving.
Examples -------- >>> json_sdf = spark.readStream.json(tempfile.mkdtemp(), schema = sdf_schema) >>> json_sdf.isStreaming True >>> json_sdf.schema == sdf_schema True """ schema=schema, primitivesAsString=primitivesAsString, prefersDecimal=prefersDecimal, allowComments=allowComments, allowUnquotedFieldNames=allowUnquotedFieldNames, allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero, allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter, mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat, timestampFormat=timestampFormat, multiLine=multiLine, allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep, locale=locale, dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup, allowNonNumericNumbers=allowNonNumericNumbers) else: raise TypeError("path can be only a single string")
"""Loads a ORC file stream, returning the result as a :class:`DataFrame`.
.. versionadded:: 2.3.0
Other Parameters ---------------- Extra options For the extra options, refer to `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-orc.html#data-source-option>`_ in the version you use.
.. # noqa
Examples -------- >>> orc_sdf = spark.readStream.schema(sdf_schema).orc(tempfile.mkdtemp()) >>> orc_sdf.isStreaming True >>> orc_sdf.schema == sdf_schema True """ recursiveFileLookup=recursiveFileLookup) else: raise TypeError("path can be only a single string")
datetimeRebaseMode=None, int96RebaseMode=None): """ Loads a Parquet file stream, returning the result as a :class:`DataFrame`.
.. versionadded:: 2.0.0
Parameters ---------- path : str the path in any Hadoop supported file system
Other Parameters ---------------- Extra options For the extra options, refer to `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_. in the version you use.
.. # noqa
Examples -------- >>> parquet_sdf = spark.readStream.schema(sdf_schema).parquet(tempfile.mkdtemp()) >>> parquet_sdf.isStreaming True >>> parquet_sdf.schema == sdf_schema True """ recursiveFileLookup=recursiveFileLookup, datetimeRebaseMode=datetimeRebaseMode, int96RebaseMode=int96RebaseMode) else: raise TypeError("path can be only a single string")
recursiveFileLookup=None): """ Loads a text file stream and returns a :class:`DataFrame` whose schema starts with a string column named "value", and followed by partitioned columns if there are any. The text files must be encoded as UTF-8.
By default, each line in the text file is a new row in the resulting DataFrame.
.. versionadded:: 2.0.0
Parameters ---------- paths : str or list string, or list of strings, for input path(s).
Other Parameters ---------------- Extra options For the extra options, refer to `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_ in the version you use.
.. # noqa
Notes ----- This API is evolving.
Examples -------- >>> text_sdf = spark.readStream.text(tempfile.mkdtemp()) >>> text_sdf.isStreaming True >>> "value" in str(text_sdf.schema) True """ wholetext=wholetext, lineSep=lineSep, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) else: raise TypeError("path can be only a single string")
comment=None, header=None, inferSchema=None, ignoreLeadingWhiteSpace=None, ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None, negativeInf=None, dateFormat=None, timestampFormat=None, maxColumns=None, maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None, columnNameOfCorruptRecord=None, multiLine=None, charToEscapeQuoteEscaping=None, enforceSchema=None, emptyValue=None, locale=None, lineSep=None, pathGlobFilter=None, recursiveFileLookup=None, unescapedQuoteHandling=None): r"""Loads a CSV file stream and returns the result as a :class:`DataFrame`.
This function will go through the input once to determine the input schema if ``inferSchema`` is enabled. To avoid going through the entire data once, disable ``inferSchema`` option or specify the schema explicitly using ``schema``.
Parameters ---------- path : str or list string, or list of strings, for input path(s). schema : :class:`pyspark.sql.types.StructType` or str, optional an optional :class:`pyspark.sql.types.StructType` for the input schema or a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``).
.. versionadded:: 2.0.0
Other Parameters ---------------- Extra options For the extra options, refer to `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_ in the version you use.
.. # noqa
Notes ----- This API is evolving.
Examples -------- >>> csv_sdf = spark.readStream.csv(tempfile.mkdtemp(), schema = sdf_schema) >>> csv_sdf.isStreaming True >>> csv_sdf.schema == sdf_schema True """ schema=schema, sep=sep, encoding=encoding, quote=quote, escape=escape, comment=comment, header=header, inferSchema=inferSchema, ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace, ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace, nullValue=nullValue, nanValue=nanValue, positiveInf=positiveInf, negativeInf=negativeInf, dateFormat=dateFormat, timestampFormat=timestampFormat, maxColumns=maxColumns, maxCharsPerColumn=maxCharsPerColumn, maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, multiLine=multiLine, charToEscapeQuoteEscaping=charToEscapeQuoteEscaping, enforceSchema=enforceSchema, emptyValue=emptyValue, locale=locale, lineSep=lineSep, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup, unescapedQuoteHandling=unescapedQuoteHandling) else: raise TypeError("path can be only a single string")
"""Define a Streaming DataFrame on a Table. The DataSource corresponding to the table should support streaming mode.
.. versionadded:: 3.1.0
Parameters ---------- tableName : str string, for the name of the table.
Returns -------- :class:`DataFrame`
Notes ----- This API is evolving.
Examples -------- >>> spark.readStream.table('input_table') # doctest: +SKIP """ else: raise TypeError("tableName can be only a single string")
""" Interface used to write a streaming :class:`DataFrame <pyspark.sql.DataFrame>` to external storage systems (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.writeStream <pyspark.sql.DataFrame.writeStream>` to access this.
.. versionadded:: 2.0.0
Notes ----- This API is evolving. """
"""Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink.
.. versionadded:: 2.0.0
Options include:
* `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 these 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.
Notes ----- This API is evolving.
Examples -------- >>> writer = sdf.writeStream.outputMode('append') """ raise ValueError('The output mode must be a non-empty string. Got: %s' % outputMode)
"""Specifies the underlying output data source.
.. versionadded:: 2.0.0
Parameters ---------- source : str string, name of the data source, which for now can be 'parquet'.
Notes ----- This API is evolving.
Examples -------- >>> writer = sdf.writeStream.format('json') """
"""Adds an output option for the underlying data source.
.. versionadded:: 2.0.0
Notes ----- This API is evolving. """
"""Adds output options for the underlying data source.
.. versionadded:: 2.0.0
Notes ----- This API is evolving. """
"""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.
.. versionadded:: 2.0.0
Parameters ---------- cols : str or list name of columns
Notes ----- This API is evolving. """ cols = cols[0]
"""Specifies the name of the :class:`StreamingQuery` that can be started with :func:`start`. This name must be unique among all the currently active queries in the associated SparkSession.
.. versionadded:: 2.0.0
Parameters ---------- queryName : str unique name for the query
Notes ----- This API is evolving.
Examples -------- >>> writer = sdf.writeStream.queryName('streaming_query') """ raise ValueError('The queryName must be a non-empty string. Got: %s' % queryName)
"""Set the trigger for the stream query. If this is not set it will run the query as fast as possible, which is equivalent to setting the trigger to ``processingTime='0 seconds'``.
.. versionadded:: 2.0.0
Parameters ---------- processingTime : str, optional a processing time interval as a string, e.g. '5 seconds', '1 minute'. Set a trigger that runs a microbatch query periodically based on the processing time. Only one trigger can be set. once : bool, optional if set to True, set a trigger that processes only one batch of data in a streaming query then terminates the query. Only one trigger can be set. continuous : str, optional a time interval as a string, e.g. '5 seconds', '1 minute'. Set a trigger that runs a continuous query with a given checkpoint interval. Only one trigger can be set.
Notes ----- This API is evolving.
Examples -------- >>> # trigger the query for execution every 5 seconds >>> writer = sdf.writeStream.trigger(processingTime='5 seconds') >>> # trigger the query for just once batch of data >>> writer = sdf.writeStream.trigger(once=True) >>> # trigger the query for execution every 5 seconds >>> writer = sdf.writeStream.trigger(continuous='5 seconds') """
raise ValueError('Value for processingTime must be a non empty string. Got: %s' % processingTime) interval)
raise ValueError('Value for once must be True. Got: %s' % once)
else: raise ValueError('Value for continuous must be a non empty string. Got: %s' % continuous) interval)
""" Sets the output of the streaming query to be processed using the provided writer ``f``. This is often used to write the output of a streaming query to arbitrary storage systems. The processing logic can be specified in two ways.
#. A **function** that takes a row as input. This is a simple way to express your processing logic. Note that this does not allow you to deduplicate generated data when failures cause reprocessing of some input data. That would require you to specify the processing logic in the next way.
#. An **object** with a ``process`` method and optional ``open`` and ``close`` methods. The object can have the following methods.
* ``open(partition_id, epoch_id)``: *Optional* method that initializes the processing (for example, open a connection, start a transaction, etc). Additionally, you can use the `partition_id` and `epoch_id` to deduplicate regenerated data (discussed later).
* ``process(row)``: *Non-optional* method that processes each :class:`Row`.
* ``close(error)``: *Optional* method that finalizes and cleans up (for example, close connection, commit transaction, etc.) after all rows have been processed.
The object will be used by Spark in the following way.
* A single copy of this object is responsible of all the data generated by a single task in a query. In other words, one instance is responsible for processing one partition of the data generated in a distributed manner.
* This object must be serializable because each task will get a fresh serialized-deserialized copy of the provided object. Hence, it is strongly recommended that any initialization for writing data (e.g. opening a connection or starting a transaction) is done after the `open(...)` method has been called, which signifies that the task is ready to generate data.
* The lifecycle of the methods are as follows.
For each partition with ``partition_id``:
... For each batch/epoch of streaming data with ``epoch_id``:
....... Method ``open(partitionId, epochId)`` is called.
....... If ``open(...)`` returns true, for each row in the partition and batch/epoch, method ``process(row)`` is called.
....... Method ``close(errorOrNull)`` is called with error (if any) seen while processing rows.
Important points to note:
* The `partitionId` and `epochId` can be used to deduplicate generated data when failures cause reprocessing of some input data. This depends on the execution mode of the query. If the streaming query is being executed in the micro-batch mode, then every partition represented by a unique tuple (partition_id, epoch_id) is guaranteed to have the same data. Hence, (partition_id, epoch_id) can be used to deduplicate and/or transactionally commit data and achieve exactly-once guarantees. However, if the streaming query is being executed in the continuous mode, then this guarantee does not hold and therefore should not be used for deduplication.
* The ``close()`` method (if exists) will be called if `open()` method exists and returns successfully (irrespective of the return value), except if the Python crashes in the middle.
.. versionadded:: 2.4.0
Notes ----- This API is evolving.
Examples -------- >>> # Print every row using a function >>> def print_row(row): ... print(row) ... >>> writer = sdf.writeStream.foreach(print_row) >>> # Print every row using a object with process() method >>> class RowPrinter: ... def open(self, partition_id, epoch_id): ... print("Opened %d, %d" % (partition_id, epoch_id)) ... return True ... def process(self, row): ... print(row) ... def close(self, error): ... print("Closed with error: %s" % str(error)) ... >>> writer = sdf.writeStream.foreach(RowPrinter()) """
# The provided object is a callable function that is supposed to be called on each row. # Construct a function that takes an iterator and calls the provided function on each # row.
else: # The provided object is not a callable function. Then it is expected to have a # 'process(row)' method, and optional 'open(partition_id, epoch_id)' and # 'close(error)' methods.
"Attribute '%s' in provided object is not callable" % method_name)
else: raise RuntimeError("Could not get batch id from TaskContext")
# Check if the data should be processed
finally:
self._spark._sc._jvm.org.apache.spark.sql.execution.python.PythonForeachWriter( wrapped_func, self._df._jdf.schema())
""" 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 DataFrame 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 DataFrame is guaranteed to exactly same for the same batchId (assuming all operations are deterministic in the query).
.. versionadded:: 2.4.0
Notes ----- This API is evolving.
Examples -------- >>> def func(batch_df, batch_id): ... batch_df.collect() ... >>> writer = sdf.writeStream.foreachBatch(func) """
**options): """Streams the contents of the :class:`DataFrame` to a data source.
The data source is specified by the ``format`` and a set of ``options``. If ``format`` is not specified, the default data source configured by ``spark.sql.sources.default`` will be used.
.. versionadded:: 2.0.0
Parameters ---------- path : str, optional the path in a Hadoop supported file system format : str, optional the format used to save outputMode : str, optional 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 these 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. partitionBy : str or list, optional names of partitioning columns queryName : str, optional unique name for the query **options : dict All other string options. You may want to provide a `checkpointLocation` for most streams, however it is not required for a `memory` stream.
Notes ----- This API is evolving.
Examples -------- >>> sq = sdf.writeStream.format('memory').queryName('this_query').start() >>> sq.isActive True >>> sq.name 'this_query' >>> sq.stop() >>> sq.isActive False >>> sq = sdf.writeStream.trigger(processingTime='5 seconds').start( ... queryName='that_query', outputMode="append", format='memory') >>> sq.name 'that_query' >>> sq.isActive True >>> sq.stop() """ self.partitionBy(partitionBy) else:
**options): """ Starts the execution of the streaming query, which will continually output results to the given table as new data arrives.
The returned :class:`StreamingQuery` object can be used to interact with the stream.
.. versionadded:: 3.1.0
Parameters ---------- tableName : str string, for the name of the table. format : str, optional the format used to save. outputMode : str, optional 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 these 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. partitionBy : str or list, optional names of partitioning columns queryName : str, optional unique name for the query **options : dict All other string options. You may want to provide a `checkpointLocation`.
Notes ----- This API is evolving.
For v1 table, partitioning columns provided by `partitionBy` will be respected no matter the table exists or not. A new table will be created if the table not exists.
For v2 table, `partitionBy` will be ignored if the table already exists. `partitionBy` will be respected only if the v2 table does not exist. Besides, the v2 table created by this API lacks some functionalities (e.g., customized properties, options, and serde info). If you need them, please create the v2 table manually before the execution to avoid creating a table with incomplete information.
Examples -------- >>> sdf.writeStream.format('parquet').queryName('query').toTable('output_table') ... # doctest: +SKIP
>>> sdf.writeStream.trigger(processingTime='5 seconds').toTable( ... 'output_table', ... queryName='that_query', ... outputMode="append", ... format='parquet', ... checkpointLocation='/tmp/checkpoint') # doctest: +SKIP """ self.outputMode(outputMode) self.partitionBy(partitionBy) self.queryName(queryName)
except py4j.protocol.Py4JError: # noqa: F821 spark = SparkSession(sc) # noqa: F821
spark.readStream.format('text').load('python/test_support/sql/streaming') globs['spark'].readStream.format('text').load('python/test_support/sql/streaming')
pyspark.sql.streaming, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF)
sys.exit(-1)
|