2015-05-19 17:23:28 -04:00
|
|
|
#
|
|
|
|
# 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.
|
|
|
|
#
|
2020-08-30 22:23:31 -04:00
|
|
|
import sys
|
2015-05-19 17:23:28 -04:00
|
|
|
|
|
|
|
from py4j.java_gateway import JavaClass
|
|
|
|
|
2018-03-25 23:42:32 -04:00
|
|
|
from pyspark import RDD, since
|
2020-07-19 21:42:33 -04:00
|
|
|
from pyspark.sql.column import _to_seq, _to_java_column
|
2020-08-30 22:23:31 -04:00
|
|
|
from pyspark.sql.types import StructType
|
2015-11-18 11:18:54 -05:00
|
|
|
from pyspark.sql import utils
|
2019-07-18 00:37:03 -04:00
|
|
|
from pyspark.sql.utils import to_str
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2016-06-29 01:07:11 -04:00
|
|
|
__all__ = ["DataFrameReader", "DataFrameWriter"]
|
2015-05-19 17:23:28 -04:00
|
|
|
|
|
|
|
|
2016-06-28 16:43:59 -04:00
|
|
|
class OptionUtils(object):
|
[SPARK-15981][SQL][STREAMING] Fixed bug and added tests in DataStreamReader Python API
## What changes were proposed in this pull request?
- Fixed bug in Python API of DataStreamReader. Because a single path was being converted to a array before calling Java DataStreamReader method (which takes a string only), it gave the following error.
```
File "/Users/tdas/Projects/Spark/spark/python/pyspark/sql/readwriter.py", line 947, in pyspark.sql.readwriter.DataStreamReader.json
Failed example:
json_sdf = spark.readStream.json(os.path.join(tempfile.mkdtemp(), 'data'), schema = sdf_schema)
Exception raised:
Traceback (most recent call last):
File "/System/Library/Frameworks/Python.framework/Versions/2.6/lib/python2.6/doctest.py", line 1253, in __run
compileflags, 1) in test.globs
File "<doctest pyspark.sql.readwriter.DataStreamReader.json[0]>", line 1, in <module>
json_sdf = spark.readStream.json(os.path.join(tempfile.mkdtemp(), 'data'), schema = sdf_schema)
File "/Users/tdas/Projects/Spark/spark/python/pyspark/sql/readwriter.py", line 963, in json
return self._df(self._jreader.json(path))
File "/Users/tdas/Projects/Spark/spark/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__
answer, self.gateway_client, self.target_id, self.name)
File "/Users/tdas/Projects/Spark/spark/python/pyspark/sql/utils.py", line 63, in deco
return f(*a, **kw)
File "/Users/tdas/Projects/Spark/spark/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 316, in get_return_value
format(target_id, ".", name, value))
Py4JError: An error occurred while calling o121.json. Trace:
py4j.Py4JException: Method json([class java.util.ArrayList]) does not exist
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318)
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:326)
at py4j.Gateway.invoke(Gateway.java:272)
at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:128)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at py4j.GatewayConnection.run(GatewayConnection.java:211)
at java.lang.Thread.run(Thread.java:744)
```
- Reduced code duplication between DataStreamReader and DataFrameWriter
- Added missing Python doctests
## How was this patch tested?
New tests
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #13703 from tdas/SPARK-15981.
2016-06-16 16:17:41 -04:00
|
|
|
|
2016-06-28 16:43:59 -04:00
|
|
|
def _set_opts(self, schema=None, **options):
|
[SPARK-15981][SQL][STREAMING] Fixed bug and added tests in DataStreamReader Python API
## What changes were proposed in this pull request?
- Fixed bug in Python API of DataStreamReader. Because a single path was being converted to a array before calling Java DataStreamReader method (which takes a string only), it gave the following error.
```
File "/Users/tdas/Projects/Spark/spark/python/pyspark/sql/readwriter.py", line 947, in pyspark.sql.readwriter.DataStreamReader.json
Failed example:
json_sdf = spark.readStream.json(os.path.join(tempfile.mkdtemp(), 'data'), schema = sdf_schema)
Exception raised:
Traceback (most recent call last):
File "/System/Library/Frameworks/Python.framework/Versions/2.6/lib/python2.6/doctest.py", line 1253, in __run
compileflags, 1) in test.globs
File "<doctest pyspark.sql.readwriter.DataStreamReader.json[0]>", line 1, in <module>
json_sdf = spark.readStream.json(os.path.join(tempfile.mkdtemp(), 'data'), schema = sdf_schema)
File "/Users/tdas/Projects/Spark/spark/python/pyspark/sql/readwriter.py", line 963, in json
return self._df(self._jreader.json(path))
File "/Users/tdas/Projects/Spark/spark/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__
answer, self.gateway_client, self.target_id, self.name)
File "/Users/tdas/Projects/Spark/spark/python/pyspark/sql/utils.py", line 63, in deco
return f(*a, **kw)
File "/Users/tdas/Projects/Spark/spark/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 316, in get_return_value
format(target_id, ".", name, value))
Py4JError: An error occurred while calling o121.json. Trace:
py4j.Py4JException: Method json([class java.util.ArrayList]) does not exist
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318)
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:326)
at py4j.Gateway.invoke(Gateway.java:272)
at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:128)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at py4j.GatewayConnection.run(GatewayConnection.java:211)
at java.lang.Thread.run(Thread.java:744)
```
- Reduced code duplication between DataStreamReader and DataFrameWriter
- Added missing Python doctests
## How was this patch tested?
New tests
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #13703 from tdas/SPARK-15981.
2016-06-16 16:17:41 -04:00
|
|
|
"""
|
2016-06-28 16:43:59 -04:00
|
|
|
Set named options (filter out those the value is None)
|
[SPARK-15981][SQL][STREAMING] Fixed bug and added tests in DataStreamReader Python API
## What changes were proposed in this pull request?
- Fixed bug in Python API of DataStreamReader. Because a single path was being converted to a array before calling Java DataStreamReader method (which takes a string only), it gave the following error.
```
File "/Users/tdas/Projects/Spark/spark/python/pyspark/sql/readwriter.py", line 947, in pyspark.sql.readwriter.DataStreamReader.json
Failed example:
json_sdf = spark.readStream.json(os.path.join(tempfile.mkdtemp(), 'data'), schema = sdf_schema)
Exception raised:
Traceback (most recent call last):
File "/System/Library/Frameworks/Python.framework/Versions/2.6/lib/python2.6/doctest.py", line 1253, in __run
compileflags, 1) in test.globs
File "<doctest pyspark.sql.readwriter.DataStreamReader.json[0]>", line 1, in <module>
json_sdf = spark.readStream.json(os.path.join(tempfile.mkdtemp(), 'data'), schema = sdf_schema)
File "/Users/tdas/Projects/Spark/spark/python/pyspark/sql/readwriter.py", line 963, in json
return self._df(self._jreader.json(path))
File "/Users/tdas/Projects/Spark/spark/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__
answer, self.gateway_client, self.target_id, self.name)
File "/Users/tdas/Projects/Spark/spark/python/pyspark/sql/utils.py", line 63, in deco
return f(*a, **kw)
File "/Users/tdas/Projects/Spark/spark/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 316, in get_return_value
format(target_id, ".", name, value))
Py4JError: An error occurred while calling o121.json. Trace:
py4j.Py4JException: Method json([class java.util.ArrayList]) does not exist
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318)
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:326)
at py4j.Gateway.invoke(Gateway.java:272)
at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:128)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at py4j.GatewayConnection.run(GatewayConnection.java:211)
at java.lang.Thread.run(Thread.java:744)
```
- Reduced code duplication between DataStreamReader and DataFrameWriter
- Added missing Python doctests
## How was this patch tested?
New tests
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #13703 from tdas/SPARK-15981.
2016-06-16 16:17:41 -04:00
|
|
|
"""
|
|
|
|
if schema is not None:
|
|
|
|
self.schema(schema)
|
2016-06-28 16:43:59 -04:00
|
|
|
for k, v in options.items():
|
|
|
|
if v is not None:
|
|
|
|
self.option(k, v)
|
|
|
|
|
|
|
|
|
|
|
|
class DataFrameReader(OptionUtils):
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
|
|
|
Interface used to load a :class:`DataFrame` from external storage systems
|
2020-02-13 21:00:35 -05:00
|
|
|
(e.g. file systems, key-value stores, etc). Use :attr:`SparkSession.read`
|
2015-05-19 17:23:28 -04:00
|
|
|
to access this.
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
.. versionadded:: 1.4
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
|
|
|
|
2016-05-11 14:24:16 -04:00
|
|
|
def __init__(self, spark):
|
|
|
|
self._jreader = spark._ssql_ctx.read()
|
|
|
|
self._spark = spark
|
2015-05-19 17:23:28 -04:00
|
|
|
|
|
|
|
def _df(self, jdf):
|
|
|
|
from pyspark.sql.dataframe import DataFrame
|
2016-05-11 14:24:16 -04:00
|
|
|
return DataFrame(jdf, self._spark)
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2015-06-02 11:37:18 -04:00
|
|
|
def format(self, source):
|
2015-06-03 03:23:34 -04:00
|
|
|
"""Specifies the input data source format.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
source : str
|
|
|
|
string, name of the data source, e.g. 'json', 'parquet'.
|
2015-06-03 03:23:34 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-05-11 14:24:16 -04:00
|
|
|
>>> df = spark.read.format('json').load('python/test_support/sql/people.json')
|
2015-06-03 03:23:34 -04:00
|
|
|
>>> df.dtypes
|
|
|
|
[('age', 'bigint'), ('name', 'string')]
|
|
|
|
|
2015-06-02 11:37:18 -04:00
|
|
|
"""
|
|
|
|
self._jreader = self._jreader.format(source)
|
|
|
|
return self
|
|
|
|
|
|
|
|
def schema(self, schema):
|
2015-06-03 03:23:34 -04:00
|
|
|
"""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
|
2015-06-02 11:37:18 -04:00
|
|
|
inference step, and thus speed up data loading.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.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``).
|
2017-06-23 23:39:41 -04:00
|
|
|
|
|
|
|
>>> s = spark.read.schema("col0 INT, col1 DOUBLE")
|
2015-06-02 11:37:18 -04:00
|
|
|
"""
|
2016-08-25 02:36:04 -04:00
|
|
|
from pyspark.sql import SparkSession
|
|
|
|
spark = SparkSession.builder.getOrCreate()
|
2017-05-11 14:06:29 -04:00
|
|
|
if isinstance(schema, StructType):
|
|
|
|
jschema = spark._jsparkSession.parseDataType(schema.json())
|
|
|
|
self._jreader = self._jreader.schema(jschema)
|
2020-07-13 22:22:44 -04:00
|
|
|
elif isinstance(schema, str):
|
2017-05-11 14:06:29 -04:00
|
|
|
self._jreader = self._jreader.schema(schema)
|
|
|
|
else:
|
|
|
|
raise TypeError("schema should be StructType or string")
|
2015-06-02 11:37:18 -04:00
|
|
|
return self
|
|
|
|
|
2015-06-29 03:13:39 -04:00
|
|
|
@since(1.5)
|
|
|
|
def option(self, key, value):
|
|
|
|
"""Adds an input option for the underlying data source.
|
|
|
|
"""
|
2015-08-05 20:28:23 -04:00
|
|
|
self._jreader = self._jreader.option(key, to_str(value))
|
2015-06-29 03:13:39 -04:00
|
|
|
return self
|
|
|
|
|
2015-06-02 11:37:18 -04:00
|
|
|
@since(1.4)
|
|
|
|
def options(self, **options):
|
2015-06-03 03:23:34 -04:00
|
|
|
"""Adds input options for the underlying data source.
|
2015-06-02 11:37:18 -04:00
|
|
|
"""
|
|
|
|
for k in options:
|
2015-08-05 20:28:23 -04:00
|
|
|
self._jreader = self._jreader.option(k, to_str(options[k]))
|
2015-06-02 11:37:18 -04:00
|
|
|
return self
|
|
|
|
|
2015-05-19 17:23:28 -04:00
|
|
|
def load(self, path=None, format=None, schema=None, **options):
|
2020-02-18 02:46:45 -05:00
|
|
|
"""Loads data from a data source and returns it as a :class:`DataFrame`.
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str or list, optional
|
|
|
|
optional string or a list of 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
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-03-16 12:36:30 -04:00
|
|
|
>>> df = spark.read.format("parquet").load('python/test_support/sql/parquet_partitioned',
|
|
|
|
... opt1=True, opt2=1, opt3='str')
|
2015-06-03 03:23:34 -04:00
|
|
|
>>> df.dtypes
|
|
|
|
[('name', 'string'), ('year', 'int'), ('month', 'int'), ('day', 'int')]
|
2015-11-24 21:16:07 -05:00
|
|
|
|
2016-05-11 14:24:16 -04:00
|
|
|
>>> df = spark.read.format('json').load(['python/test_support/sql/people.json',
|
2015-10-17 17:56:24 -04:00
|
|
|
... 'python/test_support/sql/people1.json'])
|
|
|
|
>>> df.dtypes
|
|
|
|
[('age', 'bigint'), ('aka', 'string'), ('name', 'string')]
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
|
|
|
if format is not None:
|
2015-06-02 11:37:18 -04:00
|
|
|
self.format(format)
|
2015-05-19 17:23:28 -04:00
|
|
|
if schema is not None:
|
2015-06-02 11:37:18 -04:00
|
|
|
self.schema(schema)
|
|
|
|
self.options(**options)
|
2020-07-13 22:22:44 -04:00
|
|
|
if isinstance(path, str):
|
2016-06-29 14:30:49 -04:00
|
|
|
return self._df(self._jreader.load(path))
|
|
|
|
elif path is not None:
|
2016-01-04 21:02:38 -05:00
|
|
|
if type(path) != list:
|
|
|
|
path = [path]
|
2016-05-11 14:24:16 -04:00
|
|
|
return self._df(self._jreader.load(self._spark._sc._jvm.PythonUtils.toSeq(path)))
|
2015-05-19 17:23:28 -04:00
|
|
|
else:
|
2015-06-02 11:37:18 -04:00
|
|
|
return self._df(self._jreader.load())
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2016-05-02 20:50:40 -04:00
|
|
|
def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
|
|
|
|
allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None,
|
|
|
|
allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None,
|
[SPARK-18937][SQL] Timezone support in CSV/JSON parsing
## What changes were proposed in this pull request?
This is a follow-up pr of #16308.
This pr enables timezone support in CSV/JSON parsing.
We should introduce `timeZone` option for CSV/JSON datasources (the default value of the option is session local timezone).
The datasources should use the `timeZone` option to format/parse to write/read timestamp values.
Notice that while reading, if the timestampFormat has the timezone info, the timezone will not be used because we should respect the timezone in the values.
For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT`, the values written with the default timezone option, which is `"GMT"` because session local timezone is `"GMT"` here, are:
```scala
scala> spark.conf.set("spark.sql.session.timeZone", "GMT")
scala> val df = Seq(new java.sql.Timestamp(1451606400000L)).toDF("ts")
df: org.apache.spark.sql.DataFrame = [ts: timestamp]
scala> df.show()
+-------------------+
|ts |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
scala> df.write.json("/path/to/gmtjson")
```
```sh
$ cat /path/to/gmtjson/part-*
{"ts":"2016-01-01T00:00:00.000Z"}
```
whereas setting the option to `"PST"`, they are:
```scala
scala> df.write.option("timeZone", "PST").json("/path/to/pstjson")
```
```sh
$ cat /path/to/pstjson/part-*
{"ts":"2015-12-31T16:00:00.000-08:00"}
```
We can properly read these files even if the timezone option is wrong because the timestamp values have timezone info:
```scala
scala> val schema = new StructType().add("ts", TimestampType)
schema: org.apache.spark.sql.types.StructType = StructType(StructField(ts,TimestampType,true))
scala> spark.read.schema(schema).json("/path/to/gmtjson").show()
+-------------------+
|ts |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
scala> spark.read.schema(schema).option("timeZone", "PST").json("/path/to/gmtjson").show()
+-------------------+
|ts |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
```
And even if `timezoneFormat` doesn't contain timezone info, we can properly read the values with setting correct timezone option:
```scala
scala> df.write.option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").option("timeZone", "JST").json("/path/to/jstjson")
```
```sh
$ cat /path/to/jstjson/part-*
{"ts":"2016-01-01T09:00:00"}
```
```scala
// wrong result
scala> spark.read.schema(schema).option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").json("/path/to/jstjson").show()
+-------------------+
|ts |
+-------------------+
|2016-01-01 09:00:00|
+-------------------+
// correct result
scala> spark.read.schema(schema).option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").option("timeZone", "JST").json("/path/to/jstjson").show()
+-------------------+
|ts |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
```
This pr also makes `JsonToStruct` and `StructToJson` `TimeZoneAwareExpression` to be able to evaluate values with timezone option.
## How was this patch tested?
Existing tests and added some tests.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes #16750 from ueshin/issues/SPARK-18937.
2017-02-15 16:26:34 -05:00
|
|
|
mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None,
|
2018-04-28 23:25:31 -04:00
|
|
|
multiLine=None, allowUnquotedControlChars=None, lineSep=None, samplingRatio=None,
|
2019-12-22 19:57:42 -05:00
|
|
|
dropFieldIfAllNull=None, encoding=None, locale=None, pathGlobFilter=None,
|
[SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?
Two new options, _modifiiedBefore_ and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format. _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.` In order to filter file results, a new PathFilter class was derived for this purpose. General house-keeping around classes extending PathFilter was performed for neatness. It became apparent support was needed to handle multiple potential path filters. Logic was introduced for this purpose and the associated tests written.
### Why are the changes needed?
When loading files from a data source, there can often times be thousands of file within a respective file path. In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point. This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered. This saves a ton of time automatically and reduces significant complexity managing this in code.
### Does this PR introduce _any_ user-facing change?
This PR introduces an option that can be used with batch-based Spark file data sources. A documentation update was made to reflect an example and usage of the new data source option.
**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`
_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`
_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`
### How was this patch tested?
A handful of unit tests were added to support the positive, negative, and edge case code paths.
It's also live in a handful of our Databricks dev environments. (quoted from cchighman)
Closes #30411 from HeartSaVioR/SPARK-31962.
Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-22 18:30:41 -05:00
|
|
|
recursiveFileLookup=None, allowNonNumericNumbers=None,
|
|
|
|
modifiedBefore=None, modifiedAfter=None):
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
2017-03-09 14:44:34 -05:00
|
|
|
Loads JSON files and returns the results as a :class:`DataFrame`.
|
2017-02-16 23:51:19 -05:00
|
|
|
|
2017-04-12 04:16:39 -04:00
|
|
|
`JSON Lines <http://jsonlines.org/>`_ (newline-delimited JSON) is supported by default.
|
2017-06-15 01:18:19 -04:00
|
|
|
For JSON (one record per file), set the ``multiLine`` parameter to ``true``.
|
2015-05-19 17:23:28 -04:00
|
|
|
|
|
|
|
If the ``schema`` parameter is not specified, this function goes
|
|
|
|
through the input once to determine the input schema.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str, list or :class:`RDD`
|
|
|
|
string represents path to the JSON dataset, or a list of paths,
|
|
|
|
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``).
|
[SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?
Two new options, _modifiiedBefore_ and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format. _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.` In order to filter file results, a new PathFilter class was derived for this purpose. General house-keeping around classes extending PathFilter was performed for neatness. It became apparent support was needed to handle multiple potential path filters. Logic was introduced for this purpose and the associated tests written.
### Why are the changes needed?
When loading files from a data source, there can often times be thousands of file within a respective file path. In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point. This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered. This saves a ton of time automatically and reduces significant complexity managing this in code.
### Does this PR introduce _any_ user-facing change?
This PR introduces an option that can be used with batch-based Spark file data sources. A documentation update was made to reflect an example and usage of the new data source option.
**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`
_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`
_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`
### How was this patch tested?
A handful of unit tests were added to support the positive, negative, and edge case code paths.
It's also live in a handful of our Databricks dev environments. (quoted from cchighman)
Closes #30411 from HeartSaVioR/SPARK-31962.
Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-22 18:30:41 -05:00
|
|
|
|
2021-05-21 05:05:13 -04:00
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
2021-05-31 21:58:49 -04:00
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
|
2021-05-21 05:05:13 -04:00
|
|
|
in the version you use.
|
2019-12-03 20:10:30 -05:00
|
|
|
|
2021-05-31 21:58:49 -04:00
|
|
|
.. # noqa
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-05-11 14:24:16 -04:00
|
|
|
>>> df1 = spark.read.json('python/test_support/sql/people.json')
|
2015-08-27 01:19:11 -04:00
|
|
|
>>> df1.dtypes
|
|
|
|
[('age', 'bigint'), ('name', 'string')]
|
|
|
|
>>> rdd = sc.textFile('python/test_support/sql/people.json')
|
2016-05-11 14:24:16 -04:00
|
|
|
>>> df2 = spark.read.json(rdd)
|
2015-08-27 01:19:11 -04:00
|
|
|
>>> df2.dtypes
|
2015-06-03 03:23:34 -04:00
|
|
|
[('age', 'bigint'), ('name', 'string')]
|
|
|
|
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
2016-06-28 16:43:59 -04:00
|
|
|
self._set_opts(
|
2016-06-21 13:47:51 -04:00
|
|
|
schema=schema, primitivesAsString=primitivesAsString, prefersDecimal=prefersDecimal,
|
|
|
|
allowComments=allowComments, allowUnquotedFieldNames=allowUnquotedFieldNames,
|
|
|
|
allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero,
|
|
|
|
allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter,
|
2016-08-24 16:16:20 -04:00
|
|
|
mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat,
|
2017-08-25 13:18:03 -04:00
|
|
|
timestampFormat=timestampFormat, multiLine=multiLine,
|
2018-04-25 21:14:24 -04:00
|
|
|
allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep,
|
2018-11-08 20:45:06 -05:00
|
|
|
samplingRatio=samplingRatio, dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding,
|
2020-07-28 23:14:13 -04:00
|
|
|
locale=locale, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup,
|
[SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?
Two new options, _modifiiedBefore_ and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format. _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.` In order to filter file results, a new PathFilter class was derived for this purpose. General house-keeping around classes extending PathFilter was performed for neatness. It became apparent support was needed to handle multiple potential path filters. Logic was introduced for this purpose and the associated tests written.
### Why are the changes needed?
When loading files from a data source, there can often times be thousands of file within a respective file path. In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point. This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered. This saves a ton of time automatically and reduces significant complexity managing this in code.
### Does this PR introduce _any_ user-facing change?
This PR introduces an option that can be used with batch-based Spark file data sources. A documentation update was made to reflect an example and usage of the new data source option.
**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`
_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`
_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`
### How was this patch tested?
A handful of unit tests were added to support the positive, negative, and edge case code paths.
It's also live in a handful of our Databricks dev environments. (quoted from cchighman)
Closes #30411 from HeartSaVioR/SPARK-31962.
Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-22 18:30:41 -05:00
|
|
|
modifiedBefore=modifiedBefore, modifiedAfter=modifiedAfter,
|
2020-07-28 23:14:13 -04:00
|
|
|
allowNonNumericNumbers=allowNonNumericNumbers)
|
2020-07-13 22:22:44 -04:00
|
|
|
if isinstance(path, str):
|
2016-05-11 01:21:17 -04:00
|
|
|
path = [path]
|
|
|
|
if type(path) == list:
|
2016-05-11 14:24:16 -04:00
|
|
|
return self._df(self._jreader.json(self._spark._sc._jvm.PythonUtils.toSeq(path)))
|
2015-08-27 01:19:11 -04:00
|
|
|
elif isinstance(path, RDD):
|
2016-01-04 21:02:38 -05:00
|
|
|
def func(iterator):
|
|
|
|
for x in iterator:
|
2020-07-13 22:22:44 -04:00
|
|
|
if not isinstance(x, str):
|
|
|
|
x = str(x)
|
|
|
|
if isinstance(x, str):
|
2016-01-04 21:02:38 -05:00
|
|
|
x = x.encode("utf-8")
|
|
|
|
yield x
|
|
|
|
keyed = path.mapPartitions(func)
|
|
|
|
keyed._bypass_serializer = True
|
2016-05-11 14:24:16 -04:00
|
|
|
jrdd = keyed._jrdd.map(self._spark._jvm.BytesToString())
|
2016-01-04 21:02:38 -05:00
|
|
|
return self._df(self._jreader.json(jrdd))
|
2015-08-27 01:19:11 -04:00
|
|
|
else:
|
2017-03-09 14:44:34 -05:00
|
|
|
raise TypeError("path can be only string, list or RDD")
|
2015-05-19 17:23:28 -04:00
|
|
|
|
|
|
|
def table(self, tableName):
|
|
|
|
"""Returns the specified table as a :class:`DataFrame`.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
2015-06-03 03:23:34 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
tableName : str
|
|
|
|
string, name of the table.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-11 14:24:16 -04:00
|
|
|
>>> df = spark.read.parquet('python/test_support/sql/parquet_partitioned')
|
2016-05-17 21:01:59 -04:00
|
|
|
>>> df.createOrReplaceTempView('tmpTable')
|
2016-05-11 14:24:16 -04:00
|
|
|
>>> spark.read.table('tmpTable').dtypes
|
2015-06-03 03:23:34 -04:00
|
|
|
[('name', 'string'), ('year', 'int'), ('month', 'int'), ('day', 'int')]
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
|
|
|
return self._df(self._jreader.table(tableName))
|
|
|
|
|
2019-12-03 20:10:30 -05:00
|
|
|
def parquet(self, *paths, **options):
|
[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 Parquet files, returning the result as a :class:`DataFrame`.
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
paths : str
|
|
|
|
|
|
|
|
Other Parameters
|
|
|
|
----------------
|
2021-05-21 05:05:49 -04:00
|
|
|
**options
|
|
|
|
For the extra options, refer to
|
2021-05-31 21:58:49 -04:00
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
|
2021-05-21 05:05:49 -04:00
|
|
|
in the version you use.
|
[SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?
Two new options, _modifiiedBefore_ and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format. _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.` In order to filter file results, a new PathFilter class was derived for this purpose. General house-keeping around classes extending PathFilter was performed for neatness. It became apparent support was needed to handle multiple potential path filters. Logic was introduced for this purpose and the associated tests written.
### Why are the changes needed?
When loading files from a data source, there can often times be thousands of file within a respective file path. In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point. This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered. This saves a ton of time automatically and reduces significant complexity managing this in code.
### Does this PR introduce _any_ user-facing change?
This PR introduces an option that can be used with batch-based Spark file data sources. A documentation update was made to reflect an example and usage of the new data source option.
**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`
_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`
_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`
### How was this patch tested?
A handful of unit tests were added to support the positive, negative, and edge case code paths.
It's also live in a handful of our Databricks dev environments. (quoted from cchighman)
Closes #30411 from HeartSaVioR/SPARK-31962.
Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-22 18:30:41 -05:00
|
|
|
|
2021-05-31 21:58:49 -04:00
|
|
|
.. # noqa
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-05-11 14:24:16 -04:00
|
|
|
>>> df = spark.read.parquet('python/test_support/sql/parquet_partitioned')
|
2015-06-03 03:23:34 -04:00
|
|
|
>>> df.dtypes
|
|
|
|
[('name', 'string'), ('year', 'int'), ('month', 'int'), ('day', 'int')]
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
[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
|
|
|
mergeSchema = options.get('mergeSchema', None)
|
2019-12-22 19:57:42 -05:00
|
|
|
pathGlobFilter = options.get('pathGlobFilter', None)
|
[SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?
Two new options, _modifiiedBefore_ and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format. _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.` In order to filter file results, a new PathFilter class was derived for this purpose. General house-keeping around classes extending PathFilter was performed for neatness. It became apparent support was needed to handle multiple potential path filters. Logic was introduced for this purpose and the associated tests written.
### Why are the changes needed?
When loading files from a data source, there can often times be thousands of file within a respective file path. In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point. This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered. This saves a ton of time automatically and reduces significant complexity managing this in code.
### Does this PR introduce _any_ user-facing change?
This PR introduces an option that can be used with batch-based Spark file data sources. A documentation update was made to reflect an example and usage of the new data source option.
**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`
_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`
_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`
### How was this patch tested?
A handful of unit tests were added to support the positive, negative, and edge case code paths.
It's also live in a handful of our Databricks dev environments. (quoted from cchighman)
Closes #30411 from HeartSaVioR/SPARK-31962.
Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-22 18:30:41 -05:00
|
|
|
modifiedBefore = options.get('modifiedBefore', None)
|
|
|
|
modifiedAfter = options.get('modifiedAfter', None)
|
2019-12-03 20:10:30 -05:00
|
|
|
recursiveFileLookup = options.get('recursiveFileLookup', None)
|
[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 = options.get('datetimeRebaseMode', None)
|
|
|
|
int96RebaseMode = options.get('int96RebaseMode', None)
|
2019-12-22 19:57:42 -05:00
|
|
|
self._set_opts(mergeSchema=mergeSchema, pathGlobFilter=pathGlobFilter,
|
[SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?
Two new options, _modifiiedBefore_ and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format. _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.` In order to filter file results, a new PathFilter class was derived for this purpose. General house-keeping around classes extending PathFilter was performed for neatness. It became apparent support was needed to handle multiple potential path filters. Logic was introduced for this purpose and the associated tests written.
### Why are the changes needed?
When loading files from a data source, there can often times be thousands of file within a respective file path. In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point. This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered. This saves a ton of time automatically and reduces significant complexity managing this in code.
### Does this PR introduce _any_ user-facing change?
This PR introduces an option that can be used with batch-based Spark file data sources. A documentation update was made to reflect an example and usage of the new data source option.
**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`
_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`
_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`
### How was this patch tested?
A handful of unit tests were added to support the positive, negative, and edge case code paths.
It's also live in a handful of our Databricks dev environments. (quoted from cchighman)
Closes #30411 from HeartSaVioR/SPARK-31962.
Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-22 18:30:41 -05:00
|
|
|
recursiveFileLookup=recursiveFileLookup, modifiedBefore=modifiedBefore,
|
[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
|
|
|
modifiedAfter=modifiedAfter, datetimeRebaseMode=datetimeRebaseMode,
|
|
|
|
int96RebaseMode=int96RebaseMode)
|
[SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?
Two new options, _modifiiedBefore_ and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format. _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.` In order to filter file results, a new PathFilter class was derived for this purpose. General house-keeping around classes extending PathFilter was performed for neatness. It became apparent support was needed to handle multiple potential path filters. Logic was introduced for this purpose and the associated tests written.
### Why are the changes needed?
When loading files from a data source, there can often times be thousands of file within a respective file path. In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point. This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered. This saves a ton of time automatically and reduces significant complexity managing this in code.
### Does this PR introduce _any_ user-facing change?
This PR introduces an option that can be used with batch-based Spark file data sources. A documentation update was made to reflect an example and usage of the new data source option.
**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`
_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`
_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`
### How was this patch tested?
A handful of unit tests were added to support the positive, negative, and edge case code paths.
It's also live in a handful of our Databricks dev environments. (quoted from cchighman)
Closes #30411 from HeartSaVioR/SPARK-31962.
Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-22 18:30:41 -05:00
|
|
|
|
2016-05-11 14:24:16 -04:00
|
|
|
return self._df(self._jreader.parquet(_to_seq(self._spark._sc, paths)))
|
2015-07-21 03:08:44 -04:00
|
|
|
|
2019-12-22 19:57:42 -05:00
|
|
|
def text(self, paths, wholetext=False, lineSep=None, pathGlobFilter=None,
|
[SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?
Two new options, _modifiiedBefore_ and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format. _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.` In order to filter file results, a new PathFilter class was derived for this purpose. General house-keeping around classes extending PathFilter was performed for neatness. It became apparent support was needed to handle multiple potential path filters. Logic was introduced for this purpose and the associated tests written.
### Why are the changes needed?
When loading files from a data source, there can often times be thousands of file within a respective file path. In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point. This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered. This saves a ton of time automatically and reduces significant complexity managing this in code.
### Does this PR introduce _any_ user-facing change?
This PR introduces an option that can be used with batch-based Spark file data sources. A documentation update was made to reflect an example and usage of the new data source option.
**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`
_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`
_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`
### How was this patch tested?
A handful of unit tests were added to support the positive, negative, and edge case code paths.
It's also live in a handful of our Databricks dev environments. (quoted from cchighman)
Closes #30411 from HeartSaVioR/SPARK-31962.
Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-22 18:30:41 -05:00
|
|
|
recursiveFileLookup=None, modifiedBefore=None,
|
|
|
|
modifiedAfter=None):
|
2016-06-13 00:36:41 -04:00
|
|
|
"""
|
|
|
|
Loads text files and returns a :class:`DataFrame` whose schema starts with a
|
|
|
|
string column named "value", and followed by partitioned columns if there
|
|
|
|
are any.
|
2019-03-04 18:03:39 -05:00
|
|
|
The text files must be encoded as UTF-8.
|
2015-10-28 17:28:38 -04:00
|
|
|
|
2018-03-21 12:46:47 -04:00
|
|
|
By default, each line in the text file is a new row in the resulting DataFrame.
|
2015-10-28 17:28:38 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.6.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
paths : str or list
|
|
|
|
string, or list of strings, for input path(s).
|
|
|
|
|
2021-05-26 04:12:49 -04:00
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
2021-05-31 21:58:49 -04:00
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
|
2021-05-26 04:12:49 -04:00
|
|
|
in the version you use.
|
[SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?
Two new options, _modifiiedBefore_ and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format. _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.` In order to filter file results, a new PathFilter class was derived for this purpose. General house-keeping around classes extending PathFilter was performed for neatness. It became apparent support was needed to handle multiple potential path filters. Logic was introduced for this purpose and the associated tests written.
### Why are the changes needed?
When loading files from a data source, there can often times be thousands of file within a respective file path. In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point. This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered. This saves a ton of time automatically and reduces significant complexity managing this in code.
### Does this PR introduce _any_ user-facing change?
This PR introduces an option that can be used with batch-based Spark file data sources. A documentation update was made to reflect an example and usage of the new data source option.
**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`
_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`
_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`
### How was this patch tested?
A handful of unit tests were added to support the positive, negative, and edge case code paths.
It's also live in a handful of our Databricks dev environments. (quoted from cchighman)
Closes #30411 from HeartSaVioR/SPARK-31962.
Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-22 18:30:41 -05:00
|
|
|
|
2021-05-31 21:58:49 -04:00
|
|
|
.. # noqa
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-05-11 14:24:16 -04:00
|
|
|
>>> df = spark.read.text('python/test_support/sql/text-test.txt')
|
2015-10-28 17:28:38 -04:00
|
|
|
>>> df.collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(value='hello'), Row(value='this')]
|
2017-12-14 14:19:34 -05:00
|
|
|
>>> df = spark.read.text('python/test_support/sql/text-test.txt', wholetext=True)
|
|
|
|
>>> df.collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(value='hello\\nthis')]
|
2015-10-28 17:28:38 -04:00
|
|
|
"""
|
2019-12-03 20:10:30 -05:00
|
|
|
self._set_opts(
|
2019-12-22 19:57:42 -05:00
|
|
|
wholetext=wholetext, lineSep=lineSep, pathGlobFilter=pathGlobFilter,
|
[SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?
Two new options, _modifiiedBefore_ and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format. _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.` In order to filter file results, a new PathFilter class was derived for this purpose. General house-keeping around classes extending PathFilter was performed for neatness. It became apparent support was needed to handle multiple potential path filters. Logic was introduced for this purpose and the associated tests written.
### Why are the changes needed?
When loading files from a data source, there can often times be thousands of file within a respective file path. In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point. This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered. This saves a ton of time automatically and reduces significant complexity managing this in code.
### Does this PR introduce _any_ user-facing change?
This PR introduces an option that can be used with batch-based Spark file data sources. A documentation update was made to reflect an example and usage of the new data source option.
**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`
_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`
_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`
### How was this patch tested?
A handful of unit tests were added to support the positive, negative, and edge case code paths.
It's also live in a handful of our Databricks dev environments. (quoted from cchighman)
Closes #30411 from HeartSaVioR/SPARK-31962.
Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-22 18:30:41 -05:00
|
|
|
recursiveFileLookup=recursiveFileLookup, modifiedBefore=modifiedBefore,
|
|
|
|
modifiedAfter=modifiedAfter)
|
|
|
|
|
2020-07-13 22:22:44 -04:00
|
|
|
if isinstance(paths, str):
|
2016-10-07 03:27:55 -04:00
|
|
|
paths = [paths]
|
|
|
|
return self._df(self._jreader.text(self._spark._sc._jvm.PythonUtils.toSeq(paths)))
|
2015-10-28 17:28:38 -04:00
|
|
|
|
2016-06-06 02:40:13 -04:00
|
|
|
def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None,
|
2016-06-12 02:20:40 -04:00
|
|
|
comment=None, header=None, inferSchema=None, ignoreLeadingWhiteSpace=None,
|
|
|
|
ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None,
|
2016-08-24 16:16:20 -04:00
|
|
|
negativeInf=None, dateFormat=None, timestampFormat=None, maxColumns=None,
|
2017-03-14 16:57:23 -04:00
|
|
|
maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None,
|
2018-04-29 21:45:22 -04:00
|
|
|
columnNameOfCorruptRecord=None, multiLine=None, charToEscapeQuoteEscaping=None,
|
2019-12-03 20:10:30 -05:00
|
|
|
samplingRatio=None, enforceSchema=None, emptyValue=None, locale=None, lineSep=None,
|
2020-11-27 01:47:39 -05:00
|
|
|
pathGlobFilter=None, recursiveFileLookup=None, modifiedBefore=None, modifiedAfter=None,
|
|
|
|
unescapedQuoteHandling=None):
|
2018-09-12 23:19:43 -04:00
|
|
|
r"""Loads a CSV file and returns the result as a :class:`DataFrame`.
|
2016-02-29 12:44:29 -05:00
|
|
|
|
2016-06-12 02:20:40 -04:00
|
|
|
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``.
|
2016-02-29 12:44:29 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str or list
|
|
|
|
string, or list of strings, for input path(s),
|
|
|
|
or RDD of Strings storing CSV rows.
|
|
|
|
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``).
|
2021-05-31 21:58:49 -04:00
|
|
|
|
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
|
|
|
|
in the version you use.
|
|
|
|
|
|
|
|
.. # noqa
|
[SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?
Two new options, _modifiiedBefore_ and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format. _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.` In order to filter file results, a new PathFilter class was derived for this purpose. General house-keeping around classes extending PathFilter was performed for neatness. It became apparent support was needed to handle multiple potential path filters. Logic was introduced for this purpose and the associated tests written.
### Why are the changes needed?
When loading files from a data source, there can often times be thousands of file within a respective file path. In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point. This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered. This saves a ton of time automatically and reduces significant complexity managing this in code.
### Does this PR introduce _any_ user-facing change?
This PR introduces an option that can be used with batch-based Spark file data sources. A documentation update was made to reflect an example and usage of the new data source option.
**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`
_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`
_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`
### How was this patch tested?
A handful of unit tests were added to support the positive, negative, and edge case code paths.
It's also live in a handful of our Databricks dev environments. (quoted from cchighman)
Closes #30411 from HeartSaVioR/SPARK-31962.
Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-22 18:30:41 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-05-11 14:24:16 -04:00
|
|
|
>>> df = spark.read.csv('python/test_support/sql/ages.csv')
|
2016-02-29 12:44:29 -05:00
|
|
|
>>> df.dtypes
|
[SPARK-15264][SPARK-15274][SQL] CSV Reader Error on Blank Column Names
## What changes were proposed in this pull request?
When a CSV begins with:
- `,,`
OR
- `"","",`
meaning that the first column names are either empty or blank strings and `header` is specified to be `true`, then the column name is replaced with `C` + the index number of that given column. For example, if you were to read in the CSV:
```
"","second column"
"hello", "there"
```
Then column names would become `"C0", "second column"`.
This behavior aligns with what currently happens when `header` is specified to be `false` in recent versions of Spark.
### Current Behavior in Spark <=1.6
In Spark <=1.6, a CSV with a blank column name becomes a blank string, `""`, meaning that this column cannot be accessed. However the CSV reads in without issue.
### Current Behavior in Spark 2.0
Spark throws a NullPointerError and will not read in the file.
#### Reproduction in 2.0
https://databricks-prod-cloudfront.cloud.databricks.com/public/4027ec902e239c93eaaa8714f173bcfc/346304/2828750690305044/484361/latest.html
## How was this patch tested?
A new test was added to `CSVSuite` to account for this issue. We then have asserts that test for being able to select both the empty column names as well as the regular column names.
Author: Bill Chambers <bill@databricks.com>
Author: Bill Chambers <wchambers@ischool.berkeley.edu>
Closes #13041 from anabranch/master.
2016-05-11 20:42:13 -04:00
|
|
|
[('_c0', 'string'), ('_c1', 'string')]
|
2017-09-26 22:19:45 -04:00
|
|
|
>>> rdd = sc.textFile('python/test_support/sql/ages.csv')
|
|
|
|
>>> df2 = spark.read.csv(rdd)
|
|
|
|
>>> df2.dtypes
|
|
|
|
[('_c0', 'string'), ('_c1', 'string')]
|
2016-02-29 12:44:29 -05:00
|
|
|
"""
|
2016-06-28 16:43:59 -04:00
|
|
|
self._set_opts(
|
2016-06-21 13:47:51 -04:00
|
|
|
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,
|
2016-08-24 16:16:20 -04:00
|
|
|
dateFormat=dateFormat, timestampFormat=timestampFormat, maxColumns=maxColumns,
|
|
|
|
maxCharsPerColumn=maxCharsPerColumn,
|
2017-03-14 16:57:23 -04:00
|
|
|
maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode,
|
2017-12-28 18:30:06 -05:00
|
|
|
columnNameOfCorruptRecord=columnNameOfCorruptRecord, multiLine=multiLine,
|
2018-06-04 01:02:21 -04:00
|
|
|
charToEscapeQuoteEscaping=charToEscapeQuoteEscaping, samplingRatio=samplingRatio,
|
2019-12-03 20:10:30 -05:00
|
|
|
enforceSchema=enforceSchema, emptyValue=emptyValue, locale=locale, lineSep=lineSep,
|
[SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?
Two new options, _modifiiedBefore_ and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format. _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.` In order to filter file results, a new PathFilter class was derived for this purpose. General house-keeping around classes extending PathFilter was performed for neatness. It became apparent support was needed to handle multiple potential path filters. Logic was introduced for this purpose and the associated tests written.
### Why are the changes needed?
When loading files from a data source, there can often times be thousands of file within a respective file path. In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point. This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered. This saves a ton of time automatically and reduces significant complexity managing this in code.
### Does this PR introduce _any_ user-facing change?
This PR introduces an option that can be used with batch-based Spark file data sources. A documentation update was made to reflect an example and usage of the new data source option.
**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`
_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`
_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`
### How was this patch tested?
A handful of unit tests were added to support the positive, negative, and edge case code paths.
It's also live in a handful of our Databricks dev environments. (quoted from cchighman)
Closes #30411 from HeartSaVioR/SPARK-31962.
Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-22 18:30:41 -05:00
|
|
|
pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup,
|
2020-11-27 01:47:39 -05:00
|
|
|
modifiedBefore=modifiedBefore, modifiedAfter=modifiedAfter,
|
|
|
|
unescapedQuoteHandling=unescapedQuoteHandling)
|
2020-07-13 22:22:44 -04:00
|
|
|
if isinstance(path, str):
|
2016-05-02 20:50:40 -04:00
|
|
|
path = [path]
|
2017-09-26 22:19:45 -04:00
|
|
|
if type(path) == list:
|
|
|
|
return self._df(self._jreader.csv(self._spark._sc._jvm.PythonUtils.toSeq(path)))
|
|
|
|
elif isinstance(path, RDD):
|
|
|
|
def func(iterator):
|
|
|
|
for x in iterator:
|
2020-07-13 22:22:44 -04:00
|
|
|
if not isinstance(x, str):
|
|
|
|
x = str(x)
|
|
|
|
if isinstance(x, str):
|
2017-09-26 22:19:45 -04:00
|
|
|
x = x.encode("utf-8")
|
|
|
|
yield x
|
|
|
|
keyed = path.mapPartitions(func)
|
|
|
|
keyed._bypass_serializer = True
|
|
|
|
jrdd = keyed._jrdd.map(self._spark._jvm.BytesToString())
|
|
|
|
# see SPARK-22112
|
|
|
|
# There aren't any jvm api for creating a dataframe from rdd storing csv.
|
|
|
|
# We can do it through creating a jvm dataset firstly and using the jvm api
|
|
|
|
# for creating a dataframe from dataset storing csv.
|
|
|
|
jdataset = self._spark._ssql_ctx.createDataset(
|
|
|
|
jrdd.rdd(),
|
|
|
|
self._spark._jvm.Encoders.STRING())
|
|
|
|
return self._df(self._jreader.csv(jdataset))
|
|
|
|
else:
|
|
|
|
raise TypeError("path can be only string, list or RDD")
|
2016-02-29 12:44:29 -05:00
|
|
|
|
[SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?
Two new options, _modifiiedBefore_ and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format. _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.` In order to filter file results, a new PathFilter class was derived for this purpose. General house-keeping around classes extending PathFilter was performed for neatness. It became apparent support was needed to handle multiple potential path filters. Logic was introduced for this purpose and the associated tests written.
### Why are the changes needed?
When loading files from a data source, there can often times be thousands of file within a respective file path. In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point. This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered. This saves a ton of time automatically and reduces significant complexity managing this in code.
### Does this PR introduce _any_ user-facing change?
This PR introduces an option that can be used with batch-based Spark file data sources. A documentation update was made to reflect an example and usage of the new data source option.
**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`
_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`
_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`
### How was this patch tested?
A handful of unit tests were added to support the positive, negative, and edge case code paths.
It's also live in a handful of our Databricks dev environments. (quoted from cchighman)
Closes #30411 from HeartSaVioR/SPARK-31962.
Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-22 18:30:41 -05:00
|
|
|
def orc(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=None,
|
|
|
|
modifiedBefore=None, modifiedAfter=None):
|
2017-03-09 14:44:34 -05:00
|
|
|
"""Loads ORC files, returning the result as a :class:`DataFrame`.
|
2015-07-21 03:08:44 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str or list
|
|
|
|
|
2021-05-21 05:03:57 -04:00
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
2021-05-31 21:58:49 -04:00
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-orc.html#data-source-option>`_
|
2021-05-21 05:03:57 -04:00
|
|
|
in the version you use.
|
[SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?
Two new options, _modifiiedBefore_ and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format. _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.` In order to filter file results, a new PathFilter class was derived for this purpose. General house-keeping around classes extending PathFilter was performed for neatness. It became apparent support was needed to handle multiple potential path filters. Logic was introduced for this purpose and the associated tests written.
### Why are the changes needed?
When loading files from a data source, there can often times be thousands of file within a respective file path. In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point. This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered. This saves a ton of time automatically and reduces significant complexity managing this in code.
### Does this PR introduce _any_ user-facing change?
This PR introduces an option that can be used with batch-based Spark file data sources. A documentation update was made to reflect an example and usage of the new data source option.
**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`
_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`
_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`
### How was this patch tested?
A handful of unit tests were added to support the positive, negative, and edge case code paths.
It's also live in a handful of our Databricks dev environments. (quoted from cchighman)
Closes #30411 from HeartSaVioR/SPARK-31962.
Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-22 18:30:41 -05:00
|
|
|
|
2021-05-31 21:58:49 -04:00
|
|
|
.. # noqa
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-05-11 18:31:16 -04:00
|
|
|
>>> df = spark.read.orc('python/test_support/sql/orc_partitioned')
|
2015-07-21 03:08:44 -04:00
|
|
|
>>> df.dtypes
|
|
|
|
[('a', 'bigint'), ('b', 'int'), ('c', 'int')]
|
|
|
|
"""
|
2019-12-22 19:57:42 -05:00
|
|
|
self._set_opts(mergeSchema=mergeSchema, pathGlobFilter=pathGlobFilter,
|
[SPARK-31962][SQL] Provide modifiedAfter and modifiedBefore options when filtering from a batch-based file data source
### What changes were proposed in this pull request?
Two new options, _modifiiedBefore_ and _modifiedAfter_, is provided expecting a value in 'YYYY-MM-DDTHH:mm:ss' format. _PartioningAwareFileIndex_ considers these options during the process of checking for files, just before considering applied _PathFilters_ such as `pathGlobFilter.` In order to filter file results, a new PathFilter class was derived for this purpose. General house-keeping around classes extending PathFilter was performed for neatness. It became apparent support was needed to handle multiple potential path filters. Logic was introduced for this purpose and the associated tests written.
### Why are the changes needed?
When loading files from a data source, there can often times be thousands of file within a respective file path. In many cases I've seen, we want to start loading from a folder path and ideally be able to begin loading files having modification dates past a certain point. This would mean out of thousands of potential files, only the ones with modification dates greater than the specified timestamp would be considered. This saves a ton of time automatically and reduces significant complexity managing this in code.
### Does this PR introduce _any_ user-facing change?
This PR introduces an option that can be used with batch-based Spark file data sources. A documentation update was made to reflect an example and usage of the new data source option.
**Example Usages**
_Load all CSV files modified after date:_
`spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load()`
_Load all CSV files modified before date:_
`spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load()`
_Load all CSV files modified between two dates:_
`spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00").option("modifiedBefore","2020-06-15T05:00:00").load()
`
### How was this patch tested?
A handful of unit tests were added to support the positive, negative, and edge case code paths.
It's also live in a handful of our Databricks dev environments. (quoted from cchighman)
Closes #30411 from HeartSaVioR/SPARK-31962.
Lead-authored-by: CC Highman <christopher.highman@microsoft.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-11-22 18:30:41 -05:00
|
|
|
modifiedBefore=modifiedBefore, modifiedAfter=modifiedAfter,
|
2019-12-22 19:57:42 -05:00
|
|
|
recursiveFileLookup=recursiveFileLookup)
|
2020-07-13 22:22:44 -04:00
|
|
|
if isinstance(path, str):
|
2017-03-09 14:44:34 -05:00
|
|
|
path = [path]
|
|
|
|
return self._df(self._jreader.orc(_to_seq(self._spark._sc, path)))
|
2015-05-19 17:23:28 -04:00
|
|
|
|
|
|
|
def jdbc(self, url, table, column=None, lowerBound=None, upperBound=None, numPartitions=None,
|
2015-08-14 15:46:05 -04:00
|
|
|
predicates=None, properties=None):
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
2016-05-11 18:31:16 -04:00
|
|
|
Construct a :class:`DataFrame` representing the database table named ``table``
|
|
|
|
accessible via JDBC URL ``url`` and connection ``properties``.
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2016-05-11 18:31:16 -04:00
|
|
|
Partitions of the table will be retrieved in parallel if either ``column`` or
|
2020-11-02 20:00:49 -05:00
|
|
|
``predicates`` is specified. ``lowerBound``, ``upperBound`` and ``numPartitions``
|
2017-01-17 13:37:29 -05:00
|
|
|
is needed when ``column`` is specified.
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2016-05-11 18:31:16 -04:00
|
|
|
If both ``column`` and ``predicates`` are specified, ``column`` will be used.
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
table : str
|
|
|
|
the name of the table
|
|
|
|
column : str, optional
|
2021-06-02 01:21:16 -04:00
|
|
|
alias of ``partitionColumn`` option. Refer to ``partitionColumn`` in
|
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-jdbc.html#data-source-option>`_
|
|
|
|
in the version you use.
|
2020-11-02 20:00:49 -05:00
|
|
|
predicates : list, optional
|
|
|
|
a list of expressions suitable for inclusion in WHERE clauses;
|
|
|
|
each one defines one partition of the :class:`DataFrame`
|
|
|
|
properties : dict, optional
|
|
|
|
a dictionary of JDBC database connection arguments. Normally at
|
|
|
|
least properties "user" and "password" with their corresponding values.
|
|
|
|
For example { 'user' : 'SYSTEM', 'password' : 'mypassword' }
|
|
|
|
|
2021-06-02 01:21:16 -04:00
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-jdbc.html#data-source-option>`_
|
|
|
|
in the version you use.
|
|
|
|
|
|
|
|
.. # noqa
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
Don't create too many partitions in parallel on a large cluster;
|
|
|
|
otherwise Spark might crash your external database systems.
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
:class:`DataFrame`
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
2015-08-14 15:46:05 -04:00
|
|
|
if properties is None:
|
|
|
|
properties = dict()
|
2016-05-11 14:24:16 -04:00
|
|
|
jprop = JavaClass("java.util.Properties", self._spark._sc._gateway._gateway_client)()
|
2015-05-19 17:23:28 -04:00
|
|
|
for k in properties:
|
|
|
|
jprop.setProperty(k, properties[k])
|
|
|
|
if column is not None:
|
2017-01-17 13:37:29 -05:00
|
|
|
assert lowerBound is not None, "lowerBound can not be None when ``column`` is specified"
|
|
|
|
assert upperBound is not None, "upperBound can not be None when ``column`` is specified"
|
|
|
|
assert numPartitions is not None, \
|
|
|
|
"numPartitions can not be None when ``column`` is specified"
|
2015-05-19 17:23:28 -04:00
|
|
|
return self._df(self._jreader.jdbc(url, table, column, int(lowerBound), int(upperBound),
|
|
|
|
int(numPartitions), jprop))
|
|
|
|
if predicates is not None:
|
2016-05-11 14:24:16 -04:00
|
|
|
gateway = self._spark._sc._gateway
|
2015-11-18 11:18:54 -05:00
|
|
|
jpredicates = utils.toJArray(gateway, gateway.jvm.java.lang.String, predicates)
|
|
|
|
return self._df(self._jreader.jdbc(url, table, jpredicates, jprop))
|
2015-05-19 17:23:28 -04:00
|
|
|
return self._df(self._jreader.jdbc(url, table, jprop))
|
|
|
|
|
|
|
|
|
2016-06-28 16:43:59 -04:00
|
|
|
class DataFrameWriter(OptionUtils):
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
2016-06-12 02:20:40 -04:00
|
|
|
Interface used to write a :class:`DataFrame` to external storage systems
|
2020-02-13 21:00:35 -05:00
|
|
|
(e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write`
|
2015-05-19 17:23:28 -04:00
|
|
|
to access this.
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
.. versionadded:: 1.4
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
|
|
|
def __init__(self, df):
|
|
|
|
self._df = df
|
2016-05-11 14:24:16 -04:00
|
|
|
self._spark = df.sql_ctx
|
2015-05-19 17:23:28 -04:00
|
|
|
self._jwrite = df._jdf.write()
|
|
|
|
|
2016-06-15 13:46:02 -04:00
|
|
|
def _sq(self, jsq):
|
|
|
|
from pyspark.sql.streaming import StreamingQuery
|
|
|
|
return StreamingQuery(jsq)
|
2016-04-20 13:32:01 -04:00
|
|
|
|
2015-06-02 11:37:18 -04:00
|
|
|
def mode(self, saveMode):
|
2015-06-03 03:23:34 -04:00
|
|
|
"""Specifies the behavior when data or table already exists.
|
|
|
|
|
|
|
|
Options include:
|
2015-06-02 11:37:18 -04:00
|
|
|
|
|
|
|
* `append`: Append contents of this :class:`DataFrame` to existing data.
|
|
|
|
* `overwrite`: Overwrite existing data.
|
2017-11-09 01:00:31 -05:00
|
|
|
* `error` or `errorifexists`: Throw an exception if data already exists.
|
2015-06-02 11:37:18 -04:00
|
|
|
* `ignore`: Silently ignore this operation if data already exists.
|
2015-06-03 03:23:34 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2015-06-03 03:23:34 -04:00
|
|
|
>>> df.write.mode('append').parquet(os.path.join(tempfile.mkdtemp(), 'data'))
|
2015-06-02 11:37:18 -04:00
|
|
|
"""
|
2015-06-22 16:51:23 -04:00
|
|
|
# At the JVM side, the default value of mode is already set to "error".
|
|
|
|
# So, if the given saveMode is None, we will not call JVM-side's mode method.
|
|
|
|
if saveMode is not None:
|
|
|
|
self._jwrite = self._jwrite.mode(saveMode)
|
2015-06-02 11:37:18 -04:00
|
|
|
return self
|
|
|
|
|
|
|
|
def format(self, source):
|
2015-06-03 03:23:34 -04:00
|
|
|
"""Specifies the underlying output data source.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
source : str
|
|
|
|
string, name of the data source, e.g. 'json', 'parquet'.
|
2015-06-03 03:23:34 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-06-03 03:23:34 -04:00
|
|
|
>>> df.write.format('json').save(os.path.join(tempfile.mkdtemp(), 'data'))
|
2015-06-02 11:37:18 -04:00
|
|
|
"""
|
|
|
|
self._jwrite = self._jwrite.format(source)
|
|
|
|
return self
|
|
|
|
|
2015-06-29 03:13:39 -04:00
|
|
|
@since(1.5)
|
|
|
|
def option(self, key, value):
|
|
|
|
"""Adds an output option for the underlying data source.
|
|
|
|
"""
|
2016-04-22 12:19:36 -04:00
|
|
|
self._jwrite = self._jwrite.option(key, to_str(value))
|
2015-06-29 03:13:39 -04:00
|
|
|
return self
|
|
|
|
|
2015-06-02 11:37:18 -04:00
|
|
|
@since(1.4)
|
|
|
|
def options(self, **options):
|
2015-06-03 03:23:34 -04:00
|
|
|
"""Adds output options for the underlying data source.
|
2015-06-02 11:37:18 -04:00
|
|
|
"""
|
|
|
|
for k in options:
|
2016-04-22 12:19:36 -04:00
|
|
|
self._jwrite = self._jwrite.option(k, to_str(options[k]))
|
2015-06-02 11:37:18 -04:00
|
|
|
return self
|
|
|
|
|
|
|
|
def partitionBy(self, *cols):
|
2015-06-03 03:23:34 -04:00
|
|
|
"""Partitions the output by the given columns on the file system.
|
|
|
|
|
2015-06-02 11:37:18 -04:00
|
|
|
If specified, the output is laid out on the file system similar
|
|
|
|
to Hive's partitioning scheme.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
2015-06-03 03:23:34 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
cols : str or list
|
|
|
|
name of columns
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2015-06-03 03:23:34 -04:00
|
|
|
>>> df.write.partitionBy('year', 'month').parquet(os.path.join(tempfile.mkdtemp(), 'data'))
|
2015-06-02 11:37:18 -04:00
|
|
|
"""
|
|
|
|
if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
|
|
|
|
cols = cols[0]
|
2016-05-11 14:24:16 -04:00
|
|
|
self._jwrite = self._jwrite.partitionBy(_to_seq(self._spark._sc, cols))
|
2017-05-07 22:58:27 -04:00
|
|
|
return self
|
|
|
|
|
|
|
|
def bucketBy(self, numBuckets, col, *cols):
|
|
|
|
"""Buckets the output by the given columns.If specified,
|
|
|
|
the output is laid out on the file system similar to Hive's bucketing scheme.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.3.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
numBuckets : int
|
|
|
|
the number of buckets to save
|
|
|
|
col : str, list or tuple
|
|
|
|
a name of a column, or a list of names.
|
|
|
|
cols : str
|
|
|
|
additional names (optional). If `col` is a list it should be empty.
|
2017-05-07 22:58:27 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
Applicable for file-based data sources in combination with
|
|
|
|
:py:meth:`DataFrameWriter.saveAsTable`.
|
2017-05-07 22:58:27 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2017-05-10 12:33:49 -04:00
|
|
|
>>> (df.write.format('parquet') # doctest: +SKIP
|
2017-05-07 22:58:27 -04:00
|
|
|
... .bucketBy(100, 'year', 'month')
|
|
|
|
... .mode("overwrite")
|
|
|
|
... .saveAsTable('bucketed_table'))
|
|
|
|
"""
|
|
|
|
if not isinstance(numBuckets, int):
|
|
|
|
raise TypeError("numBuckets should be an int, got {0}.".format(type(numBuckets)))
|
|
|
|
|
|
|
|
if isinstance(col, (list, tuple)):
|
|
|
|
if cols:
|
|
|
|
raise ValueError("col is a {0} but cols are not empty".format(type(col)))
|
|
|
|
|
|
|
|
col, cols = col[0], col[1:]
|
|
|
|
|
2020-07-13 22:22:44 -04:00
|
|
|
if not all(isinstance(c, str) for c in cols) or not(isinstance(col, str)):
|
2017-05-07 22:58:27 -04:00
|
|
|
raise TypeError("all names should be `str`")
|
|
|
|
|
|
|
|
self._jwrite = self._jwrite.bucketBy(numBuckets, col, _to_seq(self._spark._sc, cols))
|
|
|
|
return self
|
|
|
|
|
|
|
|
def sortBy(self, col, *cols):
|
|
|
|
"""Sorts the output in each bucket by the given columns on the file system.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.3.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
col : str, tuple or list
|
|
|
|
a name of a column, or a list of names.
|
|
|
|
cols : str
|
|
|
|
additional names (optional). If `col` is a list it should be empty.
|
2017-05-07 22:58:27 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2017-05-10 12:33:49 -04:00
|
|
|
>>> (df.write.format('parquet') # doctest: +SKIP
|
2017-05-07 22:58:27 -04:00
|
|
|
... .bucketBy(100, 'year', 'month')
|
|
|
|
... .sortBy('day')
|
|
|
|
... .mode("overwrite")
|
|
|
|
... .saveAsTable('sorted_bucketed_table'))
|
|
|
|
"""
|
|
|
|
if isinstance(col, (list, tuple)):
|
|
|
|
if cols:
|
|
|
|
raise ValueError("col is a {0} but cols are not empty".format(type(col)))
|
|
|
|
|
|
|
|
col, cols = col[0], col[1:]
|
|
|
|
|
2020-07-13 22:22:44 -04:00
|
|
|
if not all(isinstance(c, str) for c in cols) or not(isinstance(col, str)):
|
2017-05-07 22:58:27 -04:00
|
|
|
raise TypeError("all names should be `str`")
|
|
|
|
|
|
|
|
self._jwrite = self._jwrite.sortBy(col, _to_seq(self._spark._sc, cols))
|
2015-06-02 11:37:18 -04:00
|
|
|
return self
|
|
|
|
|
2015-06-29 03:22:44 -04:00
|
|
|
def save(self, path=None, format=None, mode=None, partitionBy=None, **options):
|
2015-06-03 03:23:34 -04:00
|
|
|
"""Saves the contents of the :class:`DataFrame` to a data source.
|
2015-05-19 17:23:28 -04:00
|
|
|
|
|
|
|
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.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str, optional
|
|
|
|
the path in a Hadoop supported file system
|
|
|
|
format : str, optional
|
|
|
|
the format used to save
|
|
|
|
mode : str, optional
|
|
|
|
specifies the behavior of the save operation when data already exists.
|
2015-06-03 03:23:34 -04:00
|
|
|
|
|
|
|
* ``append``: Append contents of this :class:`DataFrame` to existing data.
|
|
|
|
* ``overwrite``: Overwrite existing data.
|
|
|
|
* ``ignore``: Silently ignore this operation if data already exists.
|
2017-11-09 01:00:31 -05:00
|
|
|
* ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
|
|
|
|
exists.
|
2020-11-02 20:00:49 -05:00
|
|
|
partitionBy : list, optional
|
|
|
|
names of partitioning columns
|
|
|
|
**options : dict
|
|
|
|
all other string options
|
2015-06-03 03:23:34 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2019-11-01 14:55:29 -04:00
|
|
|
>>> df.write.mode("append").save(os.path.join(tempfile.mkdtemp(), 'data'))
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
2015-06-29 03:22:44 -04:00
|
|
|
self.mode(mode).options(**options)
|
|
|
|
if partitionBy is not None:
|
|
|
|
self.partitionBy(partitionBy)
|
2015-05-19 17:23:28 -04:00
|
|
|
if format is not None:
|
2015-06-02 11:37:18 -04:00
|
|
|
self.format(format)
|
2015-05-19 17:23:28 -04:00
|
|
|
if path is None:
|
2015-06-02 11:37:18 -04:00
|
|
|
self._jwrite.save()
|
2015-05-19 17:23:28 -04:00
|
|
|
else:
|
2015-06-02 11:37:18 -04:00
|
|
|
self._jwrite.save(path)
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2015-06-02 11:37:18 -04:00
|
|
|
@since(1.4)
|
2019-07-18 00:37:59 -04:00
|
|
|
def insertInto(self, tableName, overwrite=None):
|
2015-06-03 03:23:34 -04:00
|
|
|
"""Inserts the content of the :class:`DataFrame` to the specified table.
|
|
|
|
|
2020-05-18 07:25:02 -04:00
|
|
|
It requires that the schema of the :class:`DataFrame` is the same as the
|
2015-05-23 12:07:14 -04:00
|
|
|
schema of the table.
|
|
|
|
|
|
|
|
Optionally overwriting any existing data.
|
|
|
|
"""
|
2019-07-18 00:37:59 -04:00
|
|
|
if overwrite is not None:
|
|
|
|
self.mode("overwrite" if overwrite else "append")
|
|
|
|
self._jwrite.insertInto(tableName)
|
2015-05-23 12:07:14 -04:00
|
|
|
|
2015-06-29 03:22:44 -04:00
|
|
|
def saveAsTable(self, name, format=None, mode=None, partitionBy=None, **options):
|
2015-06-03 03:23:34 -04:00
|
|
|
"""Saves the content of the :class:`DataFrame` as the specified table.
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2015-05-23 12:07:14 -04:00
|
|
|
In the case the table already exists, behavior of this function depends on the
|
|
|
|
save mode, specified by the `mode` function (default to throwing an exception).
|
2016-06-12 02:20:40 -04:00
|
|
|
When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be
|
2015-05-23 12:07:14 -04:00
|
|
|
the same as that of the existing table.
|
2015-05-19 17:23:28 -04:00
|
|
|
|
|
|
|
* `append`: Append contents of this :class:`DataFrame` to existing data.
|
|
|
|
* `overwrite`: Overwrite existing data.
|
2017-11-09 01:00:31 -05:00
|
|
|
* `error` or `errorifexists`: Throw an exception if data already exists.
|
2015-05-19 17:23:28 -04:00
|
|
|
* `ignore`: Silently ignore this operation if data already exists.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
name : str
|
|
|
|
the table name
|
|
|
|
format : str, optional
|
|
|
|
the format used to save
|
|
|
|
mode : str, optional
|
|
|
|
one of `append`, `overwrite`, `error`, `errorifexists`, `ignore` \
|
|
|
|
(default: error)
|
|
|
|
partitionBy : str or list
|
|
|
|
names of partitioning columns
|
|
|
|
**options : dict
|
|
|
|
all other string options
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
2015-06-29 03:22:44 -04:00
|
|
|
self.mode(mode).options(**options)
|
|
|
|
if partitionBy is not None:
|
|
|
|
self.partitionBy(partitionBy)
|
2015-05-19 17:23:28 -04:00
|
|
|
if format is not None:
|
2015-06-02 11:37:18 -04:00
|
|
|
self.format(format)
|
2015-06-03 03:23:34 -04:00
|
|
|
self._jwrite.saveAsTable(name)
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2018-03-28 07:49:27 -04:00
|
|
|
def json(self, path, mode=None, compression=None, dateFormat=None, timestampFormat=None,
|
2019-10-24 13:25:04 -04:00
|
|
|
lineSep=None, encoding=None, ignoreNullFields=None):
|
2017-04-12 04:16:39 -04:00
|
|
|
"""Saves the content of the :class:`DataFrame` in JSON format
|
|
|
|
(`JSON Lines text format or newline-delimited JSON <http://jsonlines.org/>`_) at the
|
|
|
|
specified path.
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str
|
|
|
|
the path in any Hadoop supported file system
|
|
|
|
mode : str, optional
|
|
|
|
specifies the behavior of the save operation when data already exists.
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2015-06-03 03:23:34 -04:00
|
|
|
* ``append``: Append contents of this :class:`DataFrame` to existing data.
|
|
|
|
* ``overwrite``: Overwrite existing data.
|
|
|
|
* ``ignore``: Silently ignore this operation if data already exists.
|
2017-11-09 01:00:31 -05:00
|
|
|
* ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
|
|
|
|
exists.
|
2021-05-21 05:05:13 -04:00
|
|
|
|
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
2021-05-31 21:58:49 -04:00
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
|
2021-05-21 05:05:13 -04:00
|
|
|
in the version you use.
|
2020-11-02 20:00:49 -05:00
|
|
|
|
2021-05-31 21:58:49 -04:00
|
|
|
.. # noqa
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-06-03 03:23:34 -04:00
|
|
|
>>> df.write.json(os.path.join(tempfile.mkdtemp(), 'data'))
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
2016-03-03 13:30:55 -05:00
|
|
|
self.mode(mode)
|
2016-08-24 16:16:20 -04:00
|
|
|
self._set_opts(
|
2018-03-28 07:49:27 -04:00
|
|
|
compression=compression, dateFormat=dateFormat, timestampFormat=timestampFormat,
|
2019-10-24 13:25:04 -04:00
|
|
|
lineSep=lineSep, encoding=encoding, ignoreNullFields=ignoreNullFields)
|
2016-03-03 13:30:55 -05:00
|
|
|
self._jwrite.json(path)
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2016-03-03 13:30:55 -05:00
|
|
|
def parquet(self, path, mode=None, partitionBy=None, compression=None):
|
2015-06-03 03:23:34 -04:00
|
|
|
"""Saves the content of the :class:`DataFrame` in Parquet format at the specified path.
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str
|
|
|
|
the path in any Hadoop supported file system
|
|
|
|
mode : str, optional
|
|
|
|
specifies the behavior of the save operation when data already exists.
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2015-06-03 03:23:34 -04:00
|
|
|
* ``append``: Append contents of this :class:`DataFrame` to existing data.
|
|
|
|
* ``overwrite``: Overwrite existing data.
|
|
|
|
* ``ignore``: Silently ignore this operation if data already exists.
|
2017-11-09 01:00:31 -05:00
|
|
|
* ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
|
|
|
|
exists.
|
2020-11-02 20:00:49 -05:00
|
|
|
partitionBy : str or list, optional
|
|
|
|
names of partitioning columns
|
2021-05-21 05:05:49 -04:00
|
|
|
|
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
2021-05-31 21:58:49 -04:00
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_
|
2021-05-21 05:05:49 -04:00
|
|
|
in the version you use.
|
2020-11-02 20:00:49 -05:00
|
|
|
|
2021-05-31 21:58:49 -04:00
|
|
|
.. # noqa
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-06-03 03:23:34 -04:00
|
|
|
>>> df.write.parquet(os.path.join(tempfile.mkdtemp(), 'data'))
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
2015-06-29 03:22:44 -04:00
|
|
|
self.mode(mode)
|
|
|
|
if partitionBy is not None:
|
|
|
|
self.partitionBy(partitionBy)
|
2016-06-28 16:43:59 -04:00
|
|
|
self._set_opts(compression=compression)
|
2015-06-22 16:51:23 -04:00
|
|
|
self._jwrite.parquet(path)
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2018-03-21 12:46:47 -04:00
|
|
|
def text(self, path, compression=None, lineSep=None):
|
2015-10-28 17:28:38 -04:00
|
|
|
"""Saves the content of the DataFrame in a text file at the specified path.
|
2019-03-04 18:03:39 -05:00
|
|
|
The text files will be encoded as UTF-8.
|
2015-10-28 17:28:38 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.6.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str
|
|
|
|
the path in any Hadoop supported file system
|
2021-05-26 04:12:49 -04:00
|
|
|
|
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
2021-05-31 21:58:49 -04:00
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
|
2021-05-26 04:12:49 -04:00
|
|
|
in the version you use.
|
2016-02-29 12:44:29 -05:00
|
|
|
|
2021-05-31 21:58:49 -04:00
|
|
|
.. # noqa
|
|
|
|
|
2015-10-28 17:28:38 -04:00
|
|
|
The DataFrame must have only one column that is of string type.
|
|
|
|
Each row becomes a new line in the output file.
|
|
|
|
"""
|
2018-03-21 12:46:47 -04:00
|
|
|
self._set_opts(compression=compression, lineSep=lineSep)
|
2015-10-28 17:28:38 -04:00
|
|
|
self._jwrite.text(path)
|
|
|
|
|
2016-06-06 02:40:13 -04:00
|
|
|
def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=None,
|
2016-08-24 16:16:20 -04:00
|
|
|
header=None, nullValue=None, escapeQuotes=None, quoteAll=None, dateFormat=None,
|
2017-12-28 18:30:06 -05:00
|
|
|
timestampFormat=None, ignoreLeadingWhiteSpace=None, ignoreTrailingWhiteSpace=None,
|
2018-11-23 10:50:20 -05:00
|
|
|
charToEscapeQuoteEscaping=None, encoding=None, emptyValue=None, lineSep=None):
|
2018-09-12 23:19:43 -04:00
|
|
|
r"""Saves the content of the :class:`DataFrame` in CSV format at the specified path.
|
2016-02-29 12:44:29 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str
|
|
|
|
the path in any Hadoop supported file system
|
|
|
|
mode : str, optional
|
|
|
|
specifies the behavior of the save operation when data already exists.
|
2016-02-29 12:44:29 -05:00
|
|
|
|
|
|
|
* ``append``: Append contents of this :class:`DataFrame` to existing data.
|
|
|
|
* ``overwrite``: Overwrite existing data.
|
|
|
|
* ``ignore``: Silently ignore this operation if data already exists.
|
2017-11-09 01:00:31 -05:00
|
|
|
* ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
|
|
|
|
exists.
|
2016-02-29 12:44:29 -05:00
|
|
|
|
2021-05-31 21:58:49 -04:00
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
|
|
|
|
in the version you use.
|
|
|
|
|
|
|
|
.. # noqa
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-02-29 12:44:29 -05:00
|
|
|
>>> df.write.csv(os.path.join(tempfile.mkdtemp(), 'data'))
|
|
|
|
"""
|
2016-03-03 13:30:55 -05:00
|
|
|
self.mode(mode)
|
2016-06-28 16:43:59 -04:00
|
|
|
self._set_opts(compression=compression, sep=sep, quote=quote, escape=escape, header=header,
|
2016-08-24 16:16:20 -04:00
|
|
|
nullValue=nullValue, escapeQuotes=escapeQuotes, quoteAll=quoteAll,
|
[SPARK-18579][SQL] Use ignoreLeadingWhiteSpace and ignoreTrailingWhiteSpace options in CSV writing
## What changes were proposed in this pull request?
This PR proposes to support _not_ trimming the white spaces when writing out. These are `false` by default in CSV reading path but these are `true` by default in CSV writing in univocity parser.
Both `ignoreLeadingWhiteSpace` and `ignoreTrailingWhiteSpace` options are not being used for writing and therefore, we are always trimming the white spaces.
It seems we should provide a way to keep this white spaces easily.
WIth the data below:
```scala
val df = spark.read.csv(Seq("a , b , c").toDS)
df.show()
```
```
+---+----+---+
|_c0| _c1|_c2|
+---+----+---+
| a | b | c|
+---+----+---+
```
**Before**
```scala
df.write.csv("/tmp/text.csv")
spark.read.text("/tmp/text.csv").show()
```
```
+-----+
|value|
+-----+
|a,b,c|
+-----+
```
It seems this can't be worked around via `quoteAll` too.
```scala
df.write.option("quoteAll", true).csv("/tmp/text.csv")
spark.read.text("/tmp/text.csv").show()
```
```
+-----------+
| value|
+-----------+
|"a","b","c"|
+-----------+
```
**After**
```scala
df.write.option("ignoreLeadingWhiteSpace", false).option("ignoreTrailingWhiteSpace", false).csv("/tmp/text.csv")
spark.read.text("/tmp/text.csv").show()
```
```
+----------+
| value|
+----------+
|a , b , c|
+----------+
```
Note that this case is possible in R
```r
> system("cat text.csv")
f1,f2,f3
a , b , c
> df <- read.csv(file="text.csv")
> df
f1 f2 f3
1 a b c
> write.csv(df, file="text1.csv", quote=F, row.names=F)
> system("cat text1.csv")
f1,f2,f3
a , b , c
```
## How was this patch tested?
Unit tests in `CSVSuite` and manual tests for Python.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #17310 from HyukjinKwon/SPARK-18579.
2017-03-23 03:25:01 -04:00
|
|
|
dateFormat=dateFormat, timestampFormat=timestampFormat,
|
|
|
|
ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace,
|
2017-12-28 18:30:06 -05:00
|
|
|
ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace,
|
2018-07-25 02:17:20 -04:00
|
|
|
charToEscapeQuoteEscaping=charToEscapeQuoteEscaping,
|
2018-11-23 10:50:20 -05:00
|
|
|
encoding=encoding, emptyValue=emptyValue, lineSep=lineSep)
|
2016-03-03 13:30:55 -05:00
|
|
|
self._jwrite.csv(path)
|
2016-02-29 12:44:29 -05:00
|
|
|
|
2016-03-03 13:30:55 -05:00
|
|
|
def orc(self, path, mode=None, partitionBy=None, compression=None):
|
2015-07-21 03:08:44 -04:00
|
|
|
"""Saves the content of the :class:`DataFrame` in ORC format at the specified path.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str
|
|
|
|
the path in any Hadoop supported file system
|
|
|
|
mode : str, optional
|
|
|
|
specifies the behavior of the save operation when data already exists.
|
2015-07-21 03:08:44 -04:00
|
|
|
|
|
|
|
* ``append``: Append contents of this :class:`DataFrame` to existing data.
|
|
|
|
* ``overwrite``: Overwrite existing data.
|
|
|
|
* ``ignore``: Silently ignore this operation if data already exists.
|
2017-11-09 01:00:31 -05:00
|
|
|
* ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
|
|
|
|
exists.
|
2020-11-02 20:00:49 -05:00
|
|
|
partitionBy : str or list, optional
|
|
|
|
names of partitioning columns
|
2021-05-21 05:03:57 -04:00
|
|
|
|
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
2021-05-31 21:58:49 -04:00
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-orc.html#data-source-option>`_
|
2021-05-21 05:03:57 -04:00
|
|
|
in the version you use.
|
2020-11-02 20:00:49 -05:00
|
|
|
|
2021-05-31 21:58:49 -04:00
|
|
|
.. # noqa
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-05-11 18:31:16 -04:00
|
|
|
>>> orc_df = spark.read.orc('python/test_support/sql/orc_partitioned')
|
2015-07-21 03:08:44 -04:00
|
|
|
>>> orc_df.write.orc(os.path.join(tempfile.mkdtemp(), 'data'))
|
|
|
|
"""
|
|
|
|
self.mode(mode)
|
|
|
|
if partitionBy is not None:
|
|
|
|
self.partitionBy(partitionBy)
|
2016-06-29 16:32:03 -04:00
|
|
|
self._set_opts(compression=compression)
|
2015-07-21 03:08:44 -04:00
|
|
|
self._jwrite.orc(path)
|
|
|
|
|
2015-08-14 15:46:05 -04:00
|
|
|
def jdbc(self, url, table, mode=None, properties=None):
|
2016-05-27 01:39:14 -04:00
|
|
|
"""Saves the content of the :class:`DataFrame` to an external database table via JDBC.
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
table : str
|
|
|
|
Name of the table in the external database.
|
|
|
|
mode : str, optional
|
|
|
|
specifies the behavior of the save operation when data already exists.
|
2015-06-03 03:23:34 -04:00
|
|
|
|
|
|
|
* ``append``: Append contents of this :class:`DataFrame` to existing data.
|
|
|
|
* ``overwrite``: Overwrite existing data.
|
|
|
|
* ``ignore``: Silently ignore this operation if data already exists.
|
2017-11-09 01:00:31 -05:00
|
|
|
* ``error`` or ``errorifexists`` (default case): Throw an exception if data already \
|
|
|
|
exists.
|
2020-11-02 20:00:49 -05:00
|
|
|
properties : dict
|
|
|
|
a dictionary of JDBC database connection arguments. Normally at
|
|
|
|
least properties "user" and "password" with their corresponding values.
|
|
|
|
For example { 'user' : 'SYSTEM', 'password' : 'mypassword' }
|
|
|
|
|
2021-06-02 01:21:16 -04:00
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-jdbc.html#data-source-option>`_
|
|
|
|
in the version you use.
|
|
|
|
|
|
|
|
.. # noqa
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
Don't create too many partitions in parallel on a large cluster;
|
|
|
|
otherwise Spark might crash your external database systems.
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
2015-08-14 15:46:05 -04:00
|
|
|
if properties is None:
|
|
|
|
properties = dict()
|
2016-05-11 14:24:16 -04:00
|
|
|
jprop = JavaClass("java.util.Properties", self._spark._sc._gateway._gateway_client)()
|
2015-05-19 17:23:28 -04:00
|
|
|
for k in properties:
|
|
|
|
jprop.setProperty(k, properties[k])
|
2017-11-04 03:59:58 -04:00
|
|
|
self.mode(mode)._jwrite.jdbc(url, table, jprop)
|
2015-05-19 17:23:28 -04:00
|
|
|
|
|
|
|
|
2020-07-19 21:42:33 -04:00
|
|
|
class DataFrameWriterV2(object):
|
|
|
|
"""
|
|
|
|
Interface used to write a class:`pyspark.sql.dataframe.DataFrame`
|
|
|
|
to external storage using the v2 API.
|
|
|
|
|
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
"""
|
|
|
|
|
|
|
|
def __init__(self, df, table):
|
|
|
|
self._df = df
|
|
|
|
self._spark = df.sql_ctx
|
|
|
|
self._jwriter = df._jdf.writeTo(table)
|
|
|
|
|
|
|
|
@since(3.1)
|
|
|
|
def using(self, provider):
|
|
|
|
"""
|
|
|
|
Specifies a provider for the underlying output data source.
|
|
|
|
Spark's default catalog supports "parquet", "json", etc.
|
|
|
|
"""
|
|
|
|
self._jwriter.using(provider)
|
|
|
|
return self
|
|
|
|
|
|
|
|
@since(3.1)
|
|
|
|
def option(self, key, value):
|
|
|
|
"""
|
|
|
|
Add a write option.
|
|
|
|
"""
|
|
|
|
self._jwriter.option(key, to_str(value))
|
|
|
|
return self
|
|
|
|
|
|
|
|
@since(3.1)
|
|
|
|
def options(self, **options):
|
|
|
|
"""
|
|
|
|
Add write options.
|
|
|
|
"""
|
|
|
|
options = {k: to_str(v) for k, v in options.items()}
|
|
|
|
self._jwriter.options(options)
|
|
|
|
return self
|
|
|
|
|
|
|
|
@since(3.1)
|
|
|
|
def tableProperty(self, property, value):
|
|
|
|
"""
|
|
|
|
Add table property.
|
|
|
|
"""
|
|
|
|
self._jwriter.tableProperty(property, value)
|
|
|
|
return self
|
|
|
|
|
|
|
|
@since(3.1)
|
|
|
|
def partitionedBy(self, col, *cols):
|
|
|
|
"""
|
|
|
|
Partition the output table created by `create`, `createOrReplace`, or `replace` using
|
|
|
|
the given columns or transforms.
|
|
|
|
|
|
|
|
When specified, the table data will be stored by these values for efficient reads.
|
|
|
|
|
|
|
|
For example, when a table is partitioned by day, it may be stored
|
|
|
|
in a directory layout like:
|
|
|
|
|
|
|
|
* `table/day=2019-06-01/`
|
|
|
|
* `table/day=2019-06-02/`
|
|
|
|
|
|
|
|
Partitioning is one of the most widely used techniques to optimize physical data layout.
|
|
|
|
It provides a coarse-grained index for skipping unnecessary data reads when queries have
|
|
|
|
predicates on the partitioned columns. In order for partitioning to work well, the number
|
|
|
|
of distinct values in each column should typically be less than tens of thousands.
|
|
|
|
|
|
|
|
`col` and `cols` support only the following functions:
|
|
|
|
|
|
|
|
* :py:func:`pyspark.sql.functions.years`
|
|
|
|
* :py:func:`pyspark.sql.functions.months`
|
|
|
|
* :py:func:`pyspark.sql.functions.days`
|
|
|
|
* :py:func:`pyspark.sql.functions.hours`
|
|
|
|
* :py:func:`pyspark.sql.functions.bucket`
|
|
|
|
|
|
|
|
"""
|
|
|
|
col = _to_java_column(col)
|
|
|
|
cols = _to_seq(self._spark._sc, [_to_java_column(c) for c in cols])
|
|
|
|
return self
|
|
|
|
|
|
|
|
@since(3.1)
|
|
|
|
def create(self):
|
|
|
|
"""
|
|
|
|
Create a new table from the contents of the data frame.
|
|
|
|
|
|
|
|
The new table's schema, partition layout, properties, and other configuration will be
|
|
|
|
based on the configuration set on this writer.
|
|
|
|
"""
|
|
|
|
self._jwriter.create()
|
|
|
|
|
|
|
|
@since(3.1)
|
|
|
|
def replace(self):
|
|
|
|
"""
|
|
|
|
Replace an existing table with the contents of the data frame.
|
|
|
|
|
|
|
|
The existing table's schema, partition layout, properties, and other configuration will be
|
|
|
|
replaced with the contents of the data frame and the configuration set on this writer.
|
|
|
|
"""
|
|
|
|
self._jwriter.replace()
|
|
|
|
|
|
|
|
@since(3.1)
|
|
|
|
def createOrReplace(self):
|
|
|
|
"""
|
|
|
|
Create a new table or replace an existing table with the contents of the data frame.
|
|
|
|
|
|
|
|
The output table's schema, partition layout, properties,
|
|
|
|
and other configuration will be based on the contents of the data frame
|
|
|
|
and the configuration set on this writer.
|
|
|
|
If the table exists, its configuration and data will be replaced.
|
|
|
|
"""
|
|
|
|
self._jwriter.createOrReplace()
|
|
|
|
|
|
|
|
@since(3.1)
|
|
|
|
def append(self):
|
|
|
|
"""
|
|
|
|
Append the contents of the data frame to the output table.
|
|
|
|
"""
|
|
|
|
self._jwriter.append()
|
|
|
|
|
|
|
|
@since(3.1)
|
|
|
|
def overwrite(self, condition):
|
|
|
|
"""
|
|
|
|
Overwrite rows matching the given filter condition with the contents of the data frame in
|
|
|
|
the output table.
|
|
|
|
"""
|
|
|
|
self._jwriter.overwrite(condition)
|
|
|
|
|
|
|
|
@since(3.1)
|
|
|
|
def overwritePartitions(self):
|
|
|
|
"""
|
|
|
|
Overwrite all partition for which the data frame contains at least one row with the contents
|
|
|
|
of the data frame in the output table.
|
|
|
|
|
|
|
|
This operation is equivalent to Hive's `INSERT OVERWRITE ... PARTITION`, which replaces
|
|
|
|
partitions dynamically depending on the contents of the data frame.
|
|
|
|
"""
|
|
|
|
self._jwriter.overwritePartitions()
|
|
|
|
|
|
|
|
|
2015-05-19 17:23:28 -04:00
|
|
|
def _test():
|
|
|
|
import doctest
|
2015-06-03 03:23:34 -04:00
|
|
|
import os
|
|
|
|
import tempfile
|
2016-05-11 18:31:16 -04:00
|
|
|
import py4j
|
2015-05-19 17:23:28 -04:00
|
|
|
from pyspark.context import SparkContext
|
2020-08-08 11:51:57 -04:00
|
|
|
from pyspark.sql import SparkSession
|
2015-05-19 17:23:28 -04:00
|
|
|
import pyspark.sql.readwriter
|
2015-06-03 03:23:34 -04:00
|
|
|
|
|
|
|
os.chdir(os.environ["SPARK_HOME"])
|
|
|
|
|
2015-05-19 17:23:28 -04:00
|
|
|
globs = pyspark.sql.readwriter.__dict__.copy()
|
|
|
|
sc = SparkContext('local[4]', 'PythonTest')
|
2016-05-11 18:31:16 -04:00
|
|
|
try:
|
2018-04-30 21:06:23 -04:00
|
|
|
spark = SparkSession.builder.getOrCreate()
|
2016-05-11 18:31:16 -04:00
|
|
|
except py4j.protocol.Py4JError:
|
|
|
|
spark = SparkSession(sc)
|
2015-06-03 03:23:34 -04:00
|
|
|
|
|
|
|
globs['tempfile'] = tempfile
|
|
|
|
globs['os'] = os
|
2015-05-19 17:23:28 -04:00
|
|
|
globs['sc'] = sc
|
2016-05-11 18:31:16 -04:00
|
|
|
globs['spark'] = spark
|
|
|
|
globs['df'] = spark.read.parquet('python/test_support/sql/parquet_partitioned')
|
2015-05-19 17:23:28 -04:00
|
|
|
(failure_count, test_count) = doctest.testmod(
|
|
|
|
pyspark.sql.readwriter, globs=globs,
|
|
|
|
optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF)
|
2016-05-11 18:31:16 -04:00
|
|
|
sc.stop()
|
2015-05-19 17:23:28 -04:00
|
|
|
if failure_count:
|
2018-03-08 06:38:34 -05:00
|
|
|
sys.exit(-1)
|
2015-05-19 17:23:28 -04:00
|
|
|
|
|
|
|
|
|
|
|
if __name__ == "__main__":
|
|
|
|
_test()
|