Skip to content

Commit cfa6e69

Browse files
authored
fix: replace all #3311 references with specific issues in 3.5.8 diff (#3761)
- caseSensitive → #3760 (case-insensitive duplicate field detection) - SPARK-34212 decimals, schema mismatch error message, SPARK-45604 timestamp_ntz → #3720 (schema mismatch no error thrown)
1 parent 4c311d1 commit cfa6e69

1 file changed

Lines changed: 43 additions & 42 deletions

File tree

dev/diffs/3.5.8.diff

Lines changed: 43 additions & 42 deletions
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@ index bc00c448b80..82068d7a2eb 100644
5353
<!--
5454
This spark-tags test-dep is needed even though it isn't used in this module, otherwise testing-cmds that exclude
5555
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
56-
index 27ae10b3d59..78e69902dfd 100644
56+
index 27ae10b3d59..d12fb7c42c2 100644
5757
--- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
5858
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
5959
@@ -1353,6 +1353,14 @@ object SparkSession extends Logging {
@@ -71,15 +71,16 @@ index 27ae10b3d59..78e69902dfd 100644
7171
/**
7272
* Initialize extensions specified in [[StaticSQLConf]]. The classes will be applied to the
7373
* extensions passed into this function.
74-
@@ -1362,6 +1370,7 @@ object SparkSession extends Logging {
74+
@@ -1362,7 +1370,8 @@ object SparkSession extends Logging {
7575
extensions: SparkSessionExtensions): SparkSessionExtensions = {
7676
val extensionConfClassNames = sparkContext.getConf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS)
7777
.getOrElse(Seq.empty)
78-
+ val extensionClassNames = extensionConfClassNames ++ loadCometExtension(sparkContext)
7978
- extensionConfClassNames.foreach { extensionConfClassName =>
79+
+ val extensionClassNames = extensionConfClassNames ++ loadCometExtension(sparkContext)
8080
+ extensionClassNames.foreach { extensionConfClassName =>
8181
try {
8282
val extensionConfClass = Utils.classForName(extensionConfClassName)
83+
val extensionConf = extensionConfClass.getConstructor().newInstance()
8384
@@ -1396,4 +1405,12 @@ object SparkSession extends Logging {
8485
}
8586
}
@@ -517,7 +518,7 @@ index a206e97c353..79813d8e259 100644
517518

518519
test("SPARK-35884: Explain Formatted") {
519520
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala
520-
index 93275487f29..510e3087e0f 100644
521+
index 93275487f29..601cb6647fe 100644
521522
--- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala
522523
+++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala
523524
@@ -23,6 +23,7 @@ import java.nio.file.{Files, StandardOpenOption}
@@ -551,7 +552,7 @@ index 93275487f29..510e3087e0f 100644
551552
Seq("parquet", "orc").foreach { format =>
552553
- test(s"Spark native readers should respect spark.sql.caseSensitive - ${format}") {
553554
+ test(s"Spark native readers should respect spark.sql.caseSensitive - ${format}",
554-
+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) {
555+
+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3760")) {
555556
withTempDir { dir =>
556557
val tableName = s"spark_25132_${format}_native"
557558
val tableDir = dir.getCanonicalPath + s"/$tableName"
@@ -979,6 +980,37 @@ index 3cf2bfd17ab..49728c35c42 100644
979980
withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false",
980981
SQLConf.ANSI_ENABLED.key -> "true") {
981982
withTable("t") {
983+
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
984+
index 8b4ac474f87..3f79f20822f 100644
985+
--- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
986+
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
987+
@@ -223,6 +223,8 @@ class SparkSessionExtensionSuite extends SparkFunSuite with SQLHelper with Adapt
988+
withSession(extensions) { session =>
989+
session.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED, true)
990+
session.conf.set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1")
991+
+ // https://github.com/apache/datafusion-comet/issues/1197
992+
+ session.conf.set("spark.comet.enabled", false)
993+
assert(session.sessionState.columnarRules.contains(
994+
MyColumnarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule())))
995+
import session.sqlContext.implicits._
996+
@@ -281,6 +283,8 @@ class SparkSessionExtensionSuite extends SparkFunSuite with SQLHelper with Adapt
997+
}
998+
withSession(extensions) { session =>
999+
session.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED, enableAQE)
1000+
+ // https://github.com/apache/datafusion-comet/issues/1197
1001+
+ session.conf.set("spark.comet.enabled", false)
1002+
assert(session.sessionState.columnarRules.contains(
1003+
MyColumnarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule())))
1004+
import session.sqlContext.implicits._
1005+
@@ -319,6 +323,8 @@ class SparkSessionExtensionSuite extends SparkFunSuite with SQLHelper with Adapt
1006+
val session = SparkSession.builder()
1007+
.master("local[1]")
1008+
.config(COLUMN_BATCH_SIZE.key, 2)
1009+
+ // https://github.com/apache/datafusion-comet/issues/1197
1010+
+ .config("spark.comet.enabled", false)
1011+
.withExtensions { extensions =>
1012+
extensions.injectColumnar(session =>
1013+
MyColumnarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule())) }
9821014
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
9831015
index 04702201f82..5ee11f83ecf 100644
9841016
--- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
@@ -2190,7 +2222,7 @@ index 8ed9ef1630e..a865928c1b2 100644
21902222
checkAnswer(
21912223
// "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY.
21922224
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
2193-
index f6472ba3d9d..7f00caf5063 100644
2225+
index f6472ba3d9d..5ea2d938664 100644
21942226
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
21952227
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
21962228
@@ -185,7 +185,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS
@@ -2219,7 +2251,7 @@ index f6472ba3d9d..7f00caf5063 100644
22192251

22202252
- test("SPARK-34212 Parquet should read decimals correctly") {
22212253
+ test("SPARK-34212 Parquet should read decimals correctly",
2222-
+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) {
2254+
+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) {
22232255
def readParquet(schema: String, path: File): DataFrame = {
22242256
spark.read.schema(schema).parquet(path.toString)
22252257
}
@@ -2358,7 +2390,7 @@ index 5c0b7def039..151184bc98c 100644
23582390
assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size,
23592391
s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " +
23602392
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
2361-
index 3f47c5e506f..92a5eafec84 100644
2393+
index 3f47c5e506f..f1ce3194279 100644
23622394
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
23632395
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
23642396
@@ -27,6 +27,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName
@@ -2385,7 +2417,7 @@ index 3f47c5e506f..92a5eafec84 100644
23852417

23862418
- test("schema mismatch failure error message for parquet vectorized reader") {
23872419
+ test("schema mismatch failure error message for parquet vectorized reader",
2388-
+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) {
2420+
+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) {
23892421
withTempPath { dir =>
23902422
val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = true)
23912423
assert(e.getCause.isInstanceOf[SparkException])
@@ -2395,7 +2427,7 @@ index 3f47c5e506f..92a5eafec84 100644
23952427

23962428
- test("SPARK-45604: schema mismatch failure error on timestamp_ntz to array<timestamp_ntz>") {
23972429
+ test("SPARK-45604: schema mismatch failure error on timestamp_ntz to array<timestamp_ntz>",
2398-
+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) {
2430+
+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) {
23992431
import testImplicits._
24002432

24012433
withTempPath { dir =>
@@ -3272,38 +3304,7 @@ index f3be79f9022..b4b1ea8dbc4 100644
32723304
+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3312")) {
32733305

32743306
withTempDir { tempDir =>
3275-
3276-
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
3277-
index 8b4ac474..3f79f208 100644
3278-
--- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
3279-
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
3280-
@@ -223,6 +223,8 @@ class SparkSessionExtensionSuite extends SparkFunSuite with SQLHelper with Adapt
3281-
withSession(extensions) { session =>
3282-
session.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED, true)
3283-
session.conf.set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1")
3284-
+ // https://github.com/apache/datafusion-comet/issues/1197
3285-
+ session.conf.set("spark.comet.enabled", false)
3286-
assert(session.sessionState.columnarRules.contains(
3287-
MyColumnarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule())))
3288-
import session.sqlContext.implicits._
3289-
@@ -281,6 +283,8 @@ class SparkSessionExtensionSuite extends SparkFunSuite with SQLHelper with Adapt
3290-
}
3291-
withSession(extensions) { session =>
3292-
session.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED, enableAQE)
3293-
+ // https://github.com/apache/datafusion-comet/issues/1197
3294-
+ session.conf.set("spark.comet.enabled", false)
3295-
assert(session.sessionState.columnarRules.contains(
3296-
MyColumnarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule())))
3297-
import session.sqlContext.implicits._
3298-
@@ -319,6 +323,8 @@ class SparkSessionExtensionSuite extends SparkFunSuite with SQLHelper with Adapt
3299-
val session = SparkSession.builder()
3300-
.master("local[1]")
3301-
.config(COLUMN_BATCH_SIZE.key, 2)
3302-
+ // https://github.com/apache/datafusion-comet/issues/1197
3303-
+ .config("spark.comet.enabled", false)
3304-
.withExtensions { extensions =>
3305-
extensions.injectColumnar(session =>
3306-
MyColumnarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule())) }
3307+
33073308
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
33083309
index 6160c3e5f6c..0956d7d9edc 100644
33093310
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala

0 commit comments

Comments
 (0)