diff --git a/gluten-ut/spark34/pom.xml b/gluten-ut/spark34/pom.xml
index b7f20843387c..b170b89b9102 100644
--- a/gluten-ut/spark34/pom.xml
+++ b/gluten-ut/spark34/pom.xml
@@ -118,8 +118,23 @@
+
+
+ org.slf4j
+ slf4j-api
+ ${slf4j.version}
+ test
+
+
+ org.apache.logging.log4j
+ log4j-slf4j2-impl
+ ${log4j.version}
+ test
+
+ 2.0.6
+ 2.19.0
diff --git a/gluten-ut/spark34/src/test/scala/io/glutenproject/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark34/src/test/scala/io/glutenproject/utils/velox/VeloxTestSettings.scala
index 637b00175a5e..177eccc3f3f6 100644
--- a/gluten-ut/spark34/src/test/scala/io/glutenproject/utils/velox/VeloxTestSettings.scala
+++ b/gluten-ut/spark34/src/test/scala/io/glutenproject/utils/velox/VeloxTestSettings.scala
@@ -114,7 +114,6 @@ class VeloxTestSettings extends BackendTestSettings {
"INCONSISTENT_BEHAVIOR_CROSS_VERSION: compatibility with Spark 2.4/3.2 in reading/writing dates")
.exclude("FAILED_EXECUTE_UDF: execute user defined function")
.exclude("UNRECOGNIZED_SQL_TYPE: unrecognized SQL type -100")
- .exclude("INVALID_BUCKET_FILE: error if there exists any malformed bucket files")
.excludeByPrefix("SCALAR_SUBQUERY_TOO_MANY_ROWS:")
.excludeByPrefix("UNSUPPORTED_FEATURE.MULTI_ACTION_ALTER:")
enableSuite[GlutenQueryParsingErrorsSuite]
@@ -183,7 +182,7 @@ class VeloxTestSettings extends BackendTestSettings {
.includeByPrefix(
"gluten",
"SPARK-29906",
-// "SPARK-30291",
+ "SPARK-30291",
"SPARK-30403",
"SPARK-30719",
"SPARK-31384",
@@ -196,7 +195,7 @@ class VeloxTestSettings extends BackendTestSettings {
"SPARK-35585",
"SPARK-32932",
"SPARK-33494",
-// "SPARK-33933",
+ "SPARK-33933",
"SPARK-31220",
"SPARK-35874",
"SPARK-39551"
@@ -886,9 +885,6 @@ class VeloxTestSettings extends BackendTestSettings {
enableSuite[GlutenVectorizedParquetReadSchemaSuite]
enableSuite[GlutenMergedParquetReadSchemaSuite]
enableSuite[GlutenEnsureRequirementsSuite]
- // Rewrite to change the shuffle partitions for optimizing repartition
- .excludeByPrefix("SPARK-35675")
- .exclude("SPARK-41986: Introduce shuffle on SinglePartition")
// enableSuite[GlutenBroadcastJoinSuite]
// .exclude("Shouldn't change broadcast join buildSide if user clearly specified")
@@ -987,7 +983,6 @@ class VeloxTestSettings extends BackendTestSettings {
enableSuite[GlutenCountMinSketchAggQuerySuite]
enableSuite[GlutenCsvFunctionsSuite]
enableSuite[GlutenCTEHintSuite]
- .exclude("Resolve join hint in CTE")
enableSuite[GlutenCTEInlineSuiteAEOff]
enableSuite[GlutenCTEInlineSuiteAEOn]
enableSuite[GlutenDataFrameAggregateSuite]
diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/adaptive/GlutenAdaptiveQueryExecSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/adaptive/GlutenAdaptiveQueryExecSuite.scala
index 0d45df3a5ada..6bbeb801825d 100644
--- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/adaptive/GlutenAdaptiveQueryExecSuite.scala
+++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/adaptive/GlutenAdaptiveQueryExecSuite.scala
@@ -815,7 +815,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL
}
}
- ignore("gluten Logging plan changes for AQE") {
+ test("gluten Logging plan changes for AQE") {
val testAppender = new LogAppender("plan changes")
withLogAppender(testAppender) {
withSQLConf(
@@ -1450,7 +1450,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL
}
}
- ignore("gluten test log level") {
+ test("gluten test log level") {
def verifyLog(expectedLevel: Level): Unit = {
val logAppender = new LogAppender("adaptive execution")
logAppender.setThreshold(expectedLevel)
diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenEnsureRequirementsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenEnsureRequirementsSuite.scala
index 7581f0e9d371..1409b598ac63 100644
--- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenEnsureRequirementsSuite.scala
+++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenEnsureRequirementsSuite.scala
@@ -18,8 +18,6 @@ package org.apache.spark.sql.execution.exchange
import org.apache.spark.SparkConf
import org.apache.spark.sql.GlutenSQLTestsBaseTrait
-import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST
-import org.apache.spark.sql.internal.SQLConf
class GlutenEnsureRequirementsSuite extends EnsureRequirementsSuite with GlutenSQLTestsBaseTrait {
override def sparkConf: SparkConf = {
@@ -27,19 +25,4 @@ class GlutenEnsureRequirementsSuite extends EnsureRequirementsSuite with GlutenS
super.sparkConf
.set("spark.sql.shuffle.partitions", "5")
}
-
- test(
- GLUTEN_TEST +
- "SPARK-35675: EnsureRequirements remove shuffle should respect PartitioningCollection") {
- import testImplicits._
- withSQLConf(
- SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1",
- SQLConf.SHUFFLE_PARTITIONS.key -> "5",
- SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") {
- val df1 = Seq((1, 2)).toDF("c1", "c2")
- val df2 = Seq((1, 3)).toDF("c3", "c4")
- val res = df1.join(df2, $"c1" === $"c3").repartition($"c1")
- assert(res.queryExecution.executedPlan.collect { case s: ShuffleExchangeLike => s }.size == 2)
- }
- }
}
diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala
index 3321775fe364..35e477c8df3c 100644
--- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala
+++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala
@@ -30,7 +30,7 @@ import scala.collection.mutable.ArrayBuffer
class GlutenFallbackSuite extends GlutenSQLTestsTrait with AdaptiveSparkPlanHelper {
- ignore("test fallback logging") {
+ test("test fallback logging") {
val testAppender = new LogAppender("fallback reason")
withLogAppender(testAppender) {
withSQLConf(
@@ -41,13 +41,14 @@ class GlutenFallbackSuite extends GlutenSQLTestsTrait with AdaptiveSparkPlanHelp
sql("SELECT * FROM t").collect()
}
}
- val msgRegex = """Validation failed for plan: Scan parquet default\.t\[QueryId=[0-9]+\],""" +
- """ due to: columnar FileScan is not enabled in FileSourceScanExec\."""
+ val msgRegex =
+ """Validation failed for plan: Scan parquet spark_catalog.default\.t\[QueryId=[0-9]+\],""" +
+ """ due to: columnar FileScan is not enabled in FileSourceScanExec\."""
assert(testAppender.loggingEvents.exists(_.getMessage.getFormattedMessage.matches(msgRegex)))
}
}
- ignore("test fallback event") {
+ test("test fallback event") {
val kvStore = spark.sparkContext.statusStore.store.asInstanceOf[ElementTrackingStore]
val glutenStore = new GlutenSQLAppStatusStore(kvStore)
assert(glutenStore.buildInfo().info.find(_._1 == "Gluten Version").exists(_._2 == VERSION))
@@ -77,7 +78,7 @@ class GlutenFallbackSuite extends GlutenSQLTestsTrait with AdaptiveSparkPlanHelp
val id = runExecution("SELECT * FROM t")
val execution = glutenStore.execution(id)
assert(execution.isDefined)
- assert(execution.get.numGlutenNodes == 2)
+ assert(execution.get.numGlutenNodes == 1)
assert(execution.get.numFallbackNodes == 0)
assert(execution.get.fallbackNodeToReason.isEmpty)
@@ -86,9 +87,9 @@ class GlutenFallbackSuite extends GlutenSQLTestsTrait with AdaptiveSparkPlanHelp
val execution = glutenStore.execution(id)
assert(execution.isDefined)
assert(execution.get.numGlutenNodes == 0)
- assert(execution.get.numFallbackNodes == 2)
+ assert(execution.get.numFallbackNodes == 1)
val fallbackReason = execution.get.fallbackNodeToReason.head
- assert(fallbackReason._1.contains("Scan parquet default.t"))
+ assert(fallbackReason._1.contains("Scan parquet spark_catalog.default.t"))
assert(fallbackReason._2.contains("columnar FileScan is not enabled in FileSourceScanExec"))
}
}
diff --git a/shims/spark34/src/main/scala/io/glutenproject/sql/shims/spark34/Spark34Shims.scala b/shims/spark34/src/main/scala/io/glutenproject/sql/shims/spark34/Spark34Shims.scala
index e23888e38a82..b9bfba168a42 100644
--- a/shims/spark34/src/main/scala/io/glutenproject/sql/shims/spark34/Spark34Shims.scala
+++ b/shims/spark34/src/main/scala/io/glutenproject/sql/shims/spark34/Spark34Shims.scala
@@ -116,7 +116,7 @@ class Spark34Shims extends SparkShims {
f =>
BucketingUtils
.getBucketId(f.toPath.getName)
- .getOrElse(throw invalidBucketFile(f.toPath.getName))
+ .getOrElse(throw invalidBucketFile(f.urlEncodedPath))
}
}
@@ -148,10 +148,11 @@ class Spark34Shims extends SparkShims {
}
}
+ // https://issues.apache.org/jira/browse/SPARK-40400
private def invalidBucketFile(path: String): Throwable = {
new SparkException(
errorClass = "INVALID_BUCKET_FILE",
- messageParameters = Map("error" -> path),
+ messageParameters = Map("path" -> path),
cause = null)
}
}