spark-instrumented-optimizer/python/pyspark/sql/streaming.py

1628 lines
69 KiB
Python
Raw Normal View History

#
# 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.
#
import sys
[SPARK-18516][SQL] Split state and progress in streaming This PR separates the status of a `StreamingQuery` into two separate APIs: - `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available. - `recentProgress` - an array of statistics about the most recent microbatches that have executed. A recent progress contains the following information: ``` { "id" : "2be8670a-fce1-4859-a530-748f29553bb6", "name" : "query-29", "timestamp" : 1479705392724, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303, "durationMs" : { "triggerExecution" : 276, "queryPlanning" : 3, "getBatch" : 5, "getOffset" : 3, "addBatch" : 234, "walCommit" : 30 }, "currentWatermark" : 0, "stateOperators" : [ ], "sources" : [ { "description" : "KafkaSource[Subscribe[topic-14]]", "startOffset" : { "topic-14" : { "2" : 0, "4" : 1, "1" : 0, "3" : 0, "0" : 0 } }, "endOffset" : { "topic-14" : { "2" : 1, "4" : 2, "1" : 0, "3" : 0, "0" : 1 } }, "numRecords" : 3, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303 } ] } ``` Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Michael Armbrust <michael@databricks.com> Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
import json
from py4j.java_gateway import java_import
from pyspark import since, keyword_only
from pyspark.sql.column import _to_seq
from pyspark.sql.readwriter import OptionUtils, to_str
from pyspark.sql.types import StructType, StructField, StringType
from pyspark.sql.utils import ForeachBatchFunction, StreamingQueryException
__all__ = ["StreamingQuery", "StreamingQueryManager", "DataStreamReader", "DataStreamWriter"]
class StreamingQuery(object):
"""
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 __init__(self, jsq):
self._jsq = jsq
@property
@since(2.0)
def id(self):
[SPARK-18657][SPARK-18668] Make StreamingQuery.id persists across restart and not auto-generate StreamingQuery.name ## What changes were proposed in this pull request? Here are the major changes in this PR. - Added the ability to recover `StreamingQuery.id` from checkpoint location, by writing the id to `checkpointLoc/metadata`. - Added `StreamingQuery.runId` which is unique for every query started and does not persist across restarts. This is to identify each restart of a query separately (same as earlier behavior of `id`). - Removed auto-generation of `StreamingQuery.name`. The purpose of name was to have the ability to define an identifier across restarts, but since id is precisely that, there is no need for a auto-generated name. This means name becomes purely cosmetic, and is null by default. - Added `runId` to `StreamingQueryListener` events and `StreamingQueryProgress`. Implementation details - Renamed existing `StreamExecutionMetadata` to `OffsetSeqMetadata`, and moved it to the file `OffsetSeq.scala`, because that is what this metadata is tied to. Also did some refactoring to make the code cleaner (got rid of a lot of `.json` and `.getOrElse("{}")`). - Added the `id` as the new `StreamMetadata`. - When a StreamingQuery is created it gets or writes the `StreamMetadata` from `checkpointLoc/metadata`. - All internal logging in `StreamExecution` uses `(name, id, runId)` instead of just `name` TODO - [x] Test handling of name=null in json generation of StreamingQueryProgress - [x] Test handling of name=null in json generation of StreamingQueryListener events - [x] Test python API of runId ## How was this patch tested? Updated unit tests and new unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16113 from tdas/SPARK-18657.
2016-12-05 21:17:38 -05:00
"""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`.
"""
[SPARK-18516][SQL] Split state and progress in streaming This PR separates the status of a `StreamingQuery` into two separate APIs: - `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available. - `recentProgress` - an array of statistics about the most recent microbatches that have executed. A recent progress contains the following information: ``` { "id" : "2be8670a-fce1-4859-a530-748f29553bb6", "name" : "query-29", "timestamp" : 1479705392724, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303, "durationMs" : { "triggerExecution" : 276, "queryPlanning" : 3, "getBatch" : 5, "getOffset" : 3, "addBatch" : 234, "walCommit" : 30 }, "currentWatermark" : 0, "stateOperators" : [ ], "sources" : [ { "description" : "KafkaSource[Subscribe[topic-14]]", "startOffset" : { "topic-14" : { "2" : 0, "4" : 1, "1" : 0, "3" : 0, "0" : 0 } }, "endOffset" : { "topic-14" : { "2" : 1, "4" : 2, "1" : 0, "3" : 0, "0" : 1 } }, "numRecords" : 3, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303 } ] } ``` Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Michael Armbrust <michael@databricks.com> Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
return self._jsq.id().toString()
[SPARK-18657][SPARK-18668] Make StreamingQuery.id persists across restart and not auto-generate StreamingQuery.name ## What changes were proposed in this pull request? Here are the major changes in this PR. - Added the ability to recover `StreamingQuery.id` from checkpoint location, by writing the id to `checkpointLoc/metadata`. - Added `StreamingQuery.runId` which is unique for every query started and does not persist across restarts. This is to identify each restart of a query separately (same as earlier behavior of `id`). - Removed auto-generation of `StreamingQuery.name`. The purpose of name was to have the ability to define an identifier across restarts, but since id is precisely that, there is no need for a auto-generated name. This means name becomes purely cosmetic, and is null by default. - Added `runId` to `StreamingQueryListener` events and `StreamingQueryProgress`. Implementation details - Renamed existing `StreamExecutionMetadata` to `OffsetSeqMetadata`, and moved it to the file `OffsetSeq.scala`, because that is what this metadata is tied to. Also did some refactoring to make the code cleaner (got rid of a lot of `.json` and `.getOrElse("{}")`). - Added the `id` as the new `StreamMetadata`. - When a StreamingQuery is created it gets or writes the `StreamMetadata` from `checkpointLoc/metadata`. - All internal logging in `StreamExecution` uses `(name, id, runId)` instead of just `name` TODO - [x] Test handling of name=null in json generation of StreamingQueryProgress - [x] Test handling of name=null in json generation of StreamingQueryListener events - [x] Test python API of runId ## How was this patch tested? Updated unit tests and new unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16113 from tdas/SPARK-18657.
2016-12-05 21:17:38 -05:00
@property
@since(2.1)
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()
@property
@since(2.0)
def name(self):
[SPARK-18657][SPARK-18668] Make StreamingQuery.id persists across restart and not auto-generate StreamingQuery.name ## What changes were proposed in this pull request? Here are the major changes in this PR. - Added the ability to recover `StreamingQuery.id` from checkpoint location, by writing the id to `checkpointLoc/metadata`. - Added `StreamingQuery.runId` which is unique for every query started and does not persist across restarts. This is to identify each restart of a query separately (same as earlier behavior of `id`). - Removed auto-generation of `StreamingQuery.name`. The purpose of name was to have the ability to define an identifier across restarts, but since id is precisely that, there is no need for a auto-generated name. This means name becomes purely cosmetic, and is null by default. - Added `runId` to `StreamingQueryListener` events and `StreamingQueryProgress`. Implementation details - Renamed existing `StreamExecutionMetadata` to `OffsetSeqMetadata`, and moved it to the file `OffsetSeq.scala`, because that is what this metadata is tied to. Also did some refactoring to make the code cleaner (got rid of a lot of `.json` and `.getOrElse("{}")`). - Added the `id` as the new `StreamMetadata`. - When a StreamingQuery is created it gets or writes the `StreamMetadata` from `checkpointLoc/metadata`. - All internal logging in `StreamExecution` uses `(name, id, runId)` instead of just `name` TODO - [x] Test handling of name=null in json generation of StreamingQueryProgress - [x] Test handling of name=null in json generation of StreamingQueryListener events - [x] Test python API of runId ## How was this patch tested? Updated unit tests and new unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16113 from tdas/SPARK-18657.
2016-12-05 21:17:38 -05:00
"""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.
"""
return self._jsq.name()
@property
@since(2.0)
def isActive(self):
"""Whether this streaming query is currently active or not.
"""
return self._jsq.isActive()
@since(2.0)
def awaitTermination(self, timeout=None):
"""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
"""
if timeout is not None:
if not isinstance(timeout, (int, float)) or timeout < 0:
raise ValueError("timeout must be a positive integer or float. Got %s" % timeout)
return self._jsq.awaitTermination(int(timeout * 1000))
else:
return self._jsq.awaitTermination()
@property
@since(2.1)
def status(self):
"""
Returns the current status of the query.
"""
return json.loads(self._jsq.status().json())
[SPARK-18516][SQL] Split state and progress in streaming This PR separates the status of a `StreamingQuery` into two separate APIs: - `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available. - `recentProgress` - an array of statistics about the most recent microbatches that have executed. A recent progress contains the following information: ``` { "id" : "2be8670a-fce1-4859-a530-748f29553bb6", "name" : "query-29", "timestamp" : 1479705392724, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303, "durationMs" : { "triggerExecution" : 276, "queryPlanning" : 3, "getBatch" : 5, "getOffset" : 3, "addBatch" : 234, "walCommit" : 30 }, "currentWatermark" : 0, "stateOperators" : [ ], "sources" : [ { "description" : "KafkaSource[Subscribe[topic-14]]", "startOffset" : { "topic-14" : { "2" : 0, "4" : 1, "1" : 0, "3" : 0, "0" : 0 } }, "endOffset" : { "topic-14" : { "2" : 1, "4" : 2, "1" : 0, "3" : 0, "0" : 1 } }, "numRecords" : 3, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303 } ] } ``` Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Michael Armbrust <michael@databricks.com> Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
@property
@since(2.1)
def recentProgress(self):
[SPARK-18516][SQL] Split state and progress in streaming This PR separates the status of a `StreamingQuery` into two separate APIs: - `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available. - `recentProgress` - an array of statistics about the most recent microbatches that have executed. A recent progress contains the following information: ``` { "id" : "2be8670a-fce1-4859-a530-748f29553bb6", "name" : "query-29", "timestamp" : 1479705392724, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303, "durationMs" : { "triggerExecution" : 276, "queryPlanning" : 3, "getBatch" : 5, "getOffset" : 3, "addBatch" : 234, "walCommit" : 30 }, "currentWatermark" : 0, "stateOperators" : [ ], "sources" : [ { "description" : "KafkaSource[Subscribe[topic-14]]", "startOffset" : { "topic-14" : { "2" : 0, "4" : 1, "1" : 0, "3" : 0, "0" : 0 } }, "endOffset" : { "topic-14" : { "2" : 1, "4" : 2, "1" : 0, "3" : 0, "0" : 1 } }, "numRecords" : 3, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303 } ] } ``` Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Michael Armbrust <michael@databricks.com> Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
"""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`.
[SPARK-18516][SQL] Split state and progress in streaming This PR separates the status of a `StreamingQuery` into two separate APIs: - `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available. - `recentProgress` - an array of statistics about the most recent microbatches that have executed. A recent progress contains the following information: ``` { "id" : "2be8670a-fce1-4859-a530-748f29553bb6", "name" : "query-29", "timestamp" : 1479705392724, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303, "durationMs" : { "triggerExecution" : 276, "queryPlanning" : 3, "getBatch" : 5, "getOffset" : 3, "addBatch" : 234, "walCommit" : 30 }, "currentWatermark" : 0, "stateOperators" : [ ], "sources" : [ { "description" : "KafkaSource[Subscribe[topic-14]]", "startOffset" : { "topic-14" : { "2" : 0, "4" : 1, "1" : 0, "3" : 0, "0" : 0 } }, "endOffset" : { "topic-14" : { "2" : 1, "4" : 2, "1" : 0, "3" : 0, "0" : 1 } }, "numRecords" : 3, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303 } ] } ``` Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Michael Armbrust <michael@databricks.com> Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
"""
return [json.loads(p.json()) for p in self._jsq.recentProgress()]
[SPARK-18516][SQL] Split state and progress in streaming This PR separates the status of a `StreamingQuery` into two separate APIs: - `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available. - `recentProgress` - an array of statistics about the most recent microbatches that have executed. A recent progress contains the following information: ``` { "id" : "2be8670a-fce1-4859-a530-748f29553bb6", "name" : "query-29", "timestamp" : 1479705392724, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303, "durationMs" : { "triggerExecution" : 276, "queryPlanning" : 3, "getBatch" : 5, "getOffset" : 3, "addBatch" : 234, "walCommit" : 30 }, "currentWatermark" : 0, "stateOperators" : [ ], "sources" : [ { "description" : "KafkaSource[Subscribe[topic-14]]", "startOffset" : { "topic-14" : { "2" : 0, "4" : 1, "1" : 0, "3" : 0, "0" : 0 } }, "endOffset" : { "topic-14" : { "2" : 1, "4" : 2, "1" : 0, "3" : 0, "0" : 1 } }, "numRecords" : 3, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303 } ] } ``` Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Michael Armbrust <michael@databricks.com> Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
@property
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
[SPARK-18516][SQL] Split state and progress in streaming This PR separates the status of a `StreamingQuery` into two separate APIs: - `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available. - `recentProgress` - an array of statistics about the most recent microbatches that have executed. A recent progress contains the following information: ``` { "id" : "2be8670a-fce1-4859-a530-748f29553bb6", "name" : "query-29", "timestamp" : 1479705392724, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303, "durationMs" : { "triggerExecution" : 276, "queryPlanning" : 3, "getBatch" : 5, "getOffset" : 3, "addBatch" : 234, "walCommit" : 30 }, "currentWatermark" : 0, "stateOperators" : [ ], "sources" : [ { "description" : "KafkaSource[Subscribe[topic-14]]", "startOffset" : { "topic-14" : { "2" : 0, "4" : 1, "1" : 0, "3" : 0, "0" : 0 } }, "endOffset" : { "topic-14" : { "2" : 1, "4" : 2, "1" : 0, "3" : 0, "0" : 1 } }, "numRecords" : 3, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303 } ] } ``` Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Michael Armbrust <michael@databricks.com> Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
"""
lastProgress = self._jsq.lastProgress()
if lastProgress:
return json.loads(lastProgress.json())
else:
return None
[SPARK-18516][SQL] Split state and progress in streaming This PR separates the status of a `StreamingQuery` into two separate APIs: - `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available. - `recentProgress` - an array of statistics about the most recent microbatches that have executed. A recent progress contains the following information: ``` { "id" : "2be8670a-fce1-4859-a530-748f29553bb6", "name" : "query-29", "timestamp" : 1479705392724, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303, "durationMs" : { "triggerExecution" : 276, "queryPlanning" : 3, "getBatch" : 5, "getOffset" : 3, "addBatch" : 234, "walCommit" : 30 }, "currentWatermark" : 0, "stateOperators" : [ ], "sources" : [ { "description" : "KafkaSource[Subscribe[topic-14]]", "startOffset" : { "topic-14" : { "2" : 0, "4" : 1, "1" : 0, "3" : 0, "0" : 0 } }, "endOffset" : { "topic-14" : { "2" : 1, "4" : 2, "1" : 0, "3" : 0, "0" : 1 } }, "numRecords" : 3, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303 } ] } ``` Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Michael Armbrust <michael@databricks.com> Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
def processAllAvailable(self):
"""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).
"""
return self._jsq.processAllAvailable()
@since(2.0)
def stop(self):
"""Stop this streaming query.
"""
self._jsq.stop()
def explain(self, extended=False):
"""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.
print(self._jsq.explainInternal(extended))
def exception(self):
"""
.. versionadded:: 2.1.0
Returns
-------
:class:`StreamingQueryException`
the StreamingQueryException if the query was terminated by an exception, or None.
"""
if self._jsq.exception().isDefined():
je = self._jsq.exception().get()
msg = je.toString().split(': ', 1)[1] # Drop the Java StreamingQueryException type info
stackTrace = '\n\t at '.join(map(lambda x: x.toString(), je.getStackTrace()))
return StreamingQueryException(msg, stackTrace, je.getCause())
else:
return None
class StreamingQueryManager(object):
"""A class to manage all the :class:`StreamingQuery` StreamingQueries active.
.. versionadded:: 2.0.0
Notes
-----
This API is evolving.
"""
def __init__(self, jsqm):
self._jsqm = jsqm
@property
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()
"""
return [StreamingQuery(jsq) for jsq in self._jsqm.active()]
def get(self, id):
"""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()
"""
return StreamingQuery(self._jsqm.get(id))
@since(2.0)
def awaitAnyTermination(self, timeout=None):
"""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
"""
if timeout is not None:
if not isinstance(timeout, (int, float)) or timeout < 0:
raise ValueError("timeout must be a positive integer or float. Got %s" % timeout)
return self._jsqm.awaitAnyTermination(int(timeout * 1000))
else:
return self._jsqm.awaitAnyTermination()
def resetTerminated(self):
"""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()
"""
self._jsqm.resetTerminated()
class DataStreamReader(OptionUtils):
"""
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.
"""
def __init__(self, spark):
self._jreader = spark._ssql_ctx.readStream()
self._spark = spark
def _df(self, jdf):
from pyspark.sql.dataframe import DataFrame
return DataFrame(jdf, self._spark)
def format(self, source):
"""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")
"""
self._jreader = self._jreader.format(source)
return self
def schema(self, schema):
"""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")
"""
from pyspark.sql import SparkSession
spark = SparkSession.builder.getOrCreate()
if isinstance(schema, StructType):
jschema = spark._jsparkSession.parseDataType(schema.json())
self._jreader = self._jreader.schema(jschema)
elif isinstance(schema, str):
self._jreader = self._jreader.schema(schema)
else:
raise TypeError("schema should be StructType or string")
return self
def option(self, key, value):
"""Adds an input option for the underlying data source.
You can set the following option(s) for reading files:
* ``timeZone``: sets the string that indicates a time zone ID to be used to parse
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.
.. versionadded:: 2.0.0
Notes
-----
This API is evolving.
Examples
--------
>>> s = spark.readStream.option("x", 1)
"""
self._jreader = self._jreader.option(key, to_str(value))
return self
def options(self, **options):
"""Adds input options for the underlying data source.
You can set the following option(s) for reading files:
* ``timeZone``: sets the string that indicates a time zone ID to be used to parse
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.
.. versionadded:: 2.0.0
Notes
-----
This API is evolving.
Examples
--------
>>> s = spark.readStream.options(x="1", y=2)
"""
for k in options:
self._jreader = self._jreader.option(k, to_str(options[k]))
return self
def load(self, path=None, format=None, schema=None, **options):
"""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
--------
[MINOR][PYSPARK][DOCS] Fix examples in PySpark documentation ## What changes were proposed in this pull request? This PR proposes to fix wrongly indented examples in PySpark documentation ``` - >>> json_sdf = spark.readStream.format("json")\ - .schema(sdf_schema)\ - .load(tempfile.mkdtemp()) + >>> json_sdf = spark.readStream.format("json") \\ + ... .schema(sdf_schema) \\ + ... .load(tempfile.mkdtemp()) ``` ``` - people.filter(people.age > 30).join(department, people.deptId == department.id)\ + people.filter(people.age > 30).join(department, people.deptId == department.id) \\ ``` ``` - >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])), \ - LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))] + >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])), + ... LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))] ``` ``` - >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])), \ - LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))] + >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])), + ... LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))] ``` ``` - ... for x in iterator: - ... print(x) + ... for x in iterator: + ... print(x) ``` ## How was this patch tested? Manually tested. **Before** ![2016-09-26 8 36 02](https://cloud.githubusercontent.com/assets/6477701/18834471/05c7a478-8431-11e6-94bb-09aa37b12ddb.png) ![2016-09-26 9 22 16](https://cloud.githubusercontent.com/assets/6477701/18834472/06c8735c-8431-11e6-8775-78631eab0411.png) <img width="601" alt="2016-09-27 2 29 27" src="https://cloud.githubusercontent.com/assets/6477701/18861294/29c0d5b4-84bf-11e6-99c5-3c9d913c125d.png"> <img width="1056" alt="2016-09-27 2 29 58" src="https://cloud.githubusercontent.com/assets/6477701/18861298/31694cd8-84bf-11e6-9e61-9888cb8c2089.png"> <img width="1079" alt="2016-09-27 2 30 05" src="https://cloud.githubusercontent.com/assets/6477701/18861301/359722da-84bf-11e6-97f9-5f5365582d14.png"> **After** ![2016-09-26 9 29 47](https://cloud.githubusercontent.com/assets/6477701/18834467/0367f9da-8431-11e6-86d9-a490d3297339.png) ![2016-09-26 9 30 24](https://cloud.githubusercontent.com/assets/6477701/18834463/f870fae0-8430-11e6-9482-01fc47898492.png) <img width="515" alt="2016-09-27 2 28 19" src="https://cloud.githubusercontent.com/assets/6477701/18861305/3ff88b88-84bf-11e6-902c-9f725e8a8b10.png"> <img width="652" alt="2016-09-27 3 50 59" src="https://cloud.githubusercontent.com/assets/6477701/18863053/592fbc74-84ca-11e6-8dbf-99cf57947de8.png"> <img width="709" alt="2016-09-27 3 51 03" src="https://cloud.githubusercontent.com/assets/6477701/18863060/601607be-84ca-11e6-80aa-a401df41c321.png"> Author: hyukjinkwon <gurwls223@gmail.com> Closes #15242 from HyukjinKwon/minor-example-pyspark.
2016-09-28 06:19:04 -04:00
>>> json_sdf = spark.readStream.format("json") \\
... .schema(sdf_schema) \\
... .load(tempfile.mkdtemp())
>>> json_sdf.isStreaming
True
>>> json_sdf.schema == sdf_schema
True
"""
if format is not None:
self.format(format)
if schema is not None:
self.schema(schema)
self.options(**options)
if path is not None:
if type(path) != str or len(path.strip()) == 0:
raise ValueError("If the path is provided for stream, it needs to be a " +
"non-empty string. List of paths are not supported.")
return self._df(self._jreader.load(path))
else:
return self._df(self._jreader.load())
def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
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,
[SPARK-30128][DOCS][PYTHON][SQL] Document/promote 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC ### What changes were proposed in this pull request? This PR adds and exposes the options, 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC, into documentation. - `recursiveFileLookup` at file sources: https://github.com/apache/spark/pull/24830 ([SPARK-27627](https://issues.apache.org/jira/browse/SPARK-27627)) - `pathGlobFilter` at file sources: https://github.com/apache/spark/pull/24518 ([SPARK-27990](https://issues.apache.org/jira/browse/SPARK-27990)) - `mergeSchema` at ORC: https://github.com/apache/spark/pull/24043 ([SPARK-11412](https://issues.apache.org/jira/browse/SPARK-11412)) **Note that** `timeZone` option was not moved from `DataFrameReader.options` as I assume it will likely affect other datasources as well once DSv2 is complete. ### Why are the changes needed? To document available options in sources properly. ### Does this PR introduce any user-facing change? In PySpark, `pathGlobFilter` can be set via `DataFrameReader.(text|orc|parquet|json|csv)` and `DataStreamReader.(text|orc|parquet|json|csv)`. ### How was this patch tested? Manually built the doc and checked the output. Option setting in PySpark is rather a logical change. I manually tested one only: ```bash $ ls -al tmp ... -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 aa -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 ab -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 ac -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 cc ``` ```python >>> spark.read.text("tmp", pathGlobFilter="*c").show() ``` ``` +-----+ |value| +-----+ | ac| | cc| +-----+ ``` Closes #26958 from HyukjinKwon/doc-followup. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-22 19:57:42 -05:00
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``).
primitivesAsString : str or bool, optional
infers all primitive values as a string type. If None is set,
it uses the default value, ``false``.
prefersDecimal : str or bool, optional
infers all floating-point values as a decimal type. If the values
do not fit in decimal, then it infers them as doubles. If None is
set, it uses the default value, ``false``.
allowComments : str or bool, optional
ignores Java/C++ style comment in JSON records. If None is set,
it uses the default value, ``false``.
allowUnquotedFieldNames : str or bool, optional
allows unquoted JSON field names. If None is set,
it uses the default value, ``false``.
allowSingleQuotes : str or bool, optional
allows single quotes in addition to double quotes. If None is
set, it uses the default value, ``true``.
allowNumericLeadingZero : str or bool, optional
allows leading zeros in numbers (e.g. 00012). If None is
set, it uses the default value, ``false``.
allowBackslashEscapingAnyCharacter : str or bool, optional
allows accepting quoting of all character
using backslash quoting mechanism. If None is
set, it uses the default value, ``false``.
mode : str, optional
allows a mode for dealing with corrupt records during parsing. If None is
set, it uses the default value, ``PERMISSIVE``.
* ``PERMISSIVE``: when it meets a corrupted record, puts the malformed string \
into a field configured by ``columnNameOfCorruptRecord``, and sets malformed \
fields to ``null``. To keep corrupt records, an user can set a string type \
field named ``columnNameOfCorruptRecord`` in an user-defined schema. If a \
schema does not have the field, it drops corrupt records during parsing. \
When inferring a schema, it implicitly adds a ``columnNameOfCorruptRecord`` \
field in an output schema.
* ``DROPMALFORMED``: ignores the whole corrupted records.
* ``FAILFAST``: throws an exception when it meets corrupted records.
columnNameOfCorruptRecord : str, optional
allows renaming the new field having malformed string
created by ``PERMISSIVE`` mode. This overrides
``spark.sql.columnNameOfCorruptRecord``. If None is set,
it uses the value specified in
``spark.sql.columnNameOfCorruptRecord``.
dateFormat : str, optional
sets the string that indicates a date format. Custom date formats
follow the formats at
`datetime pattern <https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html>`_. # noqa
This applies to date type. If None is set, it uses the
default value, ``yyyy-MM-dd``.
timestampFormat : str, optional
sets the string that indicates a timestamp format.
Custom date formats follow the formats at
`datetime pattern <https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html>`_. # noqa
This applies to timestamp type. If None is set, it uses the
default value, ``yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]``.
multiLine : str or bool, optional
parse one record, which may span multiple lines, per file. If None is
set, it uses the default value, ``false``.
allowUnquotedControlChars : str or bool, optional
allows JSON Strings to contain unquoted control
characters (ASCII characters with value less than 32,
including tab and line feed characters) or not.
lineSep : str, optional
defines the line separator that should be used for parsing. If None is
set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``.
locale : str, optional
sets a locale as language tag in IETF BCP 47 format. If None is set,
it uses the default value, ``en-US``. For instance, ``locale`` is used while
parsing dates and timestamps.
dropFieldIfAllNull : str or bool, optional
whether to ignore column of all null values or empty
array/struct during schema inference. If None is set, it
uses the default value, ``false``.
encoding : str or bool, optional
allows to forcibly set one of standard basic or extended encoding for
the JSON files. For example UTF-16BE, UTF-32LE. If None is set,
the encoding of input JSON will be detected automatically
when the multiLine option is set to ``true``.
pathGlobFilter : str or bool, optional
an optional glob pattern to only include files with paths matching
the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`.
It does not change the behavior of
`partition discovery <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery>`_. # noqa
recursiveFileLookup : str or bool, optional
recursively scan a directory for files. Using this option
disables
`partition discovery <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery>`_. # noqa
allowNonNumericNumbers : str or bool, optional
allows JSON parser to recognize set of "Not-a-Number" (NaN)
tokens as legal floating number values. If None is set,
it uses the default value, ``true``.
* ``+INF``: for positive infinity, as well as alias of
``+Infinity`` and ``Infinity``.
* ``-INF``: for negative infinity, alias ``-Infinity``.
* ``NaN``: for other not-a-numbers, like result of division by zero.
[SPARK-27990][SPARK-29903][PYTHON] Add recursiveFileLookup option to Python DataFrameReader ### What changes were proposed in this pull request? As a follow-up to #24830, this PR adds the `recursiveFileLookup` option to the Python DataFrameReader API. ### Why are the changes needed? This PR maintains Python feature parity with Scala. ### Does this PR introduce any user-facing change? Yes. Before this PR, you'd only be able to use this option as follows: ```python spark.read.option("recursiveFileLookup", True).text("test-data").show() ``` With this PR, you can reference the option from within the format-specific method: ```python spark.read.text("test-data", recursiveFileLookup=True).show() ``` This option now also shows up in the Python API docs. ### How was this patch tested? I tested this manually by creating the following directories with dummy data: ``` test-data ├── 1.txt └── nested └── 2.txt test-parquet ├── nested │ ├── _SUCCESS │ ├── part-00000-...-.parquet ├── _SUCCESS ├── part-00000-...-.parquet ``` I then ran the following tests and confirmed the output looked good: ```python spark.read.parquet("test-parquet", recursiveFileLookup=True).show() spark.read.text("test-data", recursiveFileLookup=True).show() spark.read.csv("test-data", recursiveFileLookup=True).show() ``` `python/pyspark/sql/tests/test_readwriter.py` seems pretty sparse. I'm happy to add my tests there, though it seems we have been deferring testing like this to the Scala side of things. Closes #26718 from nchammas/SPARK-27990-recursiveFileLookup-python. Authored-by: Nicholas Chammas <nicholas.chammas@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-03 20:10:30 -05:00
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
"""
self._set_opts(
schema=schema, primitivesAsString=primitivesAsString, prefersDecimal=prefersDecimal,
allowComments=allowComments, allowUnquotedFieldNames=allowUnquotedFieldNames,
allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero,
allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter,
[SPARK-16216][SQL] Read/write timestamps and dates in ISO 8601 and dateFormat/timestampFormat option for CSV and JSON ## What changes were proposed in this pull request? ### Default - ISO 8601 Currently, CSV datasource is writing `Timestamp` and `Date` as numeric form and JSON datasource is writing both as below: - CSV ``` // TimestampType 1414459800000000 // DateType 16673 ``` - Json ``` // TimestampType 1970-01-01 11:46:40.0 // DateType 1970-01-01 ``` So, for CSV we can't read back what we write and for JSON it becomes ambiguous because the timezone is being missed. So, this PR make both **write** `Timestamp` and `Date` in ISO 8601 formatted string (please refer the [ISO 8601 specification](https://www.w3.org/TR/NOTE-datetime)). - For `Timestamp` it becomes as below: (`yyyy-MM-dd'T'HH:mm:ss.SSSZZ`) ``` 1970-01-01T02:00:01.000-01:00 ``` - For `Date` it becomes as below (`yyyy-MM-dd`) ``` 1970-01-01 ``` ### Custom date format option - `dateFormat` This PR also adds the support to write and read dates and timestamps in a formatted string as below: - **DateType** - With `dateFormat` option (e.g. `yyyy/MM/dd`) ``` +----------+ | date| +----------+ |2015/08/26| |2014/10/27| |2016/01/28| +----------+ ``` ### Custom date format option - `timestampFormat` - **TimestampType** - With `dateFormat` option (e.g. `dd/MM/yyyy HH:mm`) ``` +----------------+ | date| +----------------+ |2015/08/26 18:00| |2014/10/27 18:30| |2016/01/28 20:00| +----------------+ ``` ## How was this patch tested? Unit tests were added in `CSVSuite` and `JsonSuite`. For JSON, existing tests cover the default cases. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14279 from HyukjinKwon/SPARK-16216-json-csv.
2016-08-24 16:16:20 -04:00
mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat,
timestampFormat=timestampFormat, multiLine=multiLine,
allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep, locale=locale,
[SPARK-27990][SPARK-29903][PYTHON] Add recursiveFileLookup option to Python DataFrameReader ### What changes were proposed in this pull request? As a follow-up to #24830, this PR adds the `recursiveFileLookup` option to the Python DataFrameReader API. ### Why are the changes needed? This PR maintains Python feature parity with Scala. ### Does this PR introduce any user-facing change? Yes. Before this PR, you'd only be able to use this option as follows: ```python spark.read.option("recursiveFileLookup", True).text("test-data").show() ``` With this PR, you can reference the option from within the format-specific method: ```python spark.read.text("test-data", recursiveFileLookup=True).show() ``` This option now also shows up in the Python API docs. ### How was this patch tested? I tested this manually by creating the following directories with dummy data: ``` test-data ├── 1.txt └── nested └── 2.txt test-parquet ├── nested │ ├── _SUCCESS │ ├── part-00000-...-.parquet ├── _SUCCESS ├── part-00000-...-.parquet ``` I then ran the following tests and confirmed the output looked good: ```python spark.read.parquet("test-parquet", recursiveFileLookup=True).show() spark.read.text("test-data", recursiveFileLookup=True).show() spark.read.csv("test-data", recursiveFileLookup=True).show() ``` `python/pyspark/sql/tests/test_readwriter.py` seems pretty sparse. I'm happy to add my tests there, though it seems we have been deferring testing like this to the Scala side of things. Closes #26718 from nchammas/SPARK-27990-recursiveFileLookup-python. Authored-by: Nicholas Chammas <nicholas.chammas@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-03 20:10:30 -05:00
dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding,
pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup,
allowNonNumericNumbers=allowNonNumericNumbers)
if isinstance(path, str):
return self._df(self._jreader.json(path))
else:
raise TypeError("path can be only a single string")
[SPARK-30128][DOCS][PYTHON][SQL] Document/promote 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC ### What changes were proposed in this pull request? This PR adds and exposes the options, 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC, into documentation. - `recursiveFileLookup` at file sources: https://github.com/apache/spark/pull/24830 ([SPARK-27627](https://issues.apache.org/jira/browse/SPARK-27627)) - `pathGlobFilter` at file sources: https://github.com/apache/spark/pull/24518 ([SPARK-27990](https://issues.apache.org/jira/browse/SPARK-27990)) - `mergeSchema` at ORC: https://github.com/apache/spark/pull/24043 ([SPARK-11412](https://issues.apache.org/jira/browse/SPARK-11412)) **Note that** `timeZone` option was not moved from `DataFrameReader.options` as I assume it will likely affect other datasources as well once DSv2 is complete. ### Why are the changes needed? To document available options in sources properly. ### Does this PR introduce any user-facing change? In PySpark, `pathGlobFilter` can be set via `DataFrameReader.(text|orc|parquet|json|csv)` and `DataStreamReader.(text|orc|parquet|json|csv)`. ### How was this patch tested? Manually built the doc and checked the output. Option setting in PySpark is rather a logical change. I manually tested one only: ```bash $ ls -al tmp ... -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 aa -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 ab -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 ac -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 cc ``` ```python >>> spark.read.text("tmp", pathGlobFilter="*c").show() ``` ``` +-----+ |value| +-----+ | ac| | cc| +-----+ ``` Closes #26958 from HyukjinKwon/doc-followup. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-22 19:57:42 -05:00
def orc(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=None):
"""Loads a ORC file stream, returning the result as a :class:`DataFrame`.
.. versionadded:: 2.3.0
Parameters
----------
mergeSchema : str or bool, optional
sets whether we should merge schemas collected from all
ORC part-files. This will override ``spark.sql.orc.mergeSchema``.
The default value is specified in ``spark.sql.orc.mergeSchema``.
pathGlobFilter : str or bool, optional
an optional glob pattern to only include files with paths matching
the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`.
It does not change the behavior of `partition discovery`_.
recursiveFileLookup : str or bool, optional
recursively scan a directory for files. Using this option
disables
`partition discovery <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery>`_. # noqa
Examples
--------
>>> orc_sdf = spark.readStream.schema(sdf_schema).orc(tempfile.mkdtemp())
>>> orc_sdf.isStreaming
True
>>> orc_sdf.schema == sdf_schema
True
"""
[SPARK-30128][DOCS][PYTHON][SQL] Document/promote 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC ### What changes were proposed in this pull request? This PR adds and exposes the options, 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC, into documentation. - `recursiveFileLookup` at file sources: https://github.com/apache/spark/pull/24830 ([SPARK-27627](https://issues.apache.org/jira/browse/SPARK-27627)) - `pathGlobFilter` at file sources: https://github.com/apache/spark/pull/24518 ([SPARK-27990](https://issues.apache.org/jira/browse/SPARK-27990)) - `mergeSchema` at ORC: https://github.com/apache/spark/pull/24043 ([SPARK-11412](https://issues.apache.org/jira/browse/SPARK-11412)) **Note that** `timeZone` option was not moved from `DataFrameReader.options` as I assume it will likely affect other datasources as well once DSv2 is complete. ### Why are the changes needed? To document available options in sources properly. ### Does this PR introduce any user-facing change? In PySpark, `pathGlobFilter` can be set via `DataFrameReader.(text|orc|parquet|json|csv)` and `DataStreamReader.(text|orc|parquet|json|csv)`. ### How was this patch tested? Manually built the doc and checked the output. Option setting in PySpark is rather a logical change. I manually tested one only: ```bash $ ls -al tmp ... -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 aa -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 ab -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 ac -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 cc ``` ```python >>> spark.read.text("tmp", pathGlobFilter="*c").show() ``` ``` +-----+ |value| +-----+ | ac| | cc| +-----+ ``` Closes #26958 from HyukjinKwon/doc-followup. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-22 19:57:42 -05:00
self._set_opts(mergeSchema=mergeSchema, pathGlobFilter=pathGlobFilter,
recursiveFileLookup=recursiveFileLookup)
if isinstance(path, str):
return self._df(self._jreader.orc(path))
else:
raise TypeError("path can be only a single string")
[SPARK-34377][SQL] Add new parquet datasource options to control datetime rebasing in read ### What changes were proposed in this pull request? In the PR, I propose new options for the Parquet datasource: 1. `datetimeRebaseMode` 2. `int96RebaseMode` Both options influence on loading ancient dates and timestamps column values from parquet files. The `datetimeRebaseMode` option impacts on loading values of the `DATE`, `TIMESTAMP_MICROS` and `TIMESTAMP_MILLIS` types, `int96RebaseMode` impacts on loading of `INT96` timestamps. The options support the same values as the SQL configs `spark.sql.legacy.parquet.datetimeRebaseModeInRead` and `spark.sql.legacy.parquet.int96RebaseModeInRead` namely; - `"LEGACY"`, when an option is set to this value, Spark rebases dates/timestamps from the legacy hybrid calendar (Julian + Gregorian) to the Proleptic Gregorian calendar. - `"CORRECTED"`, dates/timestamps are read AS IS from parquet files. - `"EXCEPTION"`, when it is set as an option value, Spark will fail the reading if it sees ancient dates/timestamps that are ambiguous between the two calendars. ### Why are the changes needed? 1. New options will allow to load parquet files from at least two sources in different rebasing modes in the same query. For instance: ```scala val df1 = spark.read.option("datetimeRebaseMode", "legacy").parquet(folder1) val df2 = spark.read.option("datetimeRebaseMode", "corrected").parquet(folder2) df1.join(df2, ...) ``` Before the changes, it is impossible because the SQL config `spark.sql.legacy.parquet.datetimeRebaseModeInRead` influences on both reads. 2. Mixing of Dataset/DataFrame and RDD APIs should become possible. Since SQL configs are not propagated through RDDs, the following code fails on ancient timestamps: ```scala spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "legacy") spark.read.parquet(folder).distinct.rdd.collect() ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the modified test suites: ``` $ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV1Suite" $ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV2Suite" ``` Closes #31489 from MaxGekk/parquet-rebase-options. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-08 08:28:40 -05:00
def parquet(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=None,
datetimeRebaseMode=None, int96RebaseMode=None):
[SPARK-30091][SQL][PYTHON] Document mergeSchema option directly in the PySpark Parquet APIs ### What changes were proposed in this pull request? This change properly documents the `mergeSchema` option directly in the Python APIs for reading Parquet data. ### Why are the changes needed? The docstring for `DataFrameReader.parquet()` mentions `mergeSchema` but doesn't show it in the API. It seems like a simple oversight. Before this PR, you'd have to do this to use `mergeSchema`: ```python spark.read.option('mergeSchema', True).parquet('test-parquet').show() ``` After this PR, you can use the option as (I believe) it was intended to be used: ```python spark.read.parquet('test-parquet', mergeSchema=True).show() ``` ### Does this PR introduce any user-facing change? Yes, this PR changes the signatures of `DataFrameReader.parquet()` and `DataStreamReader.parquet()` to match their docstrings. ### How was this patch tested? Testing the `mergeSchema` option directly seems to be left to the Scala side of the codebase. I tested my change manually to confirm the API works. I also confirmed that setting `spark.sql.parquet.mergeSchema` at the session does not get overridden by leaving `mergeSchema` at its default when calling `parquet()`: ``` >>> spark.conf.set('spark.sql.parquet.mergeSchema', True) >>> spark.range(3).write.parquet('test-parquet/id') >>> spark.range(3).withColumnRenamed('id', 'name').write.parquet('test-parquet/name') >>> spark.read.option('recursiveFileLookup', True).parquet('test-parquet').show() +----+----+ | id|name| +----+----+ |null| 1| |null| 2| |null| 0| | 1|null| | 2|null| | 0|null| +----+----+ >>> spark.read.option('recursiveFileLookup', True).parquet('test-parquet', mergeSchema=False).show() +----+ | id| +----+ |null| |null| |null| | 1| | 2| | 0| +----+ ``` Closes #26730 from nchammas/parquet-merge-schema. Authored-by: Nicholas Chammas <nicholas.chammas@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-03 21:31:57 -05:00
"""
Loads a Parquet file stream, returning the result as a :class:`DataFrame`.
.. versionadded:: 2.0.0
Parameters
----------
mergeSchema : str or bool, optional
sets whether we should merge schemas collected from all
Parquet part-files. This will override
``spark.sql.parquet.mergeSchema``. The default value is specified in
``spark.sql.parquet.mergeSchema``.
pathGlobFilter : str or bool, optional
an optional glob pattern to only include files with paths matching
the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`.
It does not change the behavior of `partition discovery`_.
recursiveFileLookup : str or bool, optional
recursively scan a directory for files. Using this option
disables
`partition discovery <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery>`_. # noqa
[SPARK-34377][SQL] Add new parquet datasource options to control datetime rebasing in read ### What changes were proposed in this pull request? In the PR, I propose new options for the Parquet datasource: 1. `datetimeRebaseMode` 2. `int96RebaseMode` Both options influence on loading ancient dates and timestamps column values from parquet files. The `datetimeRebaseMode` option impacts on loading values of the `DATE`, `TIMESTAMP_MICROS` and `TIMESTAMP_MILLIS` types, `int96RebaseMode` impacts on loading of `INT96` timestamps. The options support the same values as the SQL configs `spark.sql.legacy.parquet.datetimeRebaseModeInRead` and `spark.sql.legacy.parquet.int96RebaseModeInRead` namely; - `"LEGACY"`, when an option is set to this value, Spark rebases dates/timestamps from the legacy hybrid calendar (Julian + Gregorian) to the Proleptic Gregorian calendar. - `"CORRECTED"`, dates/timestamps are read AS IS from parquet files. - `"EXCEPTION"`, when it is set as an option value, Spark will fail the reading if it sees ancient dates/timestamps that are ambiguous between the two calendars. ### Why are the changes needed? 1. New options will allow to load parquet files from at least two sources in different rebasing modes in the same query. For instance: ```scala val df1 = spark.read.option("datetimeRebaseMode", "legacy").parquet(folder1) val df2 = spark.read.option("datetimeRebaseMode", "corrected").parquet(folder2) df1.join(df2, ...) ``` Before the changes, it is impossible because the SQL config `spark.sql.legacy.parquet.datetimeRebaseModeInRead` influences on both reads. 2. Mixing of Dataset/DataFrame and RDD APIs should become possible. Since SQL configs are not propagated through RDDs, the following code fails on ancient timestamps: ```scala spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "legacy") spark.read.parquet(folder).distinct.rdd.collect() ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the modified test suites: ``` $ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV1Suite" $ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV2Suite" ``` Closes #31489 from MaxGekk/parquet-rebase-options. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-08 08:28:40 -05:00
datetimeRebaseMode : str, optional
the rebasing mode for the values of the ``DATE``, ``TIMESTAMP_MICROS``,
``TIMESTAMP_MILLIS`` logical types from the Julian to Proleptic Gregorian calendar.
* ``EXCEPTION``: Spark fails in reads of ancient dates/timestamps
that are ambiguous between the two calendars.
* ``CORRECTED``: loading of dates/timestamps without rebasing.
* ``LEGACY``: perform rebasing of ancient dates/timestamps from the Julian
to Proleptic Gregorian calendar.
If None is set, the value of the SQL config
[SPARK-34451][SQL] Add alternatives for datetime rebasing SQL configs and deprecate legacy configs ### What changes were proposed in this pull request? Move the datetime rebase SQL configs from the `legacy` namespace by: 1. Renaming of the existing rebase configs like `spark.sql.legacy.parquet.datetimeRebaseModeInRead` -> `spark.sql.parquet.datetimeRebaseModeInRead`. 2. Add the legacy configs as alternatives 3. Deprecate the legacy rebase configs. ### Why are the changes needed? The rebasing SQL configs like `spark.sql.legacy.parquet.datetimeRebaseModeInRead` can be used not only for migration from previous Spark versions but also to read/write datatime columns saved by other systems/frameworks/libs. So, the configs shouldn't be considered as legacy configs. ### Does this PR introduce _any_ user-facing change? Should not. Users will see a warning if they still use one of the legacy configs. ### How was this patch tested? 1. Manually checking new configs: ```scala scala> spark.conf.get("spark.sql.parquet.datetimeRebaseModeInRead") res0: String = EXCEPTION scala> spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "LEGACY") 21/02/17 14:57:10 WARN SQLConf: The SQL config 'spark.sql.legacy.parquet.datetimeRebaseModeInRead' has been deprecated in Spark v3.2 and may be removed in the future. Use 'spark.sql.parquet.datetimeRebaseModeInRead' instead. scala> spark.conf.get("spark.sql.parquet.datetimeRebaseModeInRead") res2: String = LEGACY ``` 2. By running a datetime rebasing test suite: ``` $ build/sbt "test:testOnly *ParquetRebaseDatetimeV1Suite" ``` Closes #31576 from MaxGekk/rebase-confs-alternatives. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-17 09:04:47 -05:00
``spark.sql.parquet.datetimeRebaseModeInRead`` is used by default.
[SPARK-34377][SQL] Add new parquet datasource options to control datetime rebasing in read ### What changes were proposed in this pull request? In the PR, I propose new options for the Parquet datasource: 1. `datetimeRebaseMode` 2. `int96RebaseMode` Both options influence on loading ancient dates and timestamps column values from parquet files. The `datetimeRebaseMode` option impacts on loading values of the `DATE`, `TIMESTAMP_MICROS` and `TIMESTAMP_MILLIS` types, `int96RebaseMode` impacts on loading of `INT96` timestamps. The options support the same values as the SQL configs `spark.sql.legacy.parquet.datetimeRebaseModeInRead` and `spark.sql.legacy.parquet.int96RebaseModeInRead` namely; - `"LEGACY"`, when an option is set to this value, Spark rebases dates/timestamps from the legacy hybrid calendar (Julian + Gregorian) to the Proleptic Gregorian calendar. - `"CORRECTED"`, dates/timestamps are read AS IS from parquet files. - `"EXCEPTION"`, when it is set as an option value, Spark will fail the reading if it sees ancient dates/timestamps that are ambiguous between the two calendars. ### Why are the changes needed? 1. New options will allow to load parquet files from at least two sources in different rebasing modes in the same query. For instance: ```scala val df1 = spark.read.option("datetimeRebaseMode", "legacy").parquet(folder1) val df2 = spark.read.option("datetimeRebaseMode", "corrected").parquet(folder2) df1.join(df2, ...) ``` Before the changes, it is impossible because the SQL config `spark.sql.legacy.parquet.datetimeRebaseModeInRead` influences on both reads. 2. Mixing of Dataset/DataFrame and RDD APIs should become possible. Since SQL configs are not propagated through RDDs, the following code fails on ancient timestamps: ```scala spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "legacy") spark.read.parquet(folder).distinct.rdd.collect() ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the modified test suites: ``` $ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV1Suite" $ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV2Suite" ``` Closes #31489 from MaxGekk/parquet-rebase-options. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-08 08:28:40 -05:00
int96RebaseMode : str, optional
the rebasing mode for ``INT96`` timestamps from the Julian to
Proleptic Gregorian calendar.
* ``EXCEPTION``: Spark fails in reads of ancient ``INT96`` timestamps
that are ambiguous between the two calendars.
* ``CORRECTED``: loading of ``INT96`` timestamps without rebasing.
* ``LEGACY``: perform rebasing of ancient ``INT96`` timestamps from the Julian
to Proleptic Gregorian calendar.
If None is set, the value of the SQL config
[SPARK-34451][SQL] Add alternatives for datetime rebasing SQL configs and deprecate legacy configs ### What changes were proposed in this pull request? Move the datetime rebase SQL configs from the `legacy` namespace by: 1. Renaming of the existing rebase configs like `spark.sql.legacy.parquet.datetimeRebaseModeInRead` -> `spark.sql.parquet.datetimeRebaseModeInRead`. 2. Add the legacy configs as alternatives 3. Deprecate the legacy rebase configs. ### Why are the changes needed? The rebasing SQL configs like `spark.sql.legacy.parquet.datetimeRebaseModeInRead` can be used not only for migration from previous Spark versions but also to read/write datatime columns saved by other systems/frameworks/libs. So, the configs shouldn't be considered as legacy configs. ### Does this PR introduce _any_ user-facing change? Should not. Users will see a warning if they still use one of the legacy configs. ### How was this patch tested? 1. Manually checking new configs: ```scala scala> spark.conf.get("spark.sql.parquet.datetimeRebaseModeInRead") res0: String = EXCEPTION scala> spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "LEGACY") 21/02/17 14:57:10 WARN SQLConf: The SQL config 'spark.sql.legacy.parquet.datetimeRebaseModeInRead' has been deprecated in Spark v3.2 and may be removed in the future. Use 'spark.sql.parquet.datetimeRebaseModeInRead' instead. scala> spark.conf.get("spark.sql.parquet.datetimeRebaseModeInRead") res2: String = LEGACY ``` 2. By running a datetime rebasing test suite: ``` $ build/sbt "test:testOnly *ParquetRebaseDatetimeV1Suite" ``` Closes #31576 from MaxGekk/rebase-confs-alternatives. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-17 09:04:47 -05:00
``spark.sql.parquet.int96RebaseModeInRead`` is used by default.
Examples
--------
>>> parquet_sdf = spark.readStream.schema(sdf_schema).parquet(tempfile.mkdtemp())
>>> parquet_sdf.isStreaming
True
>>> parquet_sdf.schema == sdf_schema
True
"""
[SPARK-30128][DOCS][PYTHON][SQL] Document/promote 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC ### What changes were proposed in this pull request? This PR adds and exposes the options, 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC, into documentation. - `recursiveFileLookup` at file sources: https://github.com/apache/spark/pull/24830 ([SPARK-27627](https://issues.apache.org/jira/browse/SPARK-27627)) - `pathGlobFilter` at file sources: https://github.com/apache/spark/pull/24518 ([SPARK-27990](https://issues.apache.org/jira/browse/SPARK-27990)) - `mergeSchema` at ORC: https://github.com/apache/spark/pull/24043 ([SPARK-11412](https://issues.apache.org/jira/browse/SPARK-11412)) **Note that** `timeZone` option was not moved from `DataFrameReader.options` as I assume it will likely affect other datasources as well once DSv2 is complete. ### Why are the changes needed? To document available options in sources properly. ### Does this PR introduce any user-facing change? In PySpark, `pathGlobFilter` can be set via `DataFrameReader.(text|orc|parquet|json|csv)` and `DataStreamReader.(text|orc|parquet|json|csv)`. ### How was this patch tested? Manually built the doc and checked the output. Option setting in PySpark is rather a logical change. I manually tested one only: ```bash $ ls -al tmp ... -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 aa -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 ab -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 ac -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 cc ``` ```python >>> spark.read.text("tmp", pathGlobFilter="*c").show() ``` ``` +-----+ |value| +-----+ | ac| | cc| +-----+ ``` Closes #26958 from HyukjinKwon/doc-followup. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-22 19:57:42 -05:00
self._set_opts(mergeSchema=mergeSchema, pathGlobFilter=pathGlobFilter,
[SPARK-34377][SQL] Add new parquet datasource options to control datetime rebasing in read ### What changes were proposed in this pull request? In the PR, I propose new options for the Parquet datasource: 1. `datetimeRebaseMode` 2. `int96RebaseMode` Both options influence on loading ancient dates and timestamps column values from parquet files. The `datetimeRebaseMode` option impacts on loading values of the `DATE`, `TIMESTAMP_MICROS` and `TIMESTAMP_MILLIS` types, `int96RebaseMode` impacts on loading of `INT96` timestamps. The options support the same values as the SQL configs `spark.sql.legacy.parquet.datetimeRebaseModeInRead` and `spark.sql.legacy.parquet.int96RebaseModeInRead` namely; - `"LEGACY"`, when an option is set to this value, Spark rebases dates/timestamps from the legacy hybrid calendar (Julian + Gregorian) to the Proleptic Gregorian calendar. - `"CORRECTED"`, dates/timestamps are read AS IS from parquet files. - `"EXCEPTION"`, when it is set as an option value, Spark will fail the reading if it sees ancient dates/timestamps that are ambiguous between the two calendars. ### Why are the changes needed? 1. New options will allow to load parquet files from at least two sources in different rebasing modes in the same query. For instance: ```scala val df1 = spark.read.option("datetimeRebaseMode", "legacy").parquet(folder1) val df2 = spark.read.option("datetimeRebaseMode", "corrected").parquet(folder2) df1.join(df2, ...) ``` Before the changes, it is impossible because the SQL config `spark.sql.legacy.parquet.datetimeRebaseModeInRead` influences on both reads. 2. Mixing of Dataset/DataFrame and RDD APIs should become possible. Since SQL configs are not propagated through RDDs, the following code fails on ancient timestamps: ```scala spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "legacy") spark.read.parquet(folder).distinct.rdd.collect() ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the modified test suites: ``` $ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV1Suite" $ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV2Suite" ``` Closes #31489 from MaxGekk/parquet-rebase-options. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-08 08:28:40 -05:00
recursiveFileLookup=recursiveFileLookup,
datetimeRebaseMode=datetimeRebaseMode, int96RebaseMode=int96RebaseMode)
if isinstance(path, str):
return self._df(self._jreader.parquet(path))
else:
raise TypeError("path can be only a single string")
[SPARK-30128][DOCS][PYTHON][SQL] Document/promote 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC ### What changes were proposed in this pull request? This PR adds and exposes the options, 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC, into documentation. - `recursiveFileLookup` at file sources: https://github.com/apache/spark/pull/24830 ([SPARK-27627](https://issues.apache.org/jira/browse/SPARK-27627)) - `pathGlobFilter` at file sources: https://github.com/apache/spark/pull/24518 ([SPARK-27990](https://issues.apache.org/jira/browse/SPARK-27990)) - `mergeSchema` at ORC: https://github.com/apache/spark/pull/24043 ([SPARK-11412](https://issues.apache.org/jira/browse/SPARK-11412)) **Note that** `timeZone` option was not moved from `DataFrameReader.options` as I assume it will likely affect other datasources as well once DSv2 is complete. ### Why are the changes needed? To document available options in sources properly. ### Does this PR introduce any user-facing change? In PySpark, `pathGlobFilter` can be set via `DataFrameReader.(text|orc|parquet|json|csv)` and `DataStreamReader.(text|orc|parquet|json|csv)`. ### How was this patch tested? Manually built the doc and checked the output. Option setting in PySpark is rather a logical change. I manually tested one only: ```bash $ ls -al tmp ... -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 aa -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 ab -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 ac -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 cc ``` ```python >>> spark.read.text("tmp", pathGlobFilter="*c").show() ``` ``` +-----+ |value| +-----+ | ac| | cc| +-----+ ``` Closes #26958 from HyukjinKwon/doc-followup. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-22 19:57:42 -05:00
def text(self, path, wholetext=False, lineSep=None, pathGlobFilter=None,
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).
wholetext : str or bool, optional
if true, read each file from input path(s) as a single row.
lineSep : str, optional
defines the line separator that should be used for parsing. If None is
set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``.
pathGlobFilter : str or bool, optional
an optional glob pattern to only include files with paths matching
the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`.
It does not change the behavior of `partition discovery`_.
recursiveFileLookup : str or bool, optional
recursively scan a directory for files. Using this option
disables
`partition discovery <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery>`_. # 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
"""
[SPARK-27990][SPARK-29903][PYTHON] Add recursiveFileLookup option to Python DataFrameReader ### What changes were proposed in this pull request? As a follow-up to #24830, this PR adds the `recursiveFileLookup` option to the Python DataFrameReader API. ### Why are the changes needed? This PR maintains Python feature parity with Scala. ### Does this PR introduce any user-facing change? Yes. Before this PR, you'd only be able to use this option as follows: ```python spark.read.option("recursiveFileLookup", True).text("test-data").show() ``` With this PR, you can reference the option from within the format-specific method: ```python spark.read.text("test-data", recursiveFileLookup=True).show() ``` This option now also shows up in the Python API docs. ### How was this patch tested? I tested this manually by creating the following directories with dummy data: ``` test-data ├── 1.txt └── nested └── 2.txt test-parquet ├── nested │ ├── _SUCCESS │ ├── part-00000-...-.parquet ├── _SUCCESS ├── part-00000-...-.parquet ``` I then ran the following tests and confirmed the output looked good: ```python spark.read.parquet("test-parquet", recursiveFileLookup=True).show() spark.read.text("test-data", recursiveFileLookup=True).show() spark.read.csv("test-data", recursiveFileLookup=True).show() ``` `python/pyspark/sql/tests/test_readwriter.py` seems pretty sparse. I'm happy to add my tests there, though it seems we have been deferring testing like this to the Scala side of things. Closes #26718 from nchammas/SPARK-27990-recursiveFileLookup-python. Authored-by: Nicholas Chammas <nicholas.chammas@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-03 20:10:30 -05:00
self._set_opts(
[SPARK-30128][DOCS][PYTHON][SQL] Document/promote 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC ### What changes were proposed in this pull request? This PR adds and exposes the options, 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC, into documentation. - `recursiveFileLookup` at file sources: https://github.com/apache/spark/pull/24830 ([SPARK-27627](https://issues.apache.org/jira/browse/SPARK-27627)) - `pathGlobFilter` at file sources: https://github.com/apache/spark/pull/24518 ([SPARK-27990](https://issues.apache.org/jira/browse/SPARK-27990)) - `mergeSchema` at ORC: https://github.com/apache/spark/pull/24043 ([SPARK-11412](https://issues.apache.org/jira/browse/SPARK-11412)) **Note that** `timeZone` option was not moved from `DataFrameReader.options` as I assume it will likely affect other datasources as well once DSv2 is complete. ### Why are the changes needed? To document available options in sources properly. ### Does this PR introduce any user-facing change? In PySpark, `pathGlobFilter` can be set via `DataFrameReader.(text|orc|parquet|json|csv)` and `DataStreamReader.(text|orc|parquet|json|csv)`. ### How was this patch tested? Manually built the doc and checked the output. Option setting in PySpark is rather a logical change. I manually tested one only: ```bash $ ls -al tmp ... -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 aa -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 ab -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 ac -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 cc ``` ```python >>> spark.read.text("tmp", pathGlobFilter="*c").show() ``` ``` +-----+ |value| +-----+ | ac| | cc| +-----+ ``` Closes #26958 from HyukjinKwon/doc-followup. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-22 19:57:42 -05:00
wholetext=wholetext, lineSep=lineSep, pathGlobFilter=pathGlobFilter,
recursiveFileLookup=recursiveFileLookup)
if isinstance(path, str):
return self._df(self._jreader.text(path))
else:
raise TypeError("path can be only a single string")
def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None,
comment=None, header=None, inferSchema=None, ignoreLeadingWhiteSpace=None,
ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None,
[SPARK-16216][SQL] Read/write timestamps and dates in ISO 8601 and dateFormat/timestampFormat option for CSV and JSON ## What changes were proposed in this pull request? ### Default - ISO 8601 Currently, CSV datasource is writing `Timestamp` and `Date` as numeric form and JSON datasource is writing both as below: - CSV ``` // TimestampType 1414459800000000 // DateType 16673 ``` - Json ``` // TimestampType 1970-01-01 11:46:40.0 // DateType 1970-01-01 ``` So, for CSV we can't read back what we write and for JSON it becomes ambiguous because the timezone is being missed. So, this PR make both **write** `Timestamp` and `Date` in ISO 8601 formatted string (please refer the [ISO 8601 specification](https://www.w3.org/TR/NOTE-datetime)). - For `Timestamp` it becomes as below: (`yyyy-MM-dd'T'HH:mm:ss.SSSZZ`) ``` 1970-01-01T02:00:01.000-01:00 ``` - For `Date` it becomes as below (`yyyy-MM-dd`) ``` 1970-01-01 ``` ### Custom date format option - `dateFormat` This PR also adds the support to write and read dates and timestamps in a formatted string as below: - **DateType** - With `dateFormat` option (e.g. `yyyy/MM/dd`) ``` +----------+ | date| +----------+ |2015/08/26| |2014/10/27| |2016/01/28| +----------+ ``` ### Custom date format option - `timestampFormat` - **TimestampType** - With `dateFormat` option (e.g. `dd/MM/yyyy HH:mm`) ``` +----------------+ | date| +----------------+ |2015/08/26 18:00| |2014/10/27 18:30| |2016/01/28 20:00| +----------------+ ``` ## How was this patch tested? Unit tests were added in `CSVSuite` and `JsonSuite`. For JSON, existing tests cover the default cases. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14279 from HyukjinKwon/SPARK-16216-json-csv.
2016-08-24 16:16:20 -04:00
negativeInf=None, dateFormat=None, timestampFormat=None, maxColumns=None,
maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None,
columnNameOfCorruptRecord=None, multiLine=None, charToEscapeQuoteEscaping=None,
[SPARK-27990][SPARK-29903][PYTHON] Add recursiveFileLookup option to Python DataFrameReader ### What changes were proposed in this pull request? As a follow-up to #24830, this PR adds the `recursiveFileLookup` option to the Python DataFrameReader API. ### Why are the changes needed? This PR maintains Python feature parity with Scala. ### Does this PR introduce any user-facing change? Yes. Before this PR, you'd only be able to use this option as follows: ```python spark.read.option("recursiveFileLookup", True).text("test-data").show() ``` With this PR, you can reference the option from within the format-specific method: ```python spark.read.text("test-data", recursiveFileLookup=True).show() ``` This option now also shows up in the Python API docs. ### How was this patch tested? I tested this manually by creating the following directories with dummy data: ``` test-data ├── 1.txt └── nested └── 2.txt test-parquet ├── nested │ ├── _SUCCESS │ ├── part-00000-...-.parquet ├── _SUCCESS ├── part-00000-...-.parquet ``` I then ran the following tests and confirmed the output looked good: ```python spark.read.parquet("test-parquet", recursiveFileLookup=True).show() spark.read.text("test-data", recursiveFileLookup=True).show() spark.read.csv("test-data", recursiveFileLookup=True).show() ``` `python/pyspark/sql/tests/test_readwriter.py` seems pretty sparse. I'm happy to add my tests there, though it seems we have been deferring testing like this to the Scala side of things. Closes #26718 from nchammas/SPARK-27990-recursiveFileLookup-python. Authored-by: Nicholas Chammas <nicholas.chammas@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-03 20:10:30 -05:00
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``).
sep : str, optional
sets a separator (one or more characters) for each field and value. If None is
set, it uses the default value, ``,``.
encoding : str, optional
decodes the CSV files by the given encoding type. If None is set,
it uses the default value, ``UTF-8``.
quote : str, optional sets a single character used for escaping quoted values where the
separator can be part of the value. If None is set, it uses the default
value, ``"``. If you would like to turn off quotations, you need to set an
empty string.
escape : str, optional
sets a single character used for escaping quotes inside an already
quoted value. If None is set, it uses the default value, ``\``.
comment : str, optional
sets a single character used for skipping lines beginning with this
character. By default (None), it is disabled.
header : str or bool, optional
uses the first line as names of columns. If None is set, it uses the
default value, ``false``.
inferSchema : str or bool, optional
infers the input schema automatically from data. It requires one extra
pass over the data. If None is set, it uses the default value, ``false``.
enforceSchema : str or bool, optional
If it is set to ``true``, the specified or inferred schema will be
forcibly applied to datasource files, and headers in CSV files will be
ignored. If the option is set to ``false``, the schema will be
validated against all headers in CSV files or the first header in RDD
if the ``header`` option is set to ``true``. Field names in the schema
and column names in CSV headers are checked by their positions
taking into account ``spark.sql.caseSensitive``. If None is set,
``true`` is used by default. Though the default value is ``true``,
it is recommended to disable the ``enforceSchema`` option
to avoid incorrect results.
ignoreLeadingWhiteSpace : str or bool, optional
a flag indicating whether or not leading whitespaces from
values being read should be skipped. If None is set, it
uses the default value, ``false``.
ignoreTrailingWhiteSpace : str or bool, optional
a flag indicating whether or not trailing whitespaces from
values being read should be skipped. If None is set, it
uses the default value, ``false``.
nullValue : str, optional
sets the string representation of a null value. If None is set, it uses
the default value, empty string. Since 2.0.1, this ``nullValue`` param
applies to all supported types including the string type.
nanValue : str, optional
sets the string representation of a non-number value. If None is set, it
uses the default value, ``NaN``.
positiveInf : str, optional
sets the string representation of a positive infinity value. If None
is set, it uses the default value, ``Inf``.
negativeInf : str, optional
sets the string representation of a negative infinity value. If None
is set, it uses the default value, ``Inf``.
dateFormat : str, optional
sets the string that indicates a date format. Custom date formats
follow the formats at
`datetime pattern <https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html>`_. # noqa
This applies to date type. If None is set, it uses the
default value, ``yyyy-MM-dd``.
timestampFormat : str, optional
sets the string that indicates a timestamp format.
Custom date formats follow the formats at
`datetime pattern <https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html>`_. # noqa
This applies to timestamp type. If None is set, it uses the
default value, ``yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]``.
maxColumns : str or int, optional
defines a hard limit of how many columns a record can have. If None is
set, it uses the default value, ``20480``.
maxCharsPerColumn : str or int, optional
defines the maximum number of characters allowed for any given
value being read. If None is set, it uses the default value,
``-1`` meaning unlimited length.
maxMalformedLogPerPartition : str or int, optional
this parameter is no longer used since Spark 2.2.0.
If specified, it is ignored.
mode : str, optional
allows a mode for dealing with corrupt records during parsing. If None is
set, it uses the default value, ``PERMISSIVE``.
* ``PERMISSIVE``: when it meets a corrupted record, puts the malformed string \
into a field configured by ``columnNameOfCorruptRecord``, and sets malformed \
fields to ``null``. To keep corrupt records, an user can set a string type \
field named ``columnNameOfCorruptRecord`` in an user-defined schema. If a \
schema does not have the field, it drops corrupt records during parsing. \
A record with less/more tokens than schema is not a corrupted record to CSV. \
When it meets a record having fewer tokens than the length of the schema, \
sets ``null`` to extra fields. When the record has more tokens than the \
length of the schema, it drops extra tokens.
* ``DROPMALFORMED``: ignores the whole corrupted records.
* ``FAILFAST``: throws an exception when it meets corrupted records.
columnNameOfCorruptRecord : str, optional
allows renaming the new field having malformed string
created by ``PERMISSIVE`` mode. This overrides
``spark.sql.columnNameOfCorruptRecord``. If None is set,
it uses the value specified in
``spark.sql.columnNameOfCorruptRecord``.
multiLine : str or bool, optional
parse one record, which may span multiple lines. If None is
set, it uses the default value, ``false``.
charToEscapeQuoteEscaping : str, optional
sets a single character used for escaping the escape for
the quote character. If None is set, the default value is
escape character when escape and quote characters are
different, ``\0`` otherwise.
emptyValue : str, optional
sets the string representation of an empty value. If None is set, it uses
the default value, empty string.
locale : str, optional
sets a locale as language tag in IETF BCP 47 format. If None is set,
it uses the default value, ``en-US``. For instance, ``locale`` is used while
parsing dates and timestamps.
lineSep : str, optional
defines the line separator that should be used for parsing. If None is
set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``.
Maximum length is 1 character.
pathGlobFilter : str or bool, optional
an optional glob pattern to only include files with paths matching
the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`.
It does not change the behavior of
`partition discovery <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery>`_. # noqa
recursiveFileLookup : str or bool, optional
recursively scan a directory for files. Using this option disables
`partition discovery <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery>`_. # noqa
unescapedQuoteHandling : str, optional
defines how the CsvParser will handle values with unescaped quotes. If None is
set, it uses the default value, ``STOP_AT_DELIMITER``.
* ``STOP_AT_CLOSING_QUOTE``: If unescaped quotes are found in the input, accumulate
the quote character and proceed parsing the value as a quoted value, until a closing
quote is found.
* ``BACK_TO_DELIMITER``: If unescaped quotes are found in the input, consider the value
as an unquoted value. This will make the parser accumulate all characters of the current
parsed value until the delimiter is found. If no delimiter is found in the value, the
parser will continue accumulating characters from the input until a delimiter or line
ending is found.
* ``STOP_AT_DELIMITER``: If unescaped quotes are found in the input, consider the value
as an unquoted value. This will make the parser accumulate all characters until the
delimiter or a line ending is found in the input.
* ``SKIP_VALUE``: If unescaped quotes are found in the input, the content parsed
for the given value will be skipped and the value set in nullValue will be produced
instead.
* ``RAISE_ERROR``: If unescaped quotes are found in the input, a TextParsingException
will be thrown.
.. versionadded:: 2.0.0
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
"""
self._set_opts(
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,
[SPARK-16216][SQL] Read/write timestamps and dates in ISO 8601 and dateFormat/timestampFormat option for CSV and JSON ## What changes were proposed in this pull request? ### Default - ISO 8601 Currently, CSV datasource is writing `Timestamp` and `Date` as numeric form and JSON datasource is writing both as below: - CSV ``` // TimestampType 1414459800000000 // DateType 16673 ``` - Json ``` // TimestampType 1970-01-01 11:46:40.0 // DateType 1970-01-01 ``` So, for CSV we can't read back what we write and for JSON it becomes ambiguous because the timezone is being missed. So, this PR make both **write** `Timestamp` and `Date` in ISO 8601 formatted string (please refer the [ISO 8601 specification](https://www.w3.org/TR/NOTE-datetime)). - For `Timestamp` it becomes as below: (`yyyy-MM-dd'T'HH:mm:ss.SSSZZ`) ``` 1970-01-01T02:00:01.000-01:00 ``` - For `Date` it becomes as below (`yyyy-MM-dd`) ``` 1970-01-01 ``` ### Custom date format option - `dateFormat` This PR also adds the support to write and read dates and timestamps in a formatted string as below: - **DateType** - With `dateFormat` option (e.g. `yyyy/MM/dd`) ``` +----------+ | date| +----------+ |2015/08/26| |2014/10/27| |2016/01/28| +----------+ ``` ### Custom date format option - `timestampFormat` - **TimestampType** - With `dateFormat` option (e.g. `dd/MM/yyyy HH:mm`) ``` +----------------+ | date| +----------------+ |2015/08/26 18:00| |2014/10/27 18:30| |2016/01/28 20:00| +----------------+ ``` ## How was this patch tested? Unit tests were added in `CSVSuite` and `JsonSuite`. For JSON, existing tests cover the default cases. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14279 from HyukjinKwon/SPARK-16216-json-csv.
2016-08-24 16:16:20 -04:00
dateFormat=dateFormat, timestampFormat=timestampFormat, maxColumns=maxColumns,
maxCharsPerColumn=maxCharsPerColumn,
maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode,
columnNameOfCorruptRecord=columnNameOfCorruptRecord, multiLine=multiLine,
charToEscapeQuoteEscaping=charToEscapeQuoteEscaping, enforceSchema=enforceSchema,
[SPARK-27990][SPARK-29903][PYTHON] Add recursiveFileLookup option to Python DataFrameReader ### What changes were proposed in this pull request? As a follow-up to #24830, this PR adds the `recursiveFileLookup` option to the Python DataFrameReader API. ### Why are the changes needed? This PR maintains Python feature parity with Scala. ### Does this PR introduce any user-facing change? Yes. Before this PR, you'd only be able to use this option as follows: ```python spark.read.option("recursiveFileLookup", True).text("test-data").show() ``` With this PR, you can reference the option from within the format-specific method: ```python spark.read.text("test-data", recursiveFileLookup=True).show() ``` This option now also shows up in the Python API docs. ### How was this patch tested? I tested this manually by creating the following directories with dummy data: ``` test-data ├── 1.txt └── nested └── 2.txt test-parquet ├── nested │ ├── _SUCCESS │ ├── part-00000-...-.parquet ├── _SUCCESS ├── part-00000-...-.parquet ``` I then ran the following tests and confirmed the output looked good: ```python spark.read.parquet("test-parquet", recursiveFileLookup=True).show() spark.read.text("test-data", recursiveFileLookup=True).show() spark.read.csv("test-data", recursiveFileLookup=True).show() ``` `python/pyspark/sql/tests/test_readwriter.py` seems pretty sparse. I'm happy to add my tests there, though it seems we have been deferring testing like this to the Scala side of things. Closes #26718 from nchammas/SPARK-27990-recursiveFileLookup-python. Authored-by: Nicholas Chammas <nicholas.chammas@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-03 20:10:30 -05:00
emptyValue=emptyValue, locale=locale, lineSep=lineSep,
pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup,
unescapedQuoteHandling=unescapedQuoteHandling)
if isinstance(path, str):
return self._df(self._jreader.csv(path))
else:
raise TypeError("path can be only a single string")
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable ### What changes were proposed in this pull request? This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming. ### Why are the changes needed? Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark. ### Does this PR introduce _any_ user-facing change? Yes, PySpark users will be able to read and write with table in Structured Streaming query. ### How was this patch tested? Manually tested. > v1 table >> create table A and ingest to the table A ``` spark.sql(""" create table table_pyspark_parquet ( value long, `timestamp` timestamp ) USING parquet """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5') query.lastProgress query.stop() ``` >> read table A and ingest to the table B which doesn't exist ``` df2 = spark.readStream.table('table_pyspark_parquet') query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2') query2.lastProgress query2.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE table_pyspark_parquet").show() spark.sql("SELECT * FROM table_pyspark_parquet").show() spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show() spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show() ``` > v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well) >> create table A and ingest to the table A ``` spark.sql(""" create table iceberg_catalog.default.table_pyspark_v2table ( value long, `timestamp` timestamp ) USING iceberg """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1') query.lastProgress query.stop() ``` >> ingest to the non-exist table B ``` df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2') query2.lastProgress query2.stop() ``` >> ingest to the non-exist table C partitioned by `value % 10` ``` df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition') query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3') query3.lastProgress query3.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() ``` Closes #30835 from HeartSaVioR/SPARK-33836. Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
def table(self, tableName):
"""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
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable ### What changes were proposed in this pull request? This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming. ### Why are the changes needed? Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark. ### Does this PR introduce _any_ user-facing change? Yes, PySpark users will be able to read and write with table in Structured Streaming query. ### How was this patch tested? Manually tested. > v1 table >> create table A and ingest to the table A ``` spark.sql(""" create table table_pyspark_parquet ( value long, `timestamp` timestamp ) USING parquet """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5') query.lastProgress query.stop() ``` >> read table A and ingest to the table B which doesn't exist ``` df2 = spark.readStream.table('table_pyspark_parquet') query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2') query2.lastProgress query2.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE table_pyspark_parquet").show() spark.sql("SELECT * FROM table_pyspark_parquet").show() spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show() spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show() ``` > v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well) >> create table A and ingest to the table A ``` spark.sql(""" create table iceberg_catalog.default.table_pyspark_v2table ( value long, `timestamp` timestamp ) USING iceberg """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1') query.lastProgress query.stop() ``` >> ingest to the non-exist table B ``` df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2') query2.lastProgress query2.stop() ``` >> ingest to the non-exist table C partitioned by `value % 10` ``` df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition') query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3') query3.lastProgress query3.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() ``` Closes #30835 from HeartSaVioR/SPARK-33836. Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
"""
if isinstance(tableName, str):
return self._df(self._jreader.table(tableName))
else:
raise TypeError("tableName can be only a single string")
class DataStreamWriter(object):
"""
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.
"""
def __init__(self, df):
self._df = df
self._spark = df.sql_ctx
self._jwrite = df._jdf.writeStream()
def _sq(self, jsq):
from pyspark.sql.streaming import StreamingQuery
return StreamingQuery(jsq)
def outputMode(self, outputMode):
"""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
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable ### What changes were proposed in this pull request? This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming. ### Why are the changes needed? Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark. ### Does this PR introduce _any_ user-facing change? Yes, PySpark users will be able to read and write with table in Structured Streaming query. ### How was this patch tested? Manually tested. > v1 table >> create table A and ingest to the table A ``` spark.sql(""" create table table_pyspark_parquet ( value long, `timestamp` timestamp ) USING parquet """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5') query.lastProgress query.stop() ``` >> read table A and ingest to the table B which doesn't exist ``` df2 = spark.readStream.table('table_pyspark_parquet') query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2') query2.lastProgress query2.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE table_pyspark_parquet").show() spark.sql("SELECT * FROM table_pyspark_parquet").show() spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show() spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show() ``` > v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well) >> create table A and ingest to the table A ``` spark.sql(""" create table iceberg_catalog.default.table_pyspark_v2table ( value long, `timestamp` timestamp ) USING iceberg """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1') query.lastProgress query.stop() ``` >> ingest to the non-exist table B ``` df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2') query2.lastProgress query2.stop() ``` >> ingest to the non-exist table C partitioned by `value % 10` ``` df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition') query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3') query3.lastProgress query3.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() ``` Closes #30835 from HeartSaVioR/SPARK-33836. Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
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')
"""
if not outputMode or type(outputMode) != str or len(outputMode.strip()) == 0:
raise ValueError('The output mode must be a non-empty string. Got: %s' % outputMode)
self._jwrite = self._jwrite.outputMode(outputMode)
return self
def format(self, source):
"""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')
"""
self._jwrite = self._jwrite.format(source)
return self
def option(self, key, value):
"""Adds an output option for the underlying data source.
You can set the following option(s) for writing files:
* ``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.
.. versionadded:: 2.0.0
Notes
-----
This API is evolving.
"""
self._jwrite = self._jwrite.option(key, to_str(value))
return self
def options(self, **options):
"""Adds output options for the underlying data source.
You can set the following option(s) for writing files:
* ``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.
.. versionadded:: 2.0.0
Notes
-----
This API is evolving.
"""
for k in options:
self._jwrite = self._jwrite.option(k, to_str(options[k]))
return self
def partitionBy(self, *cols):
"""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.
"""
if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
cols = cols[0]
self._jwrite = self._jwrite.partitionBy(_to_seq(self._spark._sc, cols))
return self
def queryName(self, queryName):
"""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')
"""
if not queryName or type(queryName) != str or len(queryName.strip()) == 0:
raise ValueError('The queryName must be a non-empty string. Got: %s' % queryName)
self._jwrite = self._jwrite.queryName(queryName)
return self
@keyword_only
[SPARK-32933][PYTHON] Use keyword-only syntax for keyword_only methods ### What changes were proposed in this pull request? This PR adjusts signatures of methods decorated with `keyword_only` to indicate using [Python 3 keyword-only syntax](https://www.python.org/dev/peps/pep-3102/). __Note__: For the moment the goal is not to replace `keyword_only`. For justification see https://github.com/apache/spark/pull/29591#discussion_r489402579 ### Why are the changes needed? Right now it is not clear that `keyword_only` methods are indeed keyword only. This proposal addresses that. In practice we could probably capture `locals` and drop `keyword_only` completel, i.e: ```python keyword_only def __init__(self, *, featuresCol="features"): ... kwargs = self._input_kwargs self.setParams(**kwargs) ``` could be replaced with ```python def __init__(self, *, featuresCol="features"): kwargs = locals() del kwargs["self"] ... self.setParams(**kwargs) ``` ### Does this PR introduce _any_ user-facing change? Docstrings and inspect tools will now indicate that `keyword_only` methods expect only keyword arguments. For example with ` LinearSVC` will change from ``` >>> from pyspark.ml.classification import LinearSVC >>> ?LinearSVC.__init__ Signature: LinearSVC.__init__( self, featuresCol='features', labelCol='label', predictionCol='prediction', maxIter=100, regParam=0.0, tol=1e-06, rawPredictionCol='rawPrediction', fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, aggregationDepth=2, ) Docstring: __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, aggregationDepth=2): File: /path/to/python/pyspark/ml/classification.py Type: function ``` to ``` >>> from pyspark.ml.classification import LinearSVC >>> ?LinearSVC.__init__ Signature: LinearSVC.__init__ ( self, *, featuresCol='features', labelCol='label', predictionCol='prediction', maxIter=100, regParam=0.0, tol=1e-06, rawPredictionCol='rawPrediction', fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, aggregationDepth=2, blockSize=1, ) Docstring: __init__(self, \*, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, aggregationDepth=2, blockSize=1): File: ~/Workspace/spark/python/pyspark/ml/classification.py Type: function ``` ### How was this patch tested? Existing tests. Closes #29799 from zero323/SPARK-32933. Authored-by: zero323 <mszymkiewicz@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-22 20:28:33 -04:00
def trigger(self, *, processingTime=None, once=None, continuous=None):
"""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')
[SPARK-19876][SS][WIP] OneTime Trigger Executor ## What changes were proposed in this pull request? An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers. In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature. ## How was this patch tested? A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly. In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests: - The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop). - The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log. - A OneTime trigger execution that results in an exception being thrown. marmbrus tdas zsxwing Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Tyson Condie <tcondie@gmail.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17219 from tcondie/stream-commit.
2017-03-23 17:32:05 -04:00
>>> # 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')
"""
params = [processingTime, once, continuous]
if params.count(None) == 3:
raise ValueError('No trigger provided')
elif params.count(None) < 2:
raise ValueError('Multiple triggers not allowed.')
[SPARK-19876][SS][WIP] OneTime Trigger Executor ## What changes were proposed in this pull request? An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers. In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature. ## How was this patch tested? A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly. In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests: - The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop). - The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log. - A OneTime trigger execution that results in an exception being thrown. marmbrus tdas zsxwing Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Tyson Condie <tcondie@gmail.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17219 from tcondie/stream-commit.
2017-03-23 17:32:05 -04:00
jTrigger = None
if processingTime is not None:
if type(processingTime) != str or len(processingTime.strip()) == 0:
[SPARK-19876][SS][WIP] OneTime Trigger Executor ## What changes were proposed in this pull request? An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers. In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature. ## How was this patch tested? A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly. In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests: - The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop). - The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log. - A OneTime trigger execution that results in an exception being thrown. marmbrus tdas zsxwing Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Tyson Condie <tcondie@gmail.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17219 from tcondie/stream-commit.
2017-03-23 17:32:05 -04:00
raise ValueError('Value for processingTime must be a non empty string. Got: %s' %
processingTime)
[SPARK-19876][SS][WIP] OneTime Trigger Executor ## What changes were proposed in this pull request? An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers. In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature. ## How was this patch tested? A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly. In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests: - The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop). - The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log. - A OneTime trigger execution that results in an exception being thrown. marmbrus tdas zsxwing Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Tyson Condie <tcondie@gmail.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17219 from tcondie/stream-commit.
2017-03-23 17:32:05 -04:00
interval = processingTime.strip()
jTrigger = self._spark._sc._jvm.org.apache.spark.sql.streaming.Trigger.ProcessingTime(
interval)
[SPARK-19876][SS][WIP] OneTime Trigger Executor ## What changes were proposed in this pull request? An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers. In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature. ## How was this patch tested? A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly. In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests: - The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop). - The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log. - A OneTime trigger execution that results in an exception being thrown. marmbrus tdas zsxwing Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Tyson Condie <tcondie@gmail.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17219 from tcondie/stream-commit.
2017-03-23 17:32:05 -04:00
elif once is not None:
if once is not True:
raise ValueError('Value for once must be True. Got: %s' % once)
jTrigger = self._spark._sc._jvm.org.apache.spark.sql.streaming.Trigger.Once()
[SPARK-19876][SS][WIP] OneTime Trigger Executor ## What changes were proposed in this pull request? An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers. In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature. ## How was this patch tested? A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly. In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests: - The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop). - The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log. - A OneTime trigger execution that results in an exception being thrown. marmbrus tdas zsxwing Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Tyson Condie <tcondie@gmail.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17219 from tcondie/stream-commit.
2017-03-23 17:32:05 -04:00
else:
if type(continuous) != str or len(continuous.strip()) == 0:
raise ValueError('Value for continuous must be a non empty string. Got: %s' %
continuous)
interval = continuous.strip()
jTrigger = self._spark._sc._jvm.org.apache.spark.sql.streaming.Trigger.Continuous(
interval)
[SPARK-19876][SS][WIP] OneTime Trigger Executor ## What changes were proposed in this pull request? An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers. In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature. ## How was this patch tested? A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly. In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests: - The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop). - The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log. - A OneTime trigger execution that results in an exception being thrown. marmbrus tdas zsxwing Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Tyson Condie <tcondie@gmail.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17219 from tcondie/stream-commit.
2017-03-23 17:32:05 -04:00
self._jwrite = self._jwrite.trigger(jTrigger)
return self
def foreach(self, f):
"""
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())
"""
from pyspark.rdd import _wrap_function
from pyspark.serializers import PickleSerializer, AutoBatchedSerializer
from pyspark.taskcontext import TaskContext
if callable(f):
# 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.
def func_without_process(_, iterator):
for x in iterator:
f(x)
return iter([])
func = func_without_process
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.
if not hasattr(f, 'process'):
raise Exception("Provided object does not have a 'process' method")
if not callable(getattr(f, 'process')):
raise Exception("Attribute 'process' in provided object is not callable")
def doesMethodExist(method_name):
exists = hasattr(f, method_name)
if exists and not callable(getattr(f, method_name)):
raise Exception(
"Attribute '%s' in provided object is not callable" % method_name)
return exists
open_exists = doesMethodExist('open')
close_exists = doesMethodExist('close')
def func_with_open_process_close(partition_id, iterator):
epoch_id = TaskContext.get().getLocalProperty('streaming.sql.batchId')
if epoch_id:
epoch_id = int(epoch_id)
else:
raise Exception("Could not get batch id from TaskContext")
# Check if the data should be processed
should_process = True
if open_exists:
should_process = f.open(partition_id, epoch_id)
error = None
try:
if should_process:
for x in iterator:
f.process(x)
except Exception as ex:
error = ex
finally:
if close_exists:
f.close(error)
if error:
raise error
return iter([])
func = func_with_open_process_close
serializer = AutoBatchedSerializer(PickleSerializer())
wrapped_func = _wrap_function(self._spark._sc, func, serializer, serializer)
jForeachWriter = \
self._spark._sc._jvm.org.apache.spark.sql.execution.python.PythonForeachWriter(
wrapped_func, self._df._jdf.schema())
self._jwrite.foreach(jForeachWriter)
return self
def foreachBatch(self, func):
"""
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)
"""
from pyspark.java_gateway import ensure_callback_server_started
gw = self._spark._sc._gateway
java_import(gw.jvm, "org.apache.spark.sql.execution.streaming.sources.*")
wrapped_func = ForeachBatchFunction(self._spark, func)
gw.jvm.PythonForeachBatchHelper.callForeachBatch(self._jwrite, wrapped_func)
ensure_callback_server_started(gw)
return self
def start(self, path=None, format=None, outputMode=None, partitionBy=None, queryName=None,
**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
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable ### What changes were proposed in this pull request? This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming. ### Why are the changes needed? Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark. ### Does this PR introduce _any_ user-facing change? Yes, PySpark users will be able to read and write with table in Structured Streaming query. ### How was this patch tested? Manually tested. > v1 table >> create table A and ingest to the table A ``` spark.sql(""" create table table_pyspark_parquet ( value long, `timestamp` timestamp ) USING parquet """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5') query.lastProgress query.stop() ``` >> read table A and ingest to the table B which doesn't exist ``` df2 = spark.readStream.table('table_pyspark_parquet') query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2') query2.lastProgress query2.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE table_pyspark_parquet").show() spark.sql("SELECT * FROM table_pyspark_parquet").show() spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show() spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show() ``` > v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well) >> create table A and ingest to the table A ``` spark.sql(""" create table iceberg_catalog.default.table_pyspark_v2table ( value long, `timestamp` timestamp ) USING iceberg """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1') query.lastProgress query.stop() ``` >> ingest to the non-exist table B ``` df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2') query2.lastProgress query2.stop() ``` >> ingest to the non-exist table C partitioned by `value % 10` ``` df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition') query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3') query3.lastProgress query3.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() ``` Closes #30835 from HeartSaVioR/SPARK-33836. Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
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.options(**options)
if outputMode is not None:
self.outputMode(outputMode)
if partitionBy is not None:
self.partitionBy(partitionBy)
if format is not None:
self.format(format)
if queryName is not None:
self.queryName(queryName)
if path is None:
return self._sq(self._jwrite.start())
else:
return self._sq(self._jwrite.start(path))
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable ### What changes were proposed in this pull request? This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming. ### Why are the changes needed? Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark. ### Does this PR introduce _any_ user-facing change? Yes, PySpark users will be able to read and write with table in Structured Streaming query. ### How was this patch tested? Manually tested. > v1 table >> create table A and ingest to the table A ``` spark.sql(""" create table table_pyspark_parquet ( value long, `timestamp` timestamp ) USING parquet """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5') query.lastProgress query.stop() ``` >> read table A and ingest to the table B which doesn't exist ``` df2 = spark.readStream.table('table_pyspark_parquet') query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2') query2.lastProgress query2.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE table_pyspark_parquet").show() spark.sql("SELECT * FROM table_pyspark_parquet").show() spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show() spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show() ``` > v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well) >> create table A and ingest to the table A ``` spark.sql(""" create table iceberg_catalog.default.table_pyspark_v2table ( value long, `timestamp` timestamp ) USING iceberg """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1') query.lastProgress query.stop() ``` >> ingest to the non-exist table B ``` df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2') query2.lastProgress query2.stop() ``` >> ingest to the non-exist table C partitioned by `value % 10` ``` df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition') query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3') query3.lastProgress query3.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() ``` Closes #30835 from HeartSaVioR/SPARK-33836. Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
def toTable(self, tableName, format=None, outputMode=None, partitionBy=None, queryName=None,
**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.
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable ### What changes were proposed in this pull request? This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming. ### Why are the changes needed? Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark. ### Does this PR introduce _any_ user-facing change? Yes, PySpark users will be able to read and write with table in Structured Streaming query. ### How was this patch tested? Manually tested. > v1 table >> create table A and ingest to the table A ``` spark.sql(""" create table table_pyspark_parquet ( value long, `timestamp` timestamp ) USING parquet """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5') query.lastProgress query.stop() ``` >> read table A and ingest to the table B which doesn't exist ``` df2 = spark.readStream.table('table_pyspark_parquet') query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2') query2.lastProgress query2.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE table_pyspark_parquet").show() spark.sql("SELECT * FROM table_pyspark_parquet").show() spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show() spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show() ``` > v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well) >> create table A and ingest to the table A ``` spark.sql(""" create table iceberg_catalog.default.table_pyspark_v2table ( value long, `timestamp` timestamp ) USING iceberg """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1') query.lastProgress query.stop() ``` >> ingest to the non-exist table B ``` df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2') query2.lastProgress query2.stop() ``` >> ingest to the non-exist table C partitioned by `value % 10` ``` df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition') query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3') query3.lastProgress query3.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() ``` Closes #30835 from HeartSaVioR/SPARK-33836. Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
.. 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.
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable ### What changes were proposed in this pull request? This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming. ### Why are the changes needed? Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark. ### Does this PR introduce _any_ user-facing change? Yes, PySpark users will be able to read and write with table in Structured Streaming query. ### How was this patch tested? Manually tested. > v1 table >> create table A and ingest to the table A ``` spark.sql(""" create table table_pyspark_parquet ( value long, `timestamp` timestamp ) USING parquet """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5') query.lastProgress query.stop() ``` >> read table A and ingest to the table B which doesn't exist ``` df2 = spark.readStream.table('table_pyspark_parquet') query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2') query2.lastProgress query2.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE table_pyspark_parquet").show() spark.sql("SELECT * FROM table_pyspark_parquet").show() spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show() spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show() ``` > v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well) >> create table A and ingest to the table A ``` spark.sql(""" create table iceberg_catalog.default.table_pyspark_v2table ( value long, `timestamp` timestamp ) USING iceberg """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1') query.lastProgress query.stop() ``` >> ingest to the non-exist table B ``` df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2') query2.lastProgress query2.stop() ``` >> ingest to the non-exist table C partitioned by `value % 10` ``` df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition') query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3') query3.lastProgress query3.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() ``` Closes #30835 from HeartSaVioR/SPARK-33836. Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
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',
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable ### What changes were proposed in this pull request? This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming. ### Why are the changes needed? Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark. ### Does this PR introduce _any_ user-facing change? Yes, PySpark users will be able to read and write with table in Structured Streaming query. ### How was this patch tested? Manually tested. > v1 table >> create table A and ingest to the table A ``` spark.sql(""" create table table_pyspark_parquet ( value long, `timestamp` timestamp ) USING parquet """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5') query.lastProgress query.stop() ``` >> read table A and ingest to the table B which doesn't exist ``` df2 = spark.readStream.table('table_pyspark_parquet') query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2') query2.lastProgress query2.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE table_pyspark_parquet").show() spark.sql("SELECT * FROM table_pyspark_parquet").show() spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show() spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show() ``` > v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well) >> create table A and ingest to the table A ``` spark.sql(""" create table iceberg_catalog.default.table_pyspark_v2table ( value long, `timestamp` timestamp ) USING iceberg """) df = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1') query.lastProgress query.stop() ``` >> ingest to the non-exist table B ``` df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2') query2.lastProgress query2.stop() ``` >> ingest to the non-exist table C partitioned by `value % 10` ``` df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load() df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition') query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3') query3.lastProgress query3.stop() ``` >> select tables ``` spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show() spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show() ``` Closes #30835 from HeartSaVioR/SPARK-33836. Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
... checkpointLocation='/tmp/checkpoint') # doctest: +SKIP
"""
self.options(**options)
if outputMode is not None:
self.outputMode(outputMode)
if partitionBy is not None:
self.partitionBy(partitionBy)
if format is not None:
self.format(format)
if queryName is not None:
self.queryName(queryName)
return self._sq(self._jwrite.toTable(tableName))
def _test():
import doctest
import os
import tempfile
[SPARK-32319][PYSPARK] Disallow the use of unused imports Disallow the use of unused imports: - Unnecessary increases the memory footprint of the application - Removes the imports that are required for the examples in the docstring from the file-scope to the example itself. This keeps the files itself clean, and gives a more complete example as it also includes the imports :) ``` fokkodriesprongFan spark % flake8 python | grep -i "imported but unused" python/pyspark/cloudpickle.py:46:1: F401 'functools.partial' imported but unused python/pyspark/cloudpickle.py:55:1: F401 'traceback' imported but unused python/pyspark/heapq3.py:868:5: F401 '_heapq.*' imported but unused python/pyspark/__init__.py:61:1: F401 'pyspark.version.__version__' imported but unused python/pyspark/__init__.py:62:1: F401 'pyspark._globals._NoValue' imported but unused python/pyspark/__init__.py:115:1: F401 'pyspark.sql.SQLContext' imported but unused python/pyspark/__init__.py:115:1: F401 'pyspark.sql.HiveContext' imported but unused python/pyspark/__init__.py:115:1: F401 'pyspark.sql.Row' imported but unused python/pyspark/rdd.py:21:1: F401 're' imported but unused python/pyspark/rdd.py:29:1: F401 'tempfile.NamedTemporaryFile' imported but unused python/pyspark/mllib/regression.py:26:1: F401 'pyspark.mllib.linalg.SparseVector' imported but unused python/pyspark/mllib/clustering.py:28:1: F401 'pyspark.mllib.linalg.SparseVector' imported but unused python/pyspark/mllib/clustering.py:28:1: F401 'pyspark.mllib.linalg.DenseVector' imported but unused python/pyspark/mllib/classification.py:26:1: F401 'pyspark.mllib.linalg.SparseVector' imported but unused python/pyspark/mllib/feature.py:28:1: F401 'pyspark.mllib.linalg.DenseVector' imported but unused python/pyspark/mllib/feature.py:28:1: F401 'pyspark.mllib.linalg.SparseVector' imported but unused python/pyspark/mllib/feature.py:30:1: F401 'pyspark.mllib.regression.LabeledPoint' imported but unused python/pyspark/mllib/tests/test_linalg.py:18:1: F401 'sys' imported but unused python/pyspark/mllib/tests/test_linalg.py:642:5: F401 'pyspark.mllib.tests.test_linalg.*' imported but unused python/pyspark/mllib/tests/test_feature.py:21:1: F401 'numpy.random' imported but unused python/pyspark/mllib/tests/test_feature.py:21:1: F401 'numpy.exp' imported but unused python/pyspark/mllib/tests/test_feature.py:23:1: F401 'pyspark.mllib.linalg.Vector' imported but unused python/pyspark/mllib/tests/test_feature.py:23:1: F401 'pyspark.mllib.linalg.VectorUDT' imported but unused python/pyspark/mllib/tests/test_feature.py:185:5: F401 'pyspark.mllib.tests.test_feature.*' imported but unused python/pyspark/mllib/tests/test_util.py:97:5: F401 'pyspark.mllib.tests.test_util.*' imported but unused python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.Vector' imported but unused python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.SparseVector' imported but unused python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.DenseVector' imported but unused python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.VectorUDT' imported but unused python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg._convert_to_vector' imported but unused python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.DenseMatrix' imported but unused python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.SparseMatrix' imported but unused python/pyspark/mllib/tests/test_stat.py:23:1: F401 'pyspark.mllib.linalg.MatrixUDT' imported but unused python/pyspark/mllib/tests/test_stat.py:181:5: F401 'pyspark.mllib.tests.test_stat.*' imported but unused python/pyspark/mllib/tests/test_streaming_algorithms.py:18:1: F401 'time.time' imported but unused python/pyspark/mllib/tests/test_streaming_algorithms.py:18:1: F401 'time.sleep' imported but unused python/pyspark/mllib/tests/test_streaming_algorithms.py:470:5: F401 'pyspark.mllib.tests.test_streaming_algorithms.*' imported but unused python/pyspark/mllib/tests/test_algorithms.py:295:5: F401 'pyspark.mllib.tests.test_algorithms.*' imported but unused python/pyspark/tests/test_serializers.py:90:13: F401 'xmlrunner' imported but unused python/pyspark/tests/test_rdd.py:21:1: F401 'sys' imported but unused python/pyspark/tests/test_rdd.py:29:1: F401 'pyspark.resource.ResourceProfile' imported but unused python/pyspark/tests/test_rdd.py:885:5: F401 'pyspark.tests.test_rdd.*' imported but unused python/pyspark/tests/test_readwrite.py:19:1: F401 'sys' imported but unused python/pyspark/tests/test_readwrite.py:22:1: F401 'array.array' imported but unused python/pyspark/tests/test_readwrite.py:309:5: F401 'pyspark.tests.test_readwrite.*' imported but unused python/pyspark/tests/test_join.py:62:5: F401 'pyspark.tests.test_join.*' imported but unused python/pyspark/tests/test_taskcontext.py:19:1: F401 'shutil' imported but unused python/pyspark/tests/test_taskcontext.py:325:5: F401 'pyspark.tests.test_taskcontext.*' imported but unused python/pyspark/tests/test_conf.py:36:5: F401 'pyspark.tests.test_conf.*' imported but unused python/pyspark/tests/test_broadcast.py:148:5: F401 'pyspark.tests.test_broadcast.*' imported but unused python/pyspark/tests/test_daemon.py:76:5: F401 'pyspark.tests.test_daemon.*' imported but unused python/pyspark/tests/test_util.py:77:5: F401 'pyspark.tests.test_util.*' imported but unused python/pyspark/tests/test_pin_thread.py:19:1: F401 'random' imported but unused python/pyspark/tests/test_pin_thread.py:149:5: F401 'pyspark.tests.test_pin_thread.*' imported but unused python/pyspark/tests/test_worker.py:19:1: F401 'sys' imported but unused python/pyspark/tests/test_worker.py:26:5: F401 'resource' imported but unused python/pyspark/tests/test_worker.py:203:5: F401 'pyspark.tests.test_worker.*' imported but unused python/pyspark/tests/test_profiler.py:101:5: F401 'pyspark.tests.test_profiler.*' imported but unused python/pyspark/tests/test_shuffle.py:18:1: F401 'sys' imported but unused python/pyspark/tests/test_shuffle.py:171:5: F401 'pyspark.tests.test_shuffle.*' imported but unused python/pyspark/tests/test_rddbarrier.py:43:5: F401 'pyspark.tests.test_rddbarrier.*' imported but unused python/pyspark/tests/test_context.py:129:13: F401 'userlibrary.UserClass' imported but unused python/pyspark/tests/test_context.py:140:13: F401 'userlib.UserClass' imported but unused python/pyspark/tests/test_context.py:310:5: F401 'pyspark.tests.test_context.*' imported but unused python/pyspark/tests/test_appsubmit.py:241:5: F401 'pyspark.tests.test_appsubmit.*' imported but unused python/pyspark/streaming/dstream.py:18:1: F401 'sys' imported but unused python/pyspark/streaming/tests/test_dstream.py:27:1: F401 'pyspark.RDD' imported but unused python/pyspark/streaming/tests/test_dstream.py:647:5: F401 'pyspark.streaming.tests.test_dstream.*' imported but unused python/pyspark/streaming/tests/test_kinesis.py:83:5: F401 'pyspark.streaming.tests.test_kinesis.*' imported but unused python/pyspark/streaming/tests/test_listener.py:152:5: F401 'pyspark.streaming.tests.test_listener.*' imported but unused python/pyspark/streaming/tests/test_context.py:178:5: F401 'pyspark.streaming.tests.test_context.*' imported but unused python/pyspark/testing/utils.py:30:5: F401 'scipy.sparse' imported but unused python/pyspark/testing/utils.py:36:5: F401 'numpy as np' imported but unused python/pyspark/ml/regression.py:25:1: F401 'pyspark.ml.tree._TreeEnsembleParams' imported but unused python/pyspark/ml/regression.py:25:1: F401 'pyspark.ml.tree._HasVarianceImpurity' imported but unused python/pyspark/ml/regression.py:29:1: F401 'pyspark.ml.wrapper.JavaParams' imported but unused python/pyspark/ml/util.py:19:1: F401 'sys' imported but unused python/pyspark/ml/__init__.py:25:1: F401 'pyspark.ml.pipeline' imported but unused python/pyspark/ml/pipeline.py:18:1: F401 'sys' imported but unused python/pyspark/ml/stat.py:22:1: F401 'pyspark.ml.linalg.DenseMatrix' imported but unused python/pyspark/ml/stat.py:22:1: F401 'pyspark.ml.linalg.Vectors' imported but unused python/pyspark/ml/tests/test_training_summary.py:18:1: F401 'sys' imported but unused python/pyspark/ml/tests/test_training_summary.py:364:5: F401 'pyspark.ml.tests.test_training_summary.*' imported but unused python/pyspark/ml/tests/test_linalg.py:381:5: F401 'pyspark.ml.tests.test_linalg.*' imported but unused python/pyspark/ml/tests/test_tuning.py:427:9: F401 'pyspark.sql.functions as F' imported but unused python/pyspark/ml/tests/test_tuning.py:757:5: F401 'pyspark.ml.tests.test_tuning.*' imported but unused python/pyspark/ml/tests/test_wrapper.py:120:5: F401 'pyspark.ml.tests.test_wrapper.*' imported but unused python/pyspark/ml/tests/test_feature.py:19:1: F401 'sys' imported but unused python/pyspark/ml/tests/test_feature.py:304:5: F401 'pyspark.ml.tests.test_feature.*' imported but unused python/pyspark/ml/tests/test_image.py:19:1: F401 'py4j' imported but unused python/pyspark/ml/tests/test_image.py:22:1: F401 'pyspark.testing.mlutils.PySparkTestCase' imported but unused python/pyspark/ml/tests/test_image.py:71:5: F401 'pyspark.ml.tests.test_image.*' imported but unused python/pyspark/ml/tests/test_persistence.py:456:5: F401 'pyspark.ml.tests.test_persistence.*' imported but unused python/pyspark/ml/tests/test_evaluation.py:56:5: F401 'pyspark.ml.tests.test_evaluation.*' imported but unused python/pyspark/ml/tests/test_stat.py:43:5: F401 'pyspark.ml.tests.test_stat.*' imported but unused python/pyspark/ml/tests/test_base.py:70:5: F401 'pyspark.ml.tests.test_base.*' imported but unused python/pyspark/ml/tests/test_param.py:20:1: F401 'sys' imported but unused python/pyspark/ml/tests/test_param.py:375:5: F401 'pyspark.ml.tests.test_param.*' imported but unused python/pyspark/ml/tests/test_pipeline.py:62:5: F401 'pyspark.ml.tests.test_pipeline.*' imported but unused python/pyspark/ml/tests/test_algorithms.py:333:5: F401 'pyspark.ml.tests.test_algorithms.*' imported but unused python/pyspark/ml/param/__init__.py:18:1: F401 'sys' imported but unused python/pyspark/resource/tests/test_resources.py:17:1: F401 'random' imported but unused python/pyspark/resource/tests/test_resources.py:20:1: F401 'pyspark.resource.ResourceProfile' imported but unused python/pyspark/resource/tests/test_resources.py:75:5: F401 'pyspark.resource.tests.test_resources.*' imported but unused python/pyspark/sql/functions.py:32:1: F401 'pyspark.sql.udf.UserDefinedFunction' imported but unused python/pyspark/sql/functions.py:34:1: F401 'pyspark.sql.pandas.functions.pandas_udf' imported but unused python/pyspark/sql/session.py:30:1: F401 'pyspark.sql.types.Row' imported but unused python/pyspark/sql/session.py:30:1: F401 'pyspark.sql.types.StringType' imported but unused python/pyspark/sql/readwriter.py:1084:5: F401 'pyspark.sql.Row' imported but unused python/pyspark/sql/context.py:26:1: F401 'pyspark.sql.types.IntegerType' imported but unused python/pyspark/sql/context.py:26:1: F401 'pyspark.sql.types.Row' imported but unused python/pyspark/sql/context.py:26:1: F401 'pyspark.sql.types.StringType' imported but unused python/pyspark/sql/context.py:27:1: F401 'pyspark.sql.udf.UDFRegistration' imported but unused python/pyspark/sql/streaming.py:1212:5: F401 'pyspark.sql.Row' imported but unused python/pyspark/sql/tests/test_utils.py:55:5: F401 'pyspark.sql.tests.test_utils.*' imported but unused python/pyspark/sql/tests/test_pandas_map.py:18:1: F401 'sys' imported but unused python/pyspark/sql/tests/test_pandas_map.py:22:1: F401 'pyspark.sql.functions.pandas_udf' imported but unused python/pyspark/sql/tests/test_pandas_map.py:22:1: F401 'pyspark.sql.functions.PandasUDFType' imported but unused python/pyspark/sql/tests/test_pandas_map.py:119:5: F401 'pyspark.sql.tests.test_pandas_map.*' imported but unused python/pyspark/sql/tests/test_catalog.py:193:5: F401 'pyspark.sql.tests.test_catalog.*' imported but unused python/pyspark/sql/tests/test_group.py:39:5: F401 'pyspark.sql.tests.test_group.*' imported but unused python/pyspark/sql/tests/test_session.py:361:5: F401 'pyspark.sql.tests.test_session.*' imported but unused python/pyspark/sql/tests/test_conf.py:49:5: F401 'pyspark.sql.tests.test_conf.*' imported but unused python/pyspark/sql/tests/test_pandas_cogrouped_map.py:19:1: F401 'sys' imported but unused python/pyspark/sql/tests/test_pandas_cogrouped_map.py:21:1: F401 'pyspark.sql.functions.sum' imported but unused python/pyspark/sql/tests/test_pandas_cogrouped_map.py:21:1: F401 'pyspark.sql.functions.PandasUDFType' imported but unused python/pyspark/sql/tests/test_pandas_cogrouped_map.py:29:5: F401 'pandas.util.testing.assert_series_equal' imported but unused python/pyspark/sql/tests/test_pandas_cogrouped_map.py:32:5: F401 'pyarrow as pa' imported but unused python/pyspark/sql/tests/test_pandas_cogrouped_map.py:248:5: F401 'pyspark.sql.tests.test_pandas_cogrouped_map.*' imported but unused python/pyspark/sql/tests/test_udf.py:24:1: F401 'py4j' imported but unused python/pyspark/sql/tests/test_pandas_udf_typehints.py:246:5: F401 'pyspark.sql.tests.test_pandas_udf_typehints.*' imported but unused python/pyspark/sql/tests/test_functions.py:19:1: F401 'sys' imported but unused python/pyspark/sql/tests/test_functions.py:362:9: F401 'pyspark.sql.functions.exists' imported but unused python/pyspark/sql/tests/test_functions.py:387:5: F401 'pyspark.sql.tests.test_functions.*' imported but unused python/pyspark/sql/tests/test_pandas_udf_scalar.py:21:1: F401 'sys' imported but unused python/pyspark/sql/tests/test_pandas_udf_scalar.py:45:5: F401 'pyarrow as pa' imported but unused python/pyspark/sql/tests/test_pandas_udf_window.py:355:5: F401 'pyspark.sql.tests.test_pandas_udf_window.*' imported but unused python/pyspark/sql/tests/test_arrow.py:38:5: F401 'pyarrow as pa' imported but unused python/pyspark/sql/tests/test_pandas_grouped_map.py:20:1: F401 'sys' imported but unused python/pyspark/sql/tests/test_pandas_grouped_map.py:38:5: F401 'pyarrow as pa' imported but unused python/pyspark/sql/tests/test_dataframe.py:382:9: F401 'pyspark.sql.DataFrame' imported but unused python/pyspark/sql/avro/functions.py:125:5: F401 'pyspark.sql.Row' imported but unused python/pyspark/sql/pandas/functions.py:19:1: F401 'sys' imported but unused ``` After: ``` fokkodriesprongFan spark % flake8 python | grep -i "imported but unused" fokkodriesprongFan spark % ``` ### What changes were proposed in this pull request? Removing unused imports from the Python files to keep everything nice and tidy. ### Why are the changes needed? Cleaning up of the imports that aren't used, and suppressing the imports that are used as references to other modules, preserving backward compatibility. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Adding the rule to the existing Flake8 checks. Closes #29121 from Fokko/SPARK-32319. Authored-by: Fokko Driesprong <fokko@apache.org> Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-08 11:51:57 -04:00
from pyspark.sql import SparkSession, SQLContext
import pyspark.sql.streaming
os.chdir(os.environ["SPARK_HOME"])
globs = pyspark.sql.streaming.__dict__.copy()
try:
spark = SparkSession.builder.getOrCreate()
except py4j.protocol.Py4JError: # noqa: F821
spark = SparkSession(sc) # noqa: F821
globs['tempfile'] = tempfile
globs['os'] = os
globs['spark'] = spark
globs['sqlContext'] = SQLContext.getOrCreate(spark.sparkContext)
globs['sdf'] = \
spark.readStream.format('text').load('python/test_support/sql/streaming')
globs['sdf_schema'] = StructType([StructField("data", StringType(), True)])
globs['df'] = \
globs['spark'].readStream.format('text').load('python/test_support/sql/streaming')
(failure_count, test_count) = doctest.testmod(
pyspark.sql.streaming, globs=globs,
optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF)
globs['spark'].stop()
[SPARK-17731][SQL][STREAMING] Metrics for structured streaming ## What changes were proposed in this pull request? Metrics are needed for monitoring structured streaming apps. Here is the design doc for implementing the necessary metrics. https://docs.google.com/document/d/1NIdcGuR1B3WIe8t7VxLrt58TJB4DtipWEbj5I_mzJys/edit?usp=sharing Specifically, this PR adds the following public APIs changes. ### New APIs - `StreamingQuery.status` returns a `StreamingQueryStatus` object (renamed from `StreamingQueryInfo`, see later) - `StreamingQueryStatus` has the following important fields - inputRate - Current rate (rows/sec) at which data is being generated by all the sources - processingRate - Current rate (rows/sec) at which the query is processing data from all the sources - ~~outputRate~~ - *Does not work with wholestage codegen* - latency - Current average latency between the data being available in source and the sink writing the corresponding output - sourceStatuses: Array[SourceStatus] - Current statuses of the sources - sinkStatus: SinkStatus - Current status of the sink - triggerStatus - Low-level detailed status of the last completed/currently active trigger - latencies - getOffset, getBatch, full trigger, wal writes - timestamps - trigger start, finish, after getOffset, after getBatch - numRows - input, output, state total/updated rows for aggregations - `SourceStatus` has the following important fields - inputRate - Current rate (rows/sec) at which data is being generated by the source - processingRate - Current rate (rows/sec) at which the query is processing data from the source - triggerStatus - Low-level detailed status of the last completed/currently active trigger - Python API for `StreamingQuery.status()` ### Breaking changes to existing APIs **Existing direct public facing APIs** - Deprecated direct public-facing APIs `StreamingQuery.sourceStatuses` and `StreamingQuery.sinkStatus` in favour of `StreamingQuery.status.sourceStatuses/sinkStatus`. - Branch 2.0 should have it deprecated, master should have it removed. **Existing advanced listener APIs** - `StreamingQueryInfo` renamed to `StreamingQueryStatus` for consistency with `SourceStatus`, `SinkStatus` - Earlier StreamingQueryInfo was used only in the advanced listener API, but now it is used in direct public-facing API (StreamingQuery.status) - Field `queryInfo` in listener events `QueryStarted`, `QueryProgress`, `QueryTerminated` changed have name `queryStatus` and return type `StreamingQueryStatus`. - Field `offsetDesc` in `SourceStatus` was Option[String], converted it to `String`. - For `SourceStatus` and `SinkStatus` made constructor private instead of private[sql] to make them more java-safe. Instead added `private[sql] object SourceStatus/SinkStatus.apply()` which are harder to accidentally use in Java. ## How was this patch tested? Old and new unit tests. - Rate calculation and other internal logic of StreamMetrics tested by StreamMetricsSuite. - New info in statuses returned through StreamingQueryListener is tested in StreamingQueryListenerSuite. - New and old info returned through StreamingQuery.status is tested in StreamingQuerySuite. - Source-specific tests for making sure input rows are counted are is source-specific test suites. - Additional tests to test minor additions in LocalTableScanExec, StateStore, etc. Metrics also manually tested using Ganglia sink Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #15307 from tdas/SPARK-17731.
2016-10-13 16:36:26 -04:00
if failure_count:
sys.exit(-1)
if __name__ == "__main__":
_test()