Skip to content

Conversation

@MaxGekk
Copy link
Member

@MaxGekk MaxGekk commented Feb 5, 2021

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:
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.

  1. 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:
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"

// The file metadata indicates if it needs rebase or not, so we can always get the
// correct result regardless of the "rebase mode" config.
Seq(LEGACY, CORRECTED, EXCEPTION).foreach { mode =>
withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key -> mode.toString) {
Copy link
Member Author

Choose a reason for hiding this comment

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

Fixed the wrong SQL conf: LEGACY_AVRO_REBASE_MODE_IN_READ

@github-actions github-actions bot added the SQL label Feb 5, 2021
@MaxGekk MaxGekk changed the title [WIP][SQL] Support parquet datasource options to control datetime rebasing in read [SPARK-34377][SQL] Support parquet datasource options to control datetime rebasing in read Feb 5, 2021
@MaxGekk
Copy link
Member Author

MaxGekk commented Feb 5, 2021

@cloud-fan @tomvanbussel @mswit-databricks Could you review this PR, please.

@MaxGekk MaxGekk changed the title [SPARK-34377][SQL] Support parquet datasource options to control datetime rebasing in read [SPARK-34377][SQL] Add new parquet datasource options to control datetime rebasing in read Feb 5, 2021
@SparkQA
Copy link

SparkQA commented Feb 5, 2021

Test build #134934 has finished for PR 31489 at commit f898288.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Feb 5, 2021

Test build #134940 has started for PR 31489 at commit eb77fc6.

@SparkQA
Copy link

SparkQA commented Feb 5, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39523/

@SparkQA
Copy link

SparkQA commented Feb 5, 2021

Kubernetes integration test status success
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39523/

@SparkQA
Copy link

SparkQA commented Feb 6, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39534/

@SparkQA
Copy link

SparkQA commented Feb 6, 2021

Kubernetes integration test status success
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39534/

@SparkQA
Copy link

SparkQA commented Feb 6, 2021

Test build #134951 has finished for PR 31489 at commit f33b5a8.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

# Conflicts:
#	sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala
@SparkQA
Copy link

SparkQA commented Feb 7, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39569/

@SparkQA
Copy link

SparkQA commented Feb 7, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39569/

@SparkQA
Copy link

SparkQA commented Feb 7, 2021

Test build #134986 has finished for PR 31489 at commit ebc7298.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

partitionSchema: StructType,
filters: Array[Filter]) extends FilePartitionReaderFactory with Logging {
filters: Array[Filter],
parquetOptions: ParquetOptions) extends FilePartitionReaderFactory with Logging {
Copy link
Contributor

Choose a reason for hiding this comment

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

Does it really work? The 2 fields of ParquetOptions are transient, and become null after (de)serialization.

Copy link
Contributor

Choose a reason for hiding this comment

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

ah nvm, we read the confs and put it in val.

@cloud-fan
Copy link
Contributor

thanks, merging to master!

@cloud-fan cloud-fan closed this in a854906 Feb 8, 2021
dongjoon-hyun pushed a commit that referenced this pull request Feb 15, 2021
### What changes were proposed in this pull request?
Mention the DS options introduced by #31529 and by #31489 in `SparkUpgradeException`.

### Why are the changes needed?
To improve user experience with Spark SQL. Before the changes, the error message recommends to set SQL configs but the configs cannot help in the some situations (see the PRs for more details).

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the error message is:

_org.apache.spark.SparkUpgradeException: You may get a different result due to the upgrading of Spark 3.0: reading dates before 1582-10-15 or timestamps before 1900-01-01T00:00:00Z from Parquet files can be ambiguous, as the files may be written by Spark 2.x or legacy versions of Hive, which uses a legacy hybrid calendar that is different from Spark 3.0+'s Proleptic Gregorian calendar. See more details in SPARK-31404. You can set the SQL config 'spark.sql.legacy.parquet.datetimeRebaseModeInRead' or the datasource option 'datetimeRebaseMode' to 'LEGACY' to rebase the datetime values w.r.t. the calendar difference during reading. To read the datetime values as it is, set the SQL config 'spark.sql.legacy.parquet.datetimeRebaseModeInRead' or the datasource option 'datetimeRebaseMode' to 'CORRECTED'._

### How was this patch tested?
1. By checking coding style: `./dev/scalastyle`
2. By running the related test suite:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ParquetRebaseDatetimeV1Suite"
```

Closes #31562 from MaxGekk/rebase-upgrade-exception.

Authored-by: Max Gekk <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
HyukjinKwon pushed a commit that referenced this pull request Feb 18, 2021
… options and SQL configs

### What changes were proposed in this pull request?
In the PR, I propose to update the Spark SQL guide about the SQL configs that are related to datetime rebasing:
- spark.sql.parquet.int96RebaseModeInWrite
- spark.sql.parquet.datetimeRebaseModeInWrite
- spark.sql.parquet.int96RebaseModeInRead
- spark.sql.parquet.datetimeRebaseModeInRead
- spark.sql.avro.datetimeRebaseModeInWrite
- spark.sql.avro.datetimeRebaseModeInRead

Parquet options added by #31489:
- datetimeRebaseMode
- int96RebaseMode

and Avro options added by #31529:
- datetimeRebaseMode

<img width="998" alt="Screenshot 2021-02-17 at 21 42 09" src="https://user-images.githubusercontent.com/1580697/108252043-3afb8900-7169-11eb-8568-511e21fa7f78.png">

### Why are the changes needed?
To inform users about supported DS options and SQL configs.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
By generating the doc and manually checking:
```
$ SKIP_API=1 SKIP_SCALADOC=1 SKIP_PYTHONDOC=1 SKIP_RDOC=1 jekyll serve --watch
```

Closes #31564 from MaxGekk/doc-rebase-options.

Authored-by: Max Gekk <[email protected]>
Signed-off-by: HyukjinKwon <[email protected]>
domybest11 pushed a commit to domybest11/spark that referenced this pull request Jun 15, 2022
…time 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 apache#31489 from MaxGekk/parquet-rebase-options.

Authored-by: Max Gekk <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>

(cherry picked from commit a854906)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants