Skip to content

Commit 4fd6630

Browse files
committed
Introduce alternatives for rebase configs
1 parent dd6383f commit 4fd6630

File tree

17 files changed

+99
-81
lines changed

17 files changed

+99
-81
lines changed

external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -117,7 +117,7 @@ private[sql] class AvroOptions(
117117
*/
118118
val datetimeRebaseModeInRead: String = parameters
119119
.get(AvroOptions.DATETIME_REBASE_MODE)
120-
.getOrElse(SQLConf.get.getConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ))
120+
.getOrElse(SQLConf.get.getConf(SQLConf.AVRO_REBASE_MODE_IN_READ))
121121
}
122122

123123
private[sql] object AvroOptions {
@@ -133,7 +133,7 @@ private[sql] object AvroOptions {
133133

134134
// The option controls rebasing of the DATE and TIMESTAMP values between
135135
// Julian and Proleptic Gregorian calendars. It impacts on the behaviour of the Avro
136-
// datasource similarly to the SQL config `spark.sql.legacy.avro.datetimeRebaseModeInRead`,
136+
// datasource similarly to the SQL config `spark.sql.avro.datetimeRebaseModeInRead`,
137137
// and can be set to the same values: `EXCEPTION`, `LEGACY` or `CORRECTED`.
138138
val DATETIME_REBASE_MODE = "datetimeRebaseMode"
139139
}

external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriter.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -45,7 +45,7 @@ private[avro] class AvroOutputWriter(
4545

4646
// Whether to rebase datetimes from Gregorian to Julian calendar in write
4747
private val datetimeRebaseMode = LegacyBehaviorPolicy.withName(
48-
SQLConf.get.getConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE))
48+
SQLConf.get.getConf(SQLConf.AVRO_REBASE_MODE_IN_WRITE))
4949

5050
// The input rows will never be null.
5151
private lazy val serializer =

external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@ private[sql] class AvroSerializer(
5353
def this(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean) = {
5454
this(rootCatalystType, rootAvroType, nullable,
5555
LegacyBehaviorPolicy.withName(SQLConf.get.getConf(
56-
SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE)))
56+
SQLConf.AVRO_REBASE_MODE_IN_WRITE)))
5757
}
5858

5959
def serialize(catalystData: Any): Any = {

external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -1805,11 +1805,11 @@ abstract class AvroSuite
18051805
private def runInMode(
18061806
modes: Seq[LegacyBehaviorPolicy.Value])(f: Map[String, String] => Unit): Unit = {
18071807
modes.foreach { mode =>
1808-
withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key -> mode.toString) {
1808+
withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_READ.key -> mode.toString) {
18091809
f(Map.empty)
18101810
}
18111811
}
1812-
withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key -> EXCEPTION.toString) {
1812+
withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_READ.key -> EXCEPTION.toString) {
18131813
modes.foreach { mode =>
18141814
f(Map(AvroOptions.DATETIME_REBASE_MODE -> mode.toString))
18151815
}
@@ -1836,10 +1836,10 @@ abstract class AvroSuite
18361836
assert(e.getCause.getCause.getCause.isInstanceOf[SparkUpgradeException])
18371837
checkDefaultLegacyRead(path2_4)
18381838

1839-
withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE.key -> CORRECTED.toString) {
1839+
withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_WRITE.key -> CORRECTED.toString) {
18401840
df.write.format("avro").mode("overwrite").save(path3_0)
18411841
}
1842-
withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE.key -> LEGACY.toString) {
1842+
withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_WRITE.key -> LEGACY.toString) {
18431843
df.write.format("avro").save(path3_0_rebase)
18441844
}
18451845

@@ -1869,10 +1869,10 @@ abstract class AvroSuite
18691869
assert(e.getCause.getCause.getCause.isInstanceOf[SparkUpgradeException])
18701870
checkDefaultLegacyRead(path2_4)
18711871

1872-
withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE.key -> CORRECTED.toString) {
1872+
withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_WRITE.key -> CORRECTED.toString) {
18731873
df.write.format("avro").option("avroSchema", avroSchema).mode("overwrite").save(path3_0)
18741874
}
1875-
withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE.key -> LEGACY.toString) {
1875+
withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_WRITE.key -> LEGACY.toString) {
18761876
df.write.format("avro").option("avroSchema", avroSchema).save(path3_0_rebase)
18771877
}
18781878

@@ -1921,7 +1921,7 @@ abstract class AvroSuite
19211921
val nonRebased = "1001-01-07 01:09:05.123456"
19221922
withTempPath { dir =>
19231923
val path = dir.getAbsolutePath
1924-
withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE.key -> LEGACY.toString) {
1924+
withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_WRITE.key -> LEGACY.toString) {
19251925
Seq(tsStr).toDF("tsS")
19261926
.select($"tsS".cast("timestamp").as("ts"))
19271927
.write.format("avro")
@@ -1962,7 +1962,7 @@ abstract class AvroSuite
19621962
|}""".stripMargin
19631963
withTempPath { dir =>
19641964
val path = dir.getAbsolutePath
1965-
withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE.key -> LEGACY.toString) {
1965+
withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_WRITE.key -> LEGACY.toString) {
19661966
Seq(tsStr).toDF("tsS")
19671967
.select($"tsS".cast("timestamp").as("ts"))
19681968
.write
@@ -1993,7 +1993,7 @@ abstract class AvroSuite
19931993
test("SPARK-31183: rebasing dates in write") {
19941994
withTempPath { dir =>
19951995
val path = dir.getAbsolutePath
1996-
withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE.key -> LEGACY.toString) {
1996+
withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_WRITE.key -> LEGACY.toString) {
19971997
Seq("1001-01-01").toDF("dateS")
19981998
.select($"dateS".cast("date").as("date"))
19991999
.write.format("avro")
@@ -2040,14 +2040,14 @@ abstract class AvroSuite
20402040
.format("avro")
20412041
.save(dir.getAbsolutePath)
20422042
}
2043-
withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE.key -> LEGACY.toString) {
2043+
withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_WRITE.key -> LEGACY.toString) {
20442044
withTempPath { dir =>
20452045
saveTs(dir)
20462046
checkMetaData(dir, SPARK_LEGACY_DATETIME, "")
20472047
}
20482048
}
20492049
Seq(CORRECTED, EXCEPTION).foreach { mode =>
2050-
withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE.key -> mode.toString) {
2050+
withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_WRITE.key -> mode.toString) {
20512051
withTempPath { dir =>
20522052
saveTs(dir)
20532053
checkMetaData(dir, SPARK_LEGACY_DATETIME, null)

external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -204,7 +204,7 @@ object AvroReadBenchmark extends SqlBasedBenchmark {
204204
}
205205
withTempPath { path =>
206206
// Write and read timestamp in the LEGACY mode to make timestamp conversions more expensive
207-
withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE.key -> "LEGACY") {
207+
withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_WRITE.key -> "LEGACY") {
208208
spark.range(rowsNum).select(columns(): _*)
209209
.write
210210
.format("avro")
@@ -218,21 +218,21 @@ object AvroReadBenchmark extends SqlBasedBenchmark {
218218
}
219219

220220
benchmark.addCase("w/o filters", numIters) { _ =>
221-
withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key -> "LEGACY") {
221+
withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_READ.key -> "LEGACY") {
222222
readback.noop()
223223
}
224224
}
225225

226226
def withFilter(configEnabled: Boolean): Unit = {
227227
withSQLConf(
228-
SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key -> "LEGACY",
228+
SQLConf.AVRO_REBASE_MODE_IN_READ.key -> "LEGACY",
229229
SQLConf.AVRO_FILTER_PUSHDOWN_ENABLED.key -> configEnabled.toString()) {
230230
readback.filter($"key" === 0).noop()
231231
}
232232
}
233233

234234
benchmark.addCase("pushdown disabled", numIters) { _ =>
235-
withSQLConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key -> "LEGACY") {
235+
withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_READ.key -> "LEGACY") {
236236
withFilter(configEnabled = false)
237237
}
238238
}

python/pyspark/sql/readwriter.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -450,7 +450,7 @@ def parquet(self, *paths, **options):
450450
to Proleptic Gregorian calendar.
451451
452452
If None is set, the value of the SQL config
453-
``spark.sql.legacy.parquet.datetimeRebaseModeInRead`` is used by default.
453+
``spark.sql.parquet.datetimeRebaseModeInRead`` is used by default.
454454
int96RebaseMode : str, optional
455455
the rebasing mode for ``INT96`` timestamps from the Julian to
456456
Proleptic Gregorian calendar.
@@ -462,7 +462,7 @@ def parquet(self, *paths, **options):
462462
to Proleptic Gregorian calendar.
463463
464464
If None is set, the value of the SQL config
465-
``spark.sql.legacy.parquet.int96RebaseModeInRead`` is used by default.
465+
``spark.sql.parquet.int96RebaseModeInRead`` is used by default.
466466
467467
Examples
468468
--------

python/pyspark/sql/streaming.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -700,7 +700,7 @@ def parquet(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLook
700700
to Proleptic Gregorian calendar.
701701
702702
If None is set, the value of the SQL config
703-
``spark.sql.legacy.parquet.datetimeRebaseModeInRead`` is used by default.
703+
``spark.sql.parquet.datetimeRebaseModeInRead`` is used by default.
704704
int96RebaseMode : str, optional
705705
the rebasing mode for ``INT96`` timestamps from the Julian to
706706
Proleptic Gregorian calendar.
@@ -712,7 +712,7 @@ def parquet(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLook
712712
to Proleptic Gregorian calendar.
713713
714714
If None is set, the value of the SQL config
715-
``spark.sql.legacy.parquet.int96RebaseModeInRead`` is used by default.
715+
``spark.sql.parquet.int96RebaseModeInRead`` is used by default.
716716
717717
Examples
718718
--------

sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala

Lines changed: 33 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -2864,22 +2864,23 @@ object SQLConf {
28642864
.booleanConf
28652865
.createWithDefault(false)
28662866

2867-
val LEGACY_PARQUET_INT96_REBASE_MODE_IN_WRITE =
2868-
buildConf("spark.sql.legacy.parquet.int96RebaseModeInWrite")
2867+
val PARQUET_INT96_REBASE_MODE_IN_WRITE =
2868+
buildConf("spark.sql.parquet.int96RebaseModeInWrite")
28692869
.internal()
28702870
.doc("When LEGACY, Spark will rebase INT96 timestamps from Proleptic Gregorian calendar to " +
28712871
"the legacy hybrid (Julian + Gregorian) calendar when writing Parquet files. " +
28722872
"When CORRECTED, Spark will not do rebase and write the timestamps as it is. " +
28732873
"When EXCEPTION, which is the default, Spark will fail the writing if it sees ancient " +
28742874
"timestamps that are ambiguous between the two calendars.")
28752875
.version("3.1.0")
2876+
.withAlternative("spark.sql.legacy.parquet.int96RebaseModeInWrite")
28762877
.stringConf
28772878
.transform(_.toUpperCase(Locale.ROOT))
28782879
.checkValues(LegacyBehaviorPolicy.values.map(_.toString))
28792880
.createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString)
28802881

2881-
val LEGACY_PARQUET_REBASE_MODE_IN_WRITE =
2882-
buildConf("spark.sql.legacy.parquet.datetimeRebaseModeInWrite")
2882+
val PARQUET_REBASE_MODE_IN_WRITE =
2883+
buildConf("spark.sql.parquet.datetimeRebaseModeInWrite")
28832884
.internal()
28842885
.doc("When LEGACY, Spark will rebase dates/timestamps from Proleptic Gregorian calendar " +
28852886
"to the legacy hybrid (Julian + Gregorian) calendar when writing Parquet files. " +
@@ -2888,15 +2889,16 @@ object SQLConf {
28882889
"ancient dates/timestamps that are ambiguous between the two calendars. " +
28892890
"This config influences on writes of the following parquet logical types: DATE, " +
28902891
"TIMESTAMP_MILLIS, TIMESTAMP_MICROS. The INT96 type has the separate config: " +
2891-
s"${LEGACY_PARQUET_INT96_REBASE_MODE_IN_WRITE.key}.")
2892+
s"${PARQUET_INT96_REBASE_MODE_IN_WRITE.key}.")
28922893
.version("3.0.0")
2894+
.withAlternative("spark.sql.legacy.parquet.datetimeRebaseModeInWrite")
28932895
.stringConf
28942896
.transform(_.toUpperCase(Locale.ROOT))
28952897
.checkValues(LegacyBehaviorPolicy.values.map(_.toString))
28962898
.createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString)
28972899

2898-
val LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ =
2899-
buildConf("spark.sql.legacy.parquet.int96RebaseModeInRead")
2900+
val PARQUET_INT96_REBASE_MODE_IN_READ =
2901+
buildConf("spark.sql.parquet.int96RebaseModeInRead")
29002902
.internal()
29012903
.doc("When LEGACY, Spark will rebase INT96 timestamps from the legacy hybrid (Julian + " +
29022904
"Gregorian) calendar to Proleptic Gregorian calendar when reading Parquet files. " +
@@ -2905,13 +2907,14 @@ object SQLConf {
29052907
"timestamps that are ambiguous between the two calendars. This config is only effective " +
29062908
"if the writer info (like Spark, Hive) of the Parquet files is unknown.")
29072909
.version("3.1.0")
2910+
.withAlternative("spark.sql.legacy.parquet.int96RebaseModeInRead")
29082911
.stringConf
29092912
.transform(_.toUpperCase(Locale.ROOT))
29102913
.checkValues(LegacyBehaviorPolicy.values.map(_.toString))
29112914
.createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString)
29122915

2913-
val LEGACY_PARQUET_REBASE_MODE_IN_READ =
2914-
buildConf("spark.sql.legacy.parquet.datetimeRebaseModeInRead")
2916+
val PARQUET_REBASE_MODE_IN_READ =
2917+
buildConf("spark.sql.parquet.datetimeRebaseModeInRead")
29152918
.internal()
29162919
.doc("When LEGACY, Spark will rebase dates/timestamps from the legacy hybrid (Julian + " +
29172920
"Gregorian) calendar to Proleptic Gregorian calendar when reading Parquet files. " +
@@ -2921,29 +2924,31 @@ object SQLConf {
29212924
"only effective if the writer info (like Spark, Hive) of the Parquet files is unknown. " +
29222925
"This config influences on reads of the following parquet logical types: DATE, " +
29232926
"TIMESTAMP_MILLIS, TIMESTAMP_MICROS. The INT96 type has the separate config: " +
2924-
s"${LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ.key}.")
2927+
s"${PARQUET_INT96_REBASE_MODE_IN_READ.key}.")
29252928
.version("3.0.0")
2929+
.withAlternative("spark.sql.legacy.parquet.datetimeRebaseModeInRead")
29262930
.stringConf
29272931
.transform(_.toUpperCase(Locale.ROOT))
29282932
.checkValues(LegacyBehaviorPolicy.values.map(_.toString))
29292933
.createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString)
29302934

2931-
val LEGACY_AVRO_REBASE_MODE_IN_WRITE =
2932-
buildConf("spark.sql.legacy.avro.datetimeRebaseModeInWrite")
2935+
val AVRO_REBASE_MODE_IN_WRITE =
2936+
buildConf("spark.sql.avro.datetimeRebaseModeInWrite")
29332937
.internal()
29342938
.doc("When LEGACY, Spark will rebase dates/timestamps from Proleptic Gregorian calendar " +
29352939
"to the legacy hybrid (Julian + Gregorian) calendar when writing Avro files. " +
29362940
"When CORRECTED, Spark will not do rebase and write the dates/timestamps as it is. " +
29372941
"When EXCEPTION, which is the default, Spark will fail the writing if it sees " +
29382942
"ancient dates/timestamps that are ambiguous between the two calendars.")
29392943
.version("3.0.0")
2944+
.withAlternative("spark.sql.legacy.avro.datetimeRebaseModeInWrite")
29402945
.stringConf
29412946
.transform(_.toUpperCase(Locale.ROOT))
29422947
.checkValues(LegacyBehaviorPolicy.values.map(_.toString))
29432948
.createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString)
29442949

2945-
val LEGACY_AVRO_REBASE_MODE_IN_READ =
2946-
buildConf("spark.sql.legacy.avro.datetimeRebaseModeInRead")
2950+
val AVRO_REBASE_MODE_IN_READ =
2951+
buildConf("spark.sql.avro.datetimeRebaseModeInRead")
29472952
.internal()
29482953
.doc("When LEGACY, Spark will rebase dates/timestamps from the legacy hybrid (Julian + " +
29492954
"Gregorian) calendar to Proleptic Gregorian calendar when reading Avro files. " +
@@ -2952,6 +2957,7 @@ object SQLConf {
29522957
"ancient dates/timestamps that are ambiguous between the two calendars. This config is " +
29532958
"only effective if the writer info (like Spark, Hive) of the Avro files is unknown.")
29542959
.version("3.0.0")
2960+
.withAlternative("spark.sql.legacy.avro.datetimeRebaseModeInRead")
29552961
.stringConf
29562962
.transform(_.toUpperCase(Locale.ROOT))
29572963
.checkValues(LegacyBehaviorPolicy.values.map(_.toString))
@@ -3127,7 +3133,19 @@ object SQLConf {
31273133
DeprecatedConfig(CONVERT_CTAS.key, "3.1",
31283134
s"Set '${LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT.key}' to false instead."),
31293135
DeprecatedConfig("spark.sql.sources.schemaStringLengthThreshold", "3.2",
3130-
s"Use '${HIVE_TABLE_PROPERTY_LENGTH_THRESHOLD.key}' instead.")
3136+
s"Use '${HIVE_TABLE_PROPERTY_LENGTH_THRESHOLD.key}' instead."),
3137+
DeprecatedConfig(PARQUET_INT96_REBASE_MODE_IN_WRITE.alternatives.head, "3.2",
3138+
s"Use '${PARQUET_INT96_REBASE_MODE_IN_WRITE.key}' instead."),
3139+
DeprecatedConfig(PARQUET_INT96_REBASE_MODE_IN_READ.alternatives.head, "3.2",
3140+
s"Use '${PARQUET_INT96_REBASE_MODE_IN_READ.key}' instead."),
3141+
DeprecatedConfig(PARQUET_REBASE_MODE_IN_WRITE.alternatives.head, "3.2",
3142+
s"Use '${PARQUET_REBASE_MODE_IN_WRITE.key}' instead."),
3143+
DeprecatedConfig(PARQUET_REBASE_MODE_IN_READ.alternatives.head, "3.2",
3144+
s"Use '${PARQUET_REBASE_MODE_IN_READ.key}' instead."),
3145+
DeprecatedConfig(AVRO_REBASE_MODE_IN_WRITE.alternatives.head, "3.2",
3146+
s"Use '${AVRO_REBASE_MODE_IN_WRITE.key}' instead."),
3147+
DeprecatedConfig(AVRO_REBASE_MODE_IN_READ.alternatives.head, "3.2",
3148+
s"Use '${AVRO_REBASE_MODE_IN_READ.key}' instead.")
31313149
)
31323150

31333151
Map(configs.map { cfg => cfg.key -> cfg } : _*)

sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -826,7 +826,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
826826
* <li>`recursiveFileLookup`: recursively scan a directory for files. Using this option
827827
* disables partition discovery</li>
828828
* <li>`datetimeRebaseMode` (default is the value specified in the SQL config
829-
* `spark.sql.legacy.parquet.datetimeRebaseModeInRead`): the rebasing mode for the values
829+
* `spark.sql.parquet.datetimeRebaseModeInRead`): the rebasing mode for the values
830830
* of the `DATE`, `TIMESTAMP_MICROS`, `TIMESTAMP_MILLIS` logical types from the Julian to
831831
* Proleptic Gregorian calendar:
832832
* <ul>
@@ -838,7 +838,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
838838
* </ul>
839839
* </li>
840840
* <li>`int96RebaseMode` (default is the value specified in the SQL config
841-
* `spark.sql.legacy.parquet.int96RebaseModeInRead`): the rebasing mode for `INT96` timestamps
841+
* `spark.sql.parquet.int96RebaseModeInRead`): the rebasing mode for `INT96` timestamps
842842
* from the Julian to Proleptic Gregorian calendar:
843843
* <ul>
844844
* <li>`EXCEPTION` : Spark fails in reads of ancient `INT96` timestamps that are ambiguous

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -135,11 +135,11 @@ object DataSourceUtils {
135135
def newRebaseExceptionInRead(format: String): SparkUpgradeException = {
136136
val (config, option) = format match {
137137
case "Parquet INT96" =>
138-
(SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ.key, ParquetOptions.INT96_REBASE_MODE)
138+
(SQLConf.PARQUET_INT96_REBASE_MODE_IN_READ.key, ParquetOptions.INT96_REBASE_MODE)
139139
case "Parquet" =>
140-
(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ.key, ParquetOptions.DATETIME_REBASE_MODE)
140+
(SQLConf.PARQUET_REBASE_MODE_IN_READ.key, ParquetOptions.DATETIME_REBASE_MODE)
141141
case "Avro" =>
142-
(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ.key, "datetimeRebaseMode")
142+
(SQLConf.AVRO_REBASE_MODE_IN_READ.key, "datetimeRebaseMode")
143143
case _ => throw new IllegalStateException("unrecognized format " + format)
144144
}
145145
new SparkUpgradeException("3.0", "reading dates before 1582-10-15 or timestamps before " +
@@ -154,9 +154,9 @@ object DataSourceUtils {
154154

155155
def newRebaseExceptionInWrite(format: String): SparkUpgradeException = {
156156
val config = format match {
157-
case "Parquet INT96" => SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_WRITE.key
158-
case "Parquet" => SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE.key
159-
case "Avro" => SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE.key
157+
case "Parquet INT96" => SQLConf.PARQUET_INT96_REBASE_MODE_IN_WRITE.key
158+
case "Parquet" => SQLConf.PARQUET_REBASE_MODE_IN_WRITE.key
159+
case "Avro" => SQLConf.AVRO_REBASE_MODE_IN_WRITE.key
160160
case _ => throw new IllegalStateException("unrecognized format " + format)
161161
}
162162
new SparkUpgradeException("3.0", "writing dates before 1582-10-15 or timestamps before " +

0 commit comments

Comments
 (0)