From 60edff9d4df0387e20205016b058310d3e2d5818 Mon Sep 17 00:00:00 2001
From: jlf <1251489546@qq.com>
Date: Wed, 12 Mar 2025 14:38:51 +0800
Subject: [PATCH 1/2] Fix UT
1. move iceberg version in profiles
2. ignore UT for CH backend
3. exclude jdk.tools in pom
4. ignore spark warehouse in gluten-ut
---
.gitignore | 3 +++
backends-clickhouse/pom.xml | 4 ++++
.../utils/clickhouse/ClickHouseTestSettings.scala | 10 ++++++++++
pom.xml | 6 ++++--
4 files changed, 21 insertions(+), 2 deletions(-)
diff --git a/.gitignore b/.gitignore
index f402cb1e83e0..ae86affe7af3 100644
--- a/.gitignore
+++ b/.gitignore
@@ -100,3 +100,6 @@ dist/
metastore_db/
.ipynb_checkpoints
+
+# For Spark warehouse
+gluten-ut/*/spark-warehouse/
\ No newline at end of file
diff --git a/backends-clickhouse/pom.xml b/backends-clickhouse/pom.xml
index e66813f3f157..6d0773554e45 100644
--- a/backends-clickhouse/pom.xml
+++ b/backends-clickhouse/pom.xml
@@ -273,6 +273,10 @@
protobuf-java
com.google.protobuf
+
+ jdk.tools
+ jdk.tools
+
diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala
index 5c4415356f9a..44cc6ed172df 100644
--- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala
+++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala
@@ -504,6 +504,8 @@ class ClickHouseTestSettings extends BackendTestSettings {
"session window groupBy with multiple keys statement - keys overlapped with sessions")
.excludeCH("SPARK-36465: filter out events with negative/zero gap duration")
.excludeCH("SPARK-36724: Support timestamp_ntz as a type of time column for SessionWindow")
+ .excludeCH(
+ "SPARK-49836 using window fn with window as parameter should preserve parent operator")
enableSuite[GlutenDataFrameSetOperationsSuite]
.exclude("SPARK-37371: UnionExec should support columnar if all children support columnar")
// Result depends on the implementation for nondeterministic expression rand.
@@ -1659,6 +1661,10 @@ class ClickHouseTestSettings extends BackendTestSettings {
.excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen off)")
.excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen on)")
.excludeCH("SPARK-32717: AQEOptimizer should respect excludedRules configuration")
+ .excludeCH("SPARK-46037: ShuffledHashJoin build left with left outer join, "
+ + "codegen off (whole-stage-codegen off)")
+ .excludeCH("SPARK-46037: ShuffledHashJoin build left with left outer join, "
+ + "codegen off (whole-stage-codegen on)")
enableSuite[GlutenOuterJoinSuiteForceShjOn]
.excludeCH("basic left outer join using ShuffledHashJoin (whole-stage-codegen off)")
.excludeCH("basic left outer join using ShuffledHashJoin (whole-stage-codegen on)")
@@ -1684,6 +1690,10 @@ class ClickHouseTestSettings extends BackendTestSettings {
.excludeCH("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)")
.excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen off)")
.excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen on)")
+ .excludeCH("SPARK-46037: ShuffledHashJoin build left with left outer join, "
+ + "codegen off (whole-stage-codegen off)")
+ .excludeCH("SPARK-46037: ShuffledHashJoin build left with left outer join, "
+ + "codegen off (whole-stage-codegen on)")
enableSuite[GlutenParametersSuite]
enableSuite[GlutenParquetCodecSuite]
// codec not supported in native
diff --git a/pom.xml b/pom.xml
index 9d58ab30e5f9..444700388203 100644
--- a/pom.xml
+++ b/pom.xml
@@ -278,6 +278,7 @@
1.8
+ 1.5.0
@@ -288,6 +289,7 @@
11
3.1.8
+ 1.8.0
@@ -298,6 +300,7 @@
17
3.1.8
+ 1.8.0
@@ -347,8 +350,7 @@
3.5
spark-sql-columnar-shims-spark35
- 3.5.2
- 1.5.0
+ 3.5.5
delta-spark
3.2.0
32
From 506c725bde721498d2156b1c0d23c4b9c7b057ef Mon Sep 17 00:00:00 2001
From: jlf <1251489546@qq.com>
Date: Tue, 18 Mar 2025 10:39:59 +0800
Subject: [PATCH 2/2] Fix build error
---
.../gluten/sql/shims/spark35/Spark35Shims.scala | 4 +++-
.../sql/shims/spark35/SparkShimProvider.scala | 2 +-
.../sql/execution/AbstractFileSourceScanExec.scala | 13 +++++++------
3 files changed, 11 insertions(+), 8 deletions(-)
diff --git a/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/Spark35Shims.scala b/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/Spark35Shims.scala
index 2203e08d3ed0..a8686f0a7b04 100644
--- a/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/Spark35Shims.scala
+++ b/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/Spark35Shims.scala
@@ -149,7 +149,8 @@ class Spark35Shims extends SparkShims {
override def filesGroupedToBuckets(
selectedPartitions: Array[PartitionDirectory]): Map[Int, Array[PartitionedFile]] = {
selectedPartitions
- .flatMap(p => p.files.map(f => PartitionedFileUtil.getPartitionedFile(f, p.values)))
+ .flatMap(
+ p => p.files.map(f => PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values)))
.groupBy {
f =>
BucketingUtils
@@ -418,6 +419,7 @@ class Spark35Shims extends SparkShims {
PartitionedFileUtil.splitFiles(
sparkSession,
FileStatusWithMetadata(file, metadata),
+ filePath,
isSplitable,
maxSplitBytes,
partitionValues)
diff --git a/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala b/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala
index eab32ab9d0b9..35049a531386 100644
--- a/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala
+++ b/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala
@@ -20,7 +20,7 @@ import org.apache.gluten.sql.shims.{SparkShimDescriptor, SparkShims}
import org.apache.gluten.sql.shims.spark35.SparkShimProvider.DESCRIPTOR
object SparkShimProvider {
- val DESCRIPTOR = SparkShimDescriptor(3, 5, 2)
+ val DESCRIPTOR = SparkShimDescriptor(3, 5, 5)
}
class SparkShimProvider extends org.apache.gluten.sql.shims.SparkShimProvider {
diff --git a/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala b/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala
index a83c763c4566..07232b11cdae 100644
--- a/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala
+++ b/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala
@@ -181,7 +181,8 @@ abstract class AbstractFileSourceScanExec(
logInfo(s"Planning with ${bucketSpec.numBuckets} buckets")
val filesGroupedToBuckets =
selectedPartitions
- .flatMap(p => p.files.map(f => PartitionedFileUtil.getPartitionedFile(f, p.values)))
+ .flatMap(
+ p => p.files.map(f => PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values)))
.groupBy {
f =>
BucketingUtils
@@ -264,14 +265,14 @@ abstract class AbstractFileSourceScanExec(
partition =>
partition.files.flatMap {
file =>
- if (shouldProcess(file.getPath)) {
- val isSplitable = relation.fileFormat.isSplitable(
- relation.sparkSession,
- relation.options,
- file.getPath)
+ val filePath = file.getPath
+ if (shouldProcess(filePath)) {
+ val isSplitable =
+ relation.fileFormat.isSplitable(relation.sparkSession, relation.options, filePath)
PartitionedFileUtil.splitFiles(
sparkSession = relation.sparkSession,
file = file,
+ filePath,
isSplitable = isSplitable,
maxSplitBytes = maxSplitBytes,
partitionValues = partition.values