Skip to content

[SPARK-35025][SQL][PYTHON][DOCS] Move Parquet data source options from Python and Scala into a single page. #32161

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 23 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
23 commits
Select commit Hold shift + click to select a range
432d8dd
[SPARK-35025] Move Parquet data source options from Python and Scala …
itholic Apr 14, 2021
fc2e064
Merge branch 'master' of https://github.com/apache/spark into SPARK-3…
itholic Apr 15, 2021
b7aa8c7
Resolve comments
itholic Apr 16, 2021
a538c41
Merge branch 'master' of https://github.com/apache/spark into SPARK-3…
itholic Apr 19, 2021
082d86d
Addressed comments
itholic Apr 19, 2021
53ef3ea
Fix Python linter failuer
itholic Apr 19, 2021
3c3e518
Remove the duplicated options from python docstring
itholic Apr 29, 2021
a5648c5
Remove more duplicated options
itholic Apr 29, 2021
50a6726
Merge branch 'master' of https://github.com/apache/spark into SPARK-3…
itholic May 10, 2021
5077c69
Revert the removal of keyword arguments
itholic May 10, 2021
bb5cd45
Resolved comments
itholic May 13, 2021
3ecc196
Add missing Generic Options link
itholic May 14, 2021
c50df9a
Merge branch 'master' of https://github.com/apache/spark into SPARK-3…
itholic May 14, 2021
883aacf
Retore the patrtitionBy
itholic May 14, 2021
9c4782f
Resolved comments
itholic May 17, 2021
c7f31b3
Merge branch 'master' of https://github.com/apache/spark into SPARK-3…
itholic May 17, 2021
ad9f8a0
Update DataStreamReader
itholic May 18, 2021
45e0f8f
Resolved comments
itholic May 20, 2021
ffc124c
One more fix
itholic May 20, 2021
41ad66e
itemize the options
itholic May 20, 2021
2272717
Resolved comments
itholic May 20, 2021
ead523d
Resolved comments
itholic May 21, 2021
d6417a8
Add noqa
itholic May 21, 2021
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
20 changes: 18 additions & 2 deletions docs/sql-data-sources-parquet.md
Original file line number Diff line number Diff line change
Expand Up @@ -255,8 +255,11 @@ REFRESH TABLE my_table;
## Data Source Option

Data source options of Parquet can be set via:
* the `.option`/`.options` methods of `DataFrameReader` or `DataFrameWriter`
* the `.option`/`.options` methods of `DataStreamReader` or `DataStreamWriter`
* the `.option`/`.options` methods of
* `DataFrameReader`
* `DataFrameWriter`
* `DataStreamReader`
* `DataStreamWriter`
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also mention:

* `OPTIONS` clause at [CREATE TABLE USING DATA_SOURCE](sql-ref-syntax-ddl-create-table-datasource.html)


<table class="table">
<tr><th><b>Property Name</b></th><th><b>Default</b></th><th><b>Meaning</b></th><th><b>Scope</b></th></tr>
Expand Down Expand Up @@ -286,7 +289,20 @@ Data source options of Parquet can be set via:
</td>
<td>read</td>
</tr>
<tr>
<td><code>mergeSchema</code></td>
<td>The SQL config <code>spark.sql.parquet.mergeSchema</code> which is <code>false</code> by default.</td>
<td>Sets whether we should merge schemas collected from all Parquet part-files. This will override <code>spark.sql.parquet.mergeSchema</code>.</td>
<td>read</td>
</tr>
<tr>
<td><code>compression</code></td>
<td>None</td>
<td>Compression codec to use when saving to file. This can be one of the known case-insensitive shorten names (none, uncompressed, snappy, gzip, lzo, brotli, lz4, and zstd). This will override <code>spark.sql.parquet.compression.codec</code>. If None is set, it uses the value specified in <code>spark.sql.parquet.compression.codec</code>.</td>
<td>write</td>
</tr>
</table>
Other generic options can be found in <a href="https://spark.apache.org/docs/latest/sql-data-sources-generic-options.html"> Generic Files Source Options</a>

### Configuration

Expand Down
64 changes: 11 additions & 53 deletions python/pyspark/sql/readwriter.py
Original file line number Diff line number Diff line change
Expand Up @@ -416,53 +416,10 @@ def parquet(self, *paths, **options):

Other Parameters
----------------
mergeSchema : str or bool, optional
sets whether we should merge schemas collected from all
Parquet part-files. This will override
``spark.sql.parquet.mergeSchema``. The default value is specified in
``spark.sql.parquet.mergeSchema``.
pathGlobFilter : str or bool, optional
an optional glob pattern to only include files with paths matching
the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`.
It does not change the behavior of
`partition discovery <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery>`_. # noqa
recursiveFileLookup : str or bool, optional
recursively scan a directory for files. Using this option
disables
`partition discovery <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery>`_. # noqa

modification times occurring before the specified time. The provided timestamp
must be in the following format: YYYY-MM-DDTHH:mm:ss (e.g. 2020-06-01T13:00:00)
modifiedBefore (batch only) : an optional timestamp to only include files with
modification times occurring before the specified time. The provided timestamp
must be in the following format: YYYY-MM-DDTHH:mm:ss (e.g. 2020-06-01T13:00:00)
modifiedAfter (batch only) : an optional timestamp to only include files with
modification times occurring after the specified time. The provided timestamp
must be in the following format: YYYY-MM-DDTHH:mm:ss (e.g. 2020-06-01T13:00:00)
datetimeRebaseMode : str, optional
the rebasing mode for the values of the ``DATE``, ``TIMESTAMP_MICROS``,
``TIMESTAMP_MILLIS`` logical types from the Julian to Proleptic Gregorian calendar.

* ``EXCEPTION``: Spark fails in reads of ancient dates/timestamps
that are ambiguous between the two calendars.
* ``CORRECTED``: loading of dates/timestamps without rebasing.
* ``LEGACY``: perform rebasing of ancient dates/timestamps from the Julian
to Proleptic Gregorian calendar.

If None is set, the value of the SQL config
``spark.sql.parquet.datetimeRebaseModeInRead`` is used by default.
int96RebaseMode : str, optional
the rebasing mode for ``INT96`` timestamps from the Julian to
Proleptic Gregorian calendar.

* ``EXCEPTION``: Spark fails in reads of ancient ``INT96`` timestamps
that are ambiguous between the two calendars.
* ``CORRECTED``: loading of ``INT96`` timestamps without rebasing.
* ``LEGACY``: perform rebasing of ancient ``INT96`` timestamps from the Julian
to Proleptic Gregorian calendar.

If None is set, the value of the SQL config
``spark.sql.parquet.int96RebaseModeInRead`` is used by default.
**options
For the extra options, refer to
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_ # noqa
in the version you use.

Examples
--------
Expand Down Expand Up @@ -1259,12 +1216,13 @@ def parquet(self, path, mode=None, partitionBy=None, compression=None):
exists.
partitionBy : str or list, optional
names of partitioning columns
compression : str, optional
compression codec to use when saving to file. This can be one of the
known case-insensitive shorten names (none, uncompressed, snappy, gzip,
lzo, brotli, lz4, and zstd). This will override
``spark.sql.parquet.compression.codec``. If None is set, it uses the
value specified in ``spark.sql.parquet.compression.codec``.

Other Parameters
----------------
Extra options
For the extra options, refer to
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_ # noqa
in the version you use.

Examples
--------
Expand Down
46 changes: 9 additions & 37 deletions python/pyspark/sql/streaming.py
Original file line number Diff line number Diff line change
Expand Up @@ -676,43 +676,15 @@ def parquet(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLook

Parameters
----------
mergeSchema : str or bool, optional
sets whether we should merge schemas collected from all
Parquet part-files. This will override
``spark.sql.parquet.mergeSchema``. The default value is specified in
``spark.sql.parquet.mergeSchema``.
pathGlobFilter : str or bool, optional
an optional glob pattern to only include files with paths matching
the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`.
It does not change the behavior of `partition discovery`_.
recursiveFileLookup : str or bool, optional
recursively scan a directory for files. Using this option
disables
`partition discovery <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery>`_. # noqa
datetimeRebaseMode : str, optional
the rebasing mode for the values of the ``DATE``, ``TIMESTAMP_MICROS``,
``TIMESTAMP_MILLIS`` logical types from the Julian to Proleptic Gregorian calendar.

* ``EXCEPTION``: Spark fails in reads of ancient dates/timestamps
that are ambiguous between the two calendars.
* ``CORRECTED``: loading of dates/timestamps without rebasing.
* ``LEGACY``: perform rebasing of ancient dates/timestamps from the Julian
to Proleptic Gregorian calendar.

If None is set, the value of the SQL config
``spark.sql.parquet.datetimeRebaseModeInRead`` is used by default.
int96RebaseMode : str, optional
the rebasing mode for ``INT96`` timestamps from the Julian to
Proleptic Gregorian calendar.

* ``EXCEPTION``: Spark fails in reads of ancient ``INT96`` timestamps
that are ambiguous between the two calendars.
* ``CORRECTED``: loading of ``INT96`` timestamps without rebasing.
* ``LEGACY``: perform rebasing of ancient ``INT96`` timestamps from the Julian
to Proleptic Gregorian calendar.

If None is set, the value of the SQL config
``spark.sql.parquet.int96RebaseModeInRead`` is used by default.
path : str
the path in any Hadoop supported file system

Other Parameters
----------------
Extra options
For the extra options, refer to
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_. # noqa
in the version you use.

Examples
--------
Expand Down
44 changes: 4 additions & 40 deletions sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
Original file line number Diff line number Diff line change
Expand Up @@ -812,46 +812,10 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
/**
* Loads a Parquet file, returning the result as a `DataFrame`.
*
* You can set the following Parquet-specific option(s) for reading Parquet files:
* <ul>
* <li>`mergeSchema` (default is the value specified in `spark.sql.parquet.mergeSchema`): sets
* whether we should merge schemas collected from all Parquet part-files. This will override
* `spark.sql.parquet.mergeSchema`.</li>
* <li>`pathGlobFilter`: an optional glob pattern to only include files with paths matching
* the pattern. The syntax follows <code>org.apache.hadoop.fs.GlobFilter</code>.
* It does not change the behavior of partition discovery.</li>
* <li>`modifiedBefore` (batch only): an optional timestamp to only include files with
* modification times occurring before the specified Time. The provided timestamp
* must be in the following form: YYYY-MM-DDTHH:mm:ss (e.g. 2020-06-01T13:00:00)</li>
* <li>`modifiedAfter` (batch only): an optional timestamp to only include files with
* modification times occurring after the specified Time. The provided timestamp
* must be in the following form: YYYY-MM-DDTHH:mm:ss (e.g. 2020-06-01T13:00:00)</li>
* <li>`recursiveFileLookup`: recursively scan a directory for files. Using this option
* disables partition discovery</li>
* <li>`datetimeRebaseMode` (default is the value specified in the SQL config
* `spark.sql.parquet.datetimeRebaseModeInRead`): the rebasing mode for the values
* of the `DATE`, `TIMESTAMP_MICROS`, `TIMESTAMP_MILLIS` logical types from the Julian to
* Proleptic Gregorian calendar:
* <ul>
* <li>`EXCEPTION` : Spark fails in reads of ancient dates/timestamps that are ambiguous
* between the two calendars</li>
* <li>`CORRECTED` : loading of dates/timestamps without rebasing</li>
* <li>`LEGACY` : perform rebasing of ancient dates/timestamps from the Julian to Proleptic
* Gregorian calendar</li>
* </ul>
* </li>
* <li>`int96RebaseMode` (default is the value specified in the SQL config
* `spark.sql.parquet.int96RebaseModeInRead`): the rebasing mode for `INT96` timestamps
* from the Julian to Proleptic Gregorian calendar:
* <ul>
* <li>`EXCEPTION` : Spark fails in reads of ancient `INT96` timestamps that are ambiguous
* between the two calendars</li>
* <li>`CORRECTED` : loading of timestamps without rebasing</li>
* <li>`LEGACY` : perform rebasing of ancient `INT96` timestamps from the Julian to Proleptic
* Gregorian calendar</li>
* </ul>
* </li>
* </ul>
* Parquet-specific option(s) for reading Parquet files can be found in
* <a href=
* "https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option">
* Data Source Option</a> in the version you use.
*
* @since 1.4.0
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -860,13 +860,10 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
* format("parquet").save(path)
* }}}
*
* You can set the following Parquet-specific option(s) for writing Parquet files:
* <ul>
* <li>`compression` (default is the value specified in `spark.sql.parquet.compression.codec`):
* compression codec to use when saving to file. This can be one of the known case-insensitive
* shorten names(`none`, `uncompressed`, `snappy`, `gzip`, `lzo`, `brotli`, `lz4`, and `zstd`).
* This will override `spark.sql.parquet.compression.codec`.</li>
* </ul>
* Parquet-specific option(s) for writing Parquet files can be found in
* <a href=
* "https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option">
* Data Source Option</a> in the version you use.
*
* @since 1.4.0
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -476,44 +476,17 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo
/**
* Loads a Parquet file stream, returning the result as a `DataFrame`.
*
* You can set the following Parquet-specific option(s) for reading Parquet files:
* You can set the following option(s):
* <ul>
* <li>`maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be
* considered in every trigger.</li>
* <li>`mergeSchema` (default is the value specified in `spark.sql.parquet.mergeSchema`): sets
* whether we should merge schemas collected from all
* Parquet part-files. This will override
* `spark.sql.parquet.mergeSchema`.</li>
* <li>`pathGlobFilter`: an optional glob pattern to only include files with paths matching
* the pattern. The syntax follows <code>org.apache.hadoop.fs.GlobFilter</code>.
* It does not change the behavior of partition discovery.</li>
* <li>`recursiveFileLookup`: recursively scan a directory for files. Using this option
* disables partition discovery</li>
* <li>`datetimeRebaseMode` (default is the value specified in the SQL config
* `spark.sql.parquet.datetimeRebaseModeInRead`): the rebasing mode for the values
* of the `DATE`, `TIMESTAMP_MICROS`, `TIMESTAMP_MILLIS` logical types from the Julian to
* Proleptic Gregorian calendar:
* <ul>
* <li>`EXCEPTION` : Spark fails in reads of ancient dates/timestamps that are ambiguous
* between the two calendars</li>
* <li>`CORRECTED` : loading of dates/timestamps without rebasing</li>
* <li>`LEGACY` : perform rebasing of ancient dates/timestamps from the Julian to Proleptic
* Gregorian calendar</li>
* </ul>
* </li>
* <li>`int96RebaseMode` (default is the value specified in the SQL config
* `spark.sql.parquet.int96RebaseModeInRead`): the rebasing mode for `INT96` timestamps
* from the Julian to Proleptic Gregorian calendar:
* <ul>
* <li>`EXCEPTION` : Spark fails in reads of ancient `INT96` timestamps that are ambiguous
* between the two calendars</li>
* <li>`CORRECTED` : loading of timestamps without rebasing</li>
* <li>`LEGACY` : perform rebasing of ancient `INT96` timestamps from the Julian to Proleptic
* Gregorian calendar</li>
* </ul>
* </li>
* </ul>
*
* Parquet-specific option(s) for reading Parquet file stream can be found in
* <a href=
* "https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option">
* Data Source Option</a> in the version you use.
*
* @since 2.0.0
*/
def parquet(path: String): DataFrame = {
Expand Down