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

chengchengjin pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
     new ccecbe5019 [GLUTEN-11088][VL] Support test Spark-4.0 in intellij 
(#11198)
ccecbe5019 is described below

commit ccecbe5019692bd57528ddbaa3cfa35094c81702
Author: Jin Chengcheng <[email protected]>
AuthorDate: Thu Nov 27 10:56:42 2025 +0000

    [GLUTEN-11088][VL] Support test Spark-4.0 in intellij (#11198)
---
 pom.xml                                               |  6 +++---
 .../gluten/sql/shims/spark40/Spark40Shims.scala       | 19 ++++++++++++++++---
 2 files changed, 19 insertions(+), 6 deletions(-)

diff --git a/pom.xml b/pom.xml
index a9f6c2657f..65ab455687 100644
--- a/pom.xml
+++ b/pom.xml
@@ -935,7 +935,7 @@
     <profile>
       <id>scala-2.13</id>
       <properties>
-        <scala.version>2.13.8</scala.version>
+        <scala.version>2.13.16</scala.version>
         <scala.binary.version>2.13</scala.binary.version>
         <spotless.scalafmt.version>3.8.3</spotless.scalafmt.version>
       </properties>
@@ -950,7 +950,7 @@
                   <compilerPlugin>
                     <groupId>org.wartremover</groupId>
                     
<artifactId>wartremover_${scala.binary.version}</artifactId>
-                    <version>3.0.6</version>
+                    <version>3.2.0</version>
                   </compilerPlugin>
                 </compilerPlugins>
                 <dependencies>
@@ -966,7 +966,7 @@
                   <arg>-deprecation</arg>
                   <arg>-feature</arg>
                   <arg>-explaintypes</arg>
-                  <arg>-target:jvm-1.8</arg>
+                  <arg>-release:8</arg>
                   <arg>-Wconf:cat=deprecation:wv,any:e</arg>
                   <arg>-Wunused:imports</arg>
                   <!--
diff --git 
a/shims/spark40/src/main/scala/org/apache/gluten/sql/shims/spark40/Spark40Shims.scala
 
b/shims/spark40/src/main/scala/org/apache/gluten/sql/shims/spark40/Spark40Shims.scala
index 85b057ebaf..9077fe5abc 100644
--- 
a/shims/spark40/src/main/scala/org/apache/gluten/sql/shims/spark40/Spark40Shims.scala
+++ 
b/shims/spark40/src/main/scala/org/apache/gluten/sql/shims/spark40/Spark40Shims.scala
@@ -51,18 +51,20 @@ import 
org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV
 import org.apache.spark.sql.execution.datasources.v2.text.TextScan
 import org.apache.spark.sql.execution.datasources.v2.utils.CatalogUtil
 import org.apache.spark.sql.execution.exchange.{BroadcastExchangeLike, 
ShuffleExchangeLike}
-import org.apache.spark.sql.execution.window.{Final, GlutenFinal, 
GlutenPartial, Partial, WindowGroupLimitExec, WindowGroupLimitExecShim}
+import org.apache.spark.sql.execution.window.{Final, Partial, _}
 import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf}
-import org.apache.spark.sql.types.{DecimalType, IntegerType, LongType, 
StructField, StructType}
+import org.apache.spark.sql.types._
 import org.apache.spark.sql.util.CaseInsensitiveStringMap
 import org.apache.spark.storage.{BlockId, BlockManagerId}
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{FileStatus, LocatedFileStatus, Path}
+import org.apache.parquet.HadoopReadOptions
 import org.apache.parquet.crypto.ParquetCryptoRuntimeException
 import org.apache.parquet.format.converter.ParquetMetadataConverter
 import org.apache.parquet.hadoop.ParquetFileReader
 import org.apache.parquet.hadoop.metadata.FileMetaData.EncryptionType
+import org.apache.parquet.hadoop.util.HadoopInputFile
 import org.apache.parquet.schema.MessageType
 
 import java.time.ZoneOffset
@@ -672,9 +674,16 @@ class Spark40Shims extends SparkShims {
   override def isParquetFileEncrypted(
       fileStatus: LocatedFileStatus,
       conf: Configuration): Boolean = {
+    val file = HadoopInputFile.fromPath(fileStatus.getPath, conf)
+    val filter = ParquetMetadataConverter.NO_FILTER
+    val options = HadoopReadOptions
+      .builder(file.getConfiguration, file.getPath)
+      .withMetadataFilter(filter)
+      .build
+    val in = file.newStream
     try {
       val footer =
-        ParquetFileReader.readFooter(conf, fileStatus.getPath, 
ParquetMetadataConverter.NO_FILTER)
+        ParquetFileReader.readFooter(file, options, in)
       val fileMetaData = footer.getFileMetaData
       fileMetaData.getEncryptionType match {
         // UNENCRYPTED file has a plaintext footer and no file encryption,
@@ -693,6 +702,10 @@ class Spark40Shims extends SparkShims {
       case e: Exception if ExceptionUtils.hasCause(e, 
classOf[ParquetCryptoRuntimeException]) =>
         true
       case e: Exception => false
+    } finally {
+      if (in != null) {
+        in.close()
+      }
     }
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to