This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 77413d443f23 [SPARK-51874][SQL][FOLLOW-UP] Revert ParquetOptions 
rebase methods to return string type
77413d443f23 is described below

commit 77413d443f23dd7a14194e516a12d2c959a357be
Author: Wenchen Fan <wenc...@databricks.com>
AuthorDate: Tue Aug 19 17:56:35 2025 +0800

    [SPARK-51874][SQL][FOLLOW-UP] Revert ParquetOptions rebase methods to 
return string type
    
    ### What changes were proposed in this pull request?
    
    This is a followup of https://github.com/apache/spark/pull/50674 . In that 
PR, we made it easier to define sql configs with enum values, and we also 
refactored some code to make things simpler.
    
    This PR reverts the API changes of `ParquetOptions`. Ideally we can change 
private APIs, but Parquet is a very popular format and there are third-party 
spark plugins that use Parquet related private APIs in Spark. We can of course 
ask these Spark plugins to update their code or add shim layers, but it's more 
friendly to avoid breaking certain private APIs if easy.
    
    ### Why are the changes needed?
    
    avoid breaking private APIs that used by Spark plugins, such as 
https://github.com/apache/hudi/blob/master/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala#L150
    
    ### Does this PR introduce _any_ user-facing change?
    
    no
    
    ### How was this patch tested?
    
    N/A
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    no
    
    Closes #52065 from cloud-fan/follow.
    
    Authored-by: Wenchen Fan <wenc...@databricks.com>
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>
---
 .../execution/datasources/parquet/ParquetFileFormat.scala   |  8 +++++---
 .../sql/execution/datasources/parquet/ParquetOptions.scala  | 13 ++++++-------
 .../v2/parquet/ParquetPartitionReaderFactory.scala          |  8 +++++---
 3 files changed, 16 insertions(+), 13 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
index 661be2b9cfa0..c56c947e3da5 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
@@ -44,7 +44,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils
 import org.apache.spark.sql.errors.QueryExecutionErrors
 import org.apache.spark.sql.execution.datasources._
 import org.apache.spark.sql.execution.vectorized.{ConstantColumnVector, 
OffHeapColumnVector, OnHeapColumnVector}
-import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf}
 import org.apache.spark.sql.sources._
 import org.apache.spark.sql.types._
 import org.apache.spark.util.{SerializableConfiguration, ThreadUtils}
@@ -182,8 +182,10 @@ class ParquetFileFormat
     val pushDownInFilterThreshold = 
sqlConf.parquetFilterPushDownInFilterThreshold
     val isCaseSensitive = sqlConf.caseSensitiveAnalysis
     val parquetOptions = new ParquetOptions(options, 
sparkSession.sessionState.conf)
-    val datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead
-    val int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead
+    val datetimeRebaseModeInRead = LegacyBehaviorPolicy.withName(
+      parquetOptions.datetimeRebaseModeInRead)
+    val int96RebaseModeInRead = LegacyBehaviorPolicy.withName(
+      parquetOptions.int96RebaseModeInRead)
 
     // Should always be set by FileSourceScanExec creating this.
     // Check conf before checking option, to allow working around an issue by 
changing conf.
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala
index eaedd99d8628..dd5669bda07c 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala
@@ -24,7 +24,7 @@ import org.apache.parquet.hadoop.ParquetOutputFormat
 import org.apache.spark.sql.catalyst.{DataSourceOptions, FileSourceOptions}
 import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
 import org.apache.spark.sql.errors.QueryExecutionErrors
-import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf}
+import org.apache.spark.sql.internal.SQLConf
 
 /**
  * Options for the Parquet data source.
@@ -74,16 +74,15 @@ class ParquetOptions(
   /**
    * The rebasing mode for the DATE and TIMESTAMP_MICROS, TIMESTAMP_MILLIS 
values in reads.
    */
-  def datetimeRebaseModeInRead: LegacyBehaviorPolicy.Value = parameters
+  def datetimeRebaseModeInRead: String = parameters
     .get(DATETIME_REBASE_MODE)
-    .map(LegacyBehaviorPolicy.withName)
-    .getOrElse(sqlConf.getConf(SQLConf.PARQUET_REBASE_MODE_IN_READ))
+    .getOrElse(sqlConf.getConf(SQLConf.PARQUET_REBASE_MODE_IN_READ).toString)
   /**
    * The rebasing mode for INT96 timestamp values in reads.
    */
-  def int96RebaseModeInRead: LegacyBehaviorPolicy.Value = parameters
-    .get(INT96_REBASE_MODE).map(LegacyBehaviorPolicy.withName)
-    .getOrElse(sqlConf.getConf(SQLConf.PARQUET_INT96_REBASE_MODE_IN_READ))
+  def int96RebaseModeInRead: String = parameters
+    .get(INT96_REBASE_MODE)
+    
.getOrElse(sqlConf.getConf(SQLConf.PARQUET_INT96_REBASE_MODE_IN_READ).toString)
 }
 
 
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala
index 70ae8068a03a..4674320e8498 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala
@@ -38,7 +38,7 @@ import org.apache.spark.sql.execution.WholeStageCodegenExec
 import org.apache.spark.sql.execution.datasources.{AggregatePushDownUtils, 
DataSourceUtils, PartitionedFile, RecordReaderIterator}
 import org.apache.spark.sql.execution.datasources.parquet._
 import org.apache.spark.sql.execution.datasources.v2._
-import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf}
 import org.apache.spark.sql.sources.Filter
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.sql.vectorized.ColumnarBatch
@@ -81,8 +81,10 @@ case class ParquetPartitionReaderFactory(
   private val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal
   private val pushDownStringPredicate = 
sqlConf.parquetFilterPushDownStringPredicate
   private val pushDownInFilterThreshold = 
sqlConf.parquetFilterPushDownInFilterThreshold
-  private val datetimeRebaseModeInRead = options.datetimeRebaseModeInRead
-  private val int96RebaseModeInRead = options.int96RebaseModeInRead
+  private val datetimeRebaseModeInRead = LegacyBehaviorPolicy.withName(
+    options.datetimeRebaseModeInRead)
+  private val int96RebaseModeInRead = LegacyBehaviorPolicy.withName(
+    options.int96RebaseModeInRead)
 
   private val parquetReaderCallback = new ParquetReaderCallback()
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to