diff --git a/0003-Fix-correctness-issue-with-persist-using-StorageLevel.NONE-on-Dataset.patch b/0003-Fix-correctness-issue-with-persist-using-StorageLevel.NONE-on-Dataset.patch new file mode 100644 index 0000000000000000000000000000000000000000..45f0938a7630eb7e412d453a079ec74575ac50fa --- /dev/null +++ b/0003-Fix-correctness-issue-with-persist-using-StorageLevel.NONE-on-Dataset.patch @@ -0,0 +1,80 @@ +From 6468f96ea42f6efe42033507c4e26600b751bfcc Mon Sep 17 00:00:00 2001 +From: Emil Ejbyfeldt +Date: Thu, 5 Oct 2023 09:41:08 +0900 +Subject: [PATCH] [SPARK-45386][SQL][3.5] Fix correctness issue with persist + using StorageLevel.NONE on Dataset + +### What changes were proposed in this pull request? +Support for InMememoryTableScanExec in AQE was added in #39624, but this patch contained a bug when a Dataset is persisted using `StorageLevel.NONE`. Before that patch a query like: +``` +import org.apache.spark.storage.StorageLevel +spark.createDataset(Seq(1, 2)).persist(StorageLevel.NONE).count() +``` +would correctly return 2. But after that patch it incorrectly returns 0. This is because AQE incorrectly determines based on the runtime statistics that are collected here: +https://github.com/apache/spark/blob/eac5a8c7e6da94bb27e926fc9a681aed6582f7d3/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala#L294 +that the input is empty. The problem is that the action that should make sure the statistics are collected here +https://github.com/apache/spark/blob/eac5a8c7e6da94bb27e926fc9a681aed6582f7d3/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala#L285-L291 +never use the iterator and when we have `StorageLevel.NONE` the persisting will also not use the iterator and we will not gather the correct statistics. + +The proposed fix in the patch just make calling persist with StorageLevel.NONE a no-op. Changing the action since it always "emptied" the iterator would also work but seems like that would be unnecessary work in a lot of normal circumstances. + +### Why are the changes needed? +The current code has a correctness issue. + +### Does this PR introduce _any_ user-facing change? +Yes, fixes the correctness issue. + +### How was this patch tested? +New and existing unit tests. + +### Was this patch authored or co-authored using generative AI tooling? +No + +Closes #43213 from eejbyfeldt/SPARK-45386-branch-3.5. + +Authored-by: Emil Ejbyfeldt +Signed-off-by: Hyukjin Kwon +--- + .../scala/org/apache/spark/sql/execution/CacheManager.scala | 4 +++- + .../src/test/scala/org/apache/spark/sql/DatasetSuite.scala | 6 ++++++ + 2 files changed, 9 insertions(+), 1 deletion(-) + +diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +index 064819275e004..e906c74f8a5ee 100644 +--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala ++++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +@@ -113,7 +113,9 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { + planToCache: LogicalPlan, + tableName: Option[String], + storageLevel: StorageLevel): Unit = { +- if (lookupCachedData(planToCache).nonEmpty) { ++ if (storageLevel == StorageLevel.NONE) { ++ // Do nothing for StorageLevel.NONE since it will not actually cache any data. ++ } else if (lookupCachedData(planToCache).nonEmpty) { + logWarning("Asked to cache already cached data.") + } else { + val sessionWithConfigsOff = getOrCloneSessionWithConfigsOff(spark) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +index c967540541a5c..6d9c43f866a0c 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +@@ -45,6 +45,7 @@ import org.apache.spark.sql.functions._ + import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.test.SharedSparkSession + import org.apache.spark.sql.types._ ++import org.apache.spark.storage.StorageLevel + + case class TestDataPoint(x: Int, y: Double, s: String, t: TestDataPoint2) + case class TestDataPoint2(x: Int, s: String) +@@ -2535,6 +2536,11 @@ class DatasetSuite extends QueryTest + + checkDataset(ds.filter(f(col("_1"))), Tuple1(ValueClass(2))) + } ++ ++ test("SPARK-45386: persist with StorageLevel.NONE should give correct count") { ++ val ds = Seq(1, 2).toDS().persist(StorageLevel.NONE) ++ assert(ds.count() == 2) ++ } + } + + class DatasetLargeResultCollectingSuite extends QueryTest diff --git a/0004-CommandResultExec-to-override-iterator-methods-to-avoid-triggering-multiple-jobs.patch b/0004-CommandResultExec-to-override-iterator-methods-to-avoid-triggering-multiple-jobs.patch new file mode 100644 index 0000000000000000000000000000000000000000..110be67fc94e841dd5b2605599485837e28b1bbe --- /dev/null +++ b/0004-CommandResultExec-to-override-iterator-methods-to-avoid-triggering-multiple-jobs.patch @@ -0,0 +1,56 @@ +From 24f88b319c88bfe55e8b2b683193a85842bdad88 Mon Sep 17 00:00:00 2001 +From: yorksity +Date: Tue, 10 Oct 2023 14:36:23 +0800 +Subject: [PATCH] [SPARK-45205][SQL] CommandResultExec to override iterator + methods to avoid triggering multiple jobs +MIME-Version: 1.0 +Content-Type: text/plain; charset=UTF-8 +Content-Transfer-Encoding: 8bit + +### What changes were proposed in this pull request? + +After SPARK-35378 was changed, the execution of statements such as ‘show parititions test' became slower. The change point is that the execution process changes from ExecutedCommandEnec to CommandResultExec, but ExecutedCommandExec originally implemented the following method + +override def executeToIterator(): Iterator[InternalRow] = sideEffectResult.iterator + +CommandResultExec is not rewritten, so when the hasNext method is executed, a job process is created, resulting in increased time-consuming + +### Why are the changes needed? + +Improve performance when show partitions/tables. + +### Does this PR introduce _any_ user-facing change? + +No + +### How was this patch tested? + +Existing tests should cover this. + +### Was this patch authored or co-authored using generative AI tooling? + +No + +Closes #43270 from yorksity/SPARK-45205. + +Authored-by: yorksity +Signed-off-by: Wenchen Fan +(cherry picked from commit c9c99222e828d556552694dfb48c75bf0703a2c4) +Signed-off-by: Wenchen Fan +--- + .../org/apache/spark/sql/execution/CommandResultExec.scala | 2 ++ + 1 file changed, 2 insertions(+) + +diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CommandResultExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CommandResultExec.scala +index 5f38278d2dc67..45e3e41ab053d 100644 +--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CommandResultExec.scala ++++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CommandResultExec.scala +@@ -81,6 +81,8 @@ case class CommandResultExec( + unsafeRows + } + ++ override def executeToIterator(): Iterator[InternalRow] = unsafeRows.iterator ++ + override def executeTake(limit: Int): Array[InternalRow] = { + val taken = unsafeRows.take(limit) + longMetric("numOutputRows").add(taken.size) diff --git a/0005-Cache-Invalidation-Issue-with-JDBC-Table.patch b/0005-Cache-Invalidation-Issue-with-JDBC-Table.patch new file mode 100644 index 0000000000000000000000000000000000000000..d1328a8b95e5f679da227d158e29c8e21b3db6b0 --- /dev/null +++ b/0005-Cache-Invalidation-Issue-with-JDBC-Table.patch @@ -0,0 +1,98 @@ +From 81a7f8f184cd597208fcad72130354288a0c9f79 Mon Sep 17 00:00:00 2001 +From: liangyongyuan +Date: Tue, 10 Oct 2023 14:40:33 +0800 +Subject: [PATCH] [SPARK-45449][SQL] Cache Invalidation Issue with JDBC Table + +### What changes were proposed in this pull request? +Add an equals method to `JDBCOptions` that considers two instances equal if their `JDBCOptions.parameters` are the same. + +### Why are the changes needed? +We have identified a cache invalidation issue when caching JDBC tables in Spark SQL. The cached table is unexpectedly invalidated when queried, leading to a re-read from the JDBC table instead of retrieving data from the cache. +Example SQL: + +``` +CACHE TABLE cache_t SELECT * FROM mysql.test.test1; +SELECT * FROM cache_t; +``` +Expected Behavior: +The expectation is that querying the cached table (cache_t) should retrieve the result from the cache without re-evaluating the execution plan. + +Actual Behavior: +However, the cache is invalidated, and the content is re-read from the JDBC table. + +Root Cause: +The issue lies in the `CacheData` class, where the comparison involves `JDBCTable`. The `JDBCTable` is a case class: + +`case class JDBCTable(ident: Identifier, schema: StructType, jdbcOptions: JDBCOptions)` +The comparison of non-case class components, such as `jdbcOptions`, involves pointer comparison. This leads to unnecessary cache invalidation. + +### Does this PR introduce _any_ user-facing change? +No + +### How was this patch tested? +Add uts + +### Was this patch authored or co-authored using generative AI tooling? +No + +Closes #43258 from lyy-pineapple/spark-git-cache. + +Authored-by: liangyongyuan +Signed-off-by: Wenchen Fan +(cherry picked from commit d073f2d3e2f67a4b612e020a583e23dc1fa63aab) +Signed-off-by: Wenchen Fan +--- + .../execution/datasources/jdbc/JDBCOptions.scala | 8 ++++++++ + .../v2/jdbc/JDBCTableCatalogSuite.scala | 15 +++++++++++++++ + 2 files changed, 23 insertions(+) + +diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +index 268a65b81ff68..57651684070f7 100644 +--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala ++++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +@@ -239,6 +239,14 @@ class JDBCOptions( + .get(JDBC_PREFER_TIMESTAMP_NTZ) + .map(_.toBoolean) + .getOrElse(SQLConf.get.timestampType == TimestampNTZType) ++ ++ override def hashCode: Int = this.parameters.hashCode() ++ ++ override def equals(other: Any): Boolean = other match { ++ case otherOption: JDBCOptions => ++ otherOption.parameters.equals(this.parameters) ++ case _ => false ++ } + } + + class JdbcOptionsInWrite( +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala +index 6b85911dca773..eed64b873c451 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala +@@ -26,6 +26,7 @@ import org.apache.spark.sql.{AnalysisException, QueryTest, Row} + import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, TableAlreadyExistsException} + import org.apache.spark.sql.catalyst.parser.ParseException + import org.apache.spark.sql.catalyst.util.CharVarcharUtils ++import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec + import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.test.SharedSparkSession + import org.apache.spark.sql.types._ +@@ -512,4 +513,18 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { + assert(t.schema === replaced) + } + } ++ ++ test("SPARK-45449: Cache Invalidation Issue with JDBC Table") { ++ withTable("h2.test.cache_t") { ++ withConnection { conn => ++ conn.prepareStatement( ++ """CREATE TABLE "test"."cache_t" (id decimal(25) PRIMARY KEY NOT NULL, ++ |name TEXT(32) NOT NULL)""".stripMargin).executeUpdate() ++ } ++ sql("INSERT OVERWRITE h2.test.cache_t SELECT 1 AS id, 'a' AS name") ++ sql("CACHE TABLE t1 SELECT id, name FROM h2.test.cache_t") ++ val plan = sql("select * from t1").queryExecution.sparkPlan ++ assert(plan.isInstanceOf[InMemoryTableScanExec]) ++ } ++ } + } diff --git a/0006-Fix-flaky-WholeStageCodegenSparkSubmitSuite.patch b/0006-Fix-flaky-WholeStageCodegenSparkSubmitSuite.patch new file mode 100644 index 0000000000000000000000000000000000000000..bf9e2b31a5a2b2de1b169afd29ceac9f5741243f --- /dev/null +++ b/0006-Fix-flaky-WholeStageCodegenSparkSubmitSuite.patch @@ -0,0 +1,116 @@ +From 6a5747d66e53ed0d934cdd9ca5c9bd9fde6868e6 Mon Sep 17 00:00:00 2001 +From: Kent Yao +Date: Tue, 17 Oct 2023 22:19:18 +0800 +Subject: [PATCH] [SPARK-45568][TESTS] Fix flaky + WholeStageCodegenSparkSubmitSuite + +### What changes were proposed in this pull request? + +WholeStageCodegenSparkSubmitSuite is [flaky](https://github.com/apache/spark/actions/runs/6479534195/job/17593342589) because SHUFFLE_PARTITIONS(200) creates 200 reducers for one total core and improper stop progress causes executor launcher reties. The heavy load and reties might result in timeout test failures. + +### Why are the changes needed? + +CI robustness + +### Does this PR introduce _any_ user-facing change? + +no + +### How was this patch tested? + +existing WholeStageCodegenSparkSubmitSuite +### Was this patch authored or co-authored using generative AI tooling? + +no + +Closes #43394 from yaooqinn/SPARK-45568. + +Authored-by: Kent Yao +Signed-off-by: Kent Yao +(cherry picked from commit f00ec39542a5f9ac75d8c24f0f04a7be703c8d7c) +Signed-off-by: Kent Yao +--- + .../WholeStageCodegenSparkSubmitSuite.scala | 57 ++++++++++--------- + 1 file changed, 30 insertions(+), 27 deletions(-) + +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala +index e253de76221ad..69145d890fc19 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala +@@ -26,6 +26,7 @@ import org.apache.spark.deploy.SparkSubmitTestUtils + import org.apache.spark.internal.Logging + import org.apache.spark.sql.{QueryTest, Row, SparkSession} + import org.apache.spark.sql.functions.{array, col, count, lit} ++import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.types.IntegerType + import org.apache.spark.tags.ExtendedSQLTest + import org.apache.spark.unsafe.Platform +@@ -70,39 +71,41 @@ class WholeStageCodegenSparkSubmitSuite extends SparkSubmitTestUtils + + object WholeStageCodegenSparkSubmitSuite extends Assertions with Logging { + +- var spark: SparkSession = _ +- + def main(args: Array[String]): Unit = { + TestUtils.configTestLog4j2("INFO") + +- spark = SparkSession.builder().getOrCreate() ++ val spark = SparkSession.builder() ++ .config(SQLConf.SHUFFLE_PARTITIONS.key, "2") ++ .getOrCreate() ++ ++ try { ++ // Make sure the test is run where the driver and the executors uses different object layouts ++ val driverArrayHeaderSize = Platform.BYTE_ARRAY_OFFSET ++ val executorArrayHeaderSize = ++ spark.sparkContext.range(0, 1).map(_ => Platform.BYTE_ARRAY_OFFSET).collect().head ++ assert(driverArrayHeaderSize > executorArrayHeaderSize) + +- // Make sure the test is run where the driver and the executors uses different object layouts +- val driverArrayHeaderSize = Platform.BYTE_ARRAY_OFFSET +- val executorArrayHeaderSize = +- spark.sparkContext.range(0, 1).map(_ => Platform.BYTE_ARRAY_OFFSET).collect.head.toInt +- assert(driverArrayHeaderSize > executorArrayHeaderSize) ++ val df = spark.range(71773).select((col("id") % lit(10)).cast(IntegerType) as "v") ++ .groupBy(array(col("v"))).agg(count(col("*"))) ++ val plan = df.queryExecution.executedPlan ++ assert(plan.exists(_.isInstanceOf[WholeStageCodegenExec])) + +- val df = spark.range(71773).select((col("id") % lit(10)).cast(IntegerType) as "v") +- .groupBy(array(col("v"))).agg(count(col("*"))) +- val plan = df.queryExecution.executedPlan +- assert(plan.exists(_.isInstanceOf[WholeStageCodegenExec])) ++ val expectedAnswer = ++ Row(Array(0), 7178) :: ++ Row(Array(1), 7178) :: ++ Row(Array(2), 7178) :: ++ Row(Array(3), 7177) :: ++ Row(Array(4), 7177) :: ++ Row(Array(5), 7177) :: ++ Row(Array(6), 7177) :: ++ Row(Array(7), 7177) :: ++ Row(Array(8), 7177) :: ++ Row(Array(9), 7177) :: Nil + +- val expectedAnswer = +- Row(Array(0), 7178) :: +- Row(Array(1), 7178) :: +- Row(Array(2), 7178) :: +- Row(Array(3), 7177) :: +- Row(Array(4), 7177) :: +- Row(Array(5), 7177) :: +- Row(Array(6), 7177) :: +- Row(Array(7), 7177) :: +- Row(Array(8), 7177) :: +- Row(Array(9), 7177) :: Nil +- val result = df.collect +- QueryTest.sameRows(result.toSeq, expectedAnswer) match { +- case Some(errMsg) => fail(errMsg) +- case _ => ++ QueryTest.checkAnswer(df, expectedAnswer) ++ } finally { ++ spark.stop() + } ++ + } + } diff --git a/0007-Fix-subquery-execution-failure-with-TakeOrderedAndProjectExec.patch b/0007-Fix-subquery-execution-failure-with-TakeOrderedAndProjectExec.patch new file mode 100644 index 0000000000000000000000000000000000000000..5ea4bc50484ada03f668ef890b48bf04e0a8e347 --- /dev/null +++ b/0007-Fix-subquery-execution-failure-with-TakeOrderedAndProjectExec.patch @@ -0,0 +1,84 @@ +From f47b63c6a62fb6f1fd894f64736847719af7a199 Mon Sep 17 00:00:00 2001 +From: allisonwang-db +Date: Fri, 20 Oct 2023 08:36:42 +0800 +Subject: [PATCH] [SPARK-45584][SQL] Fix subquery execution failure with + TakeOrderedAndProjectExec + +This PR fixes a bug when there are subqueries in `TakeOrderedAndProjectExec`. The executeCollect method does not wait for subqueries to finish and it can result in IllegalArgumentException when executing a simple query. +For example this query: +``` +WITH t2 AS ( + SELECT * FROM t1 ORDER BY id +) +SELECT *, (SELECT COUNT(*) FROM t2) FROM t2 LIMIT 10 +``` +will fail with this error +``` + java.lang.IllegalArgumentException: requirement failed: Subquery subquery#242, [id=#109] has not finished +``` + +To fix a bug. + +No + +New unit test + +No + +Closes #43419 from allisonwang-db/spark-45584-subquery-failure. + +Authored-by: allisonwang-db +Signed-off-by: Wenchen Fan +(cherry picked from commit 8fd915ffaba1cc99813cc8d6d2a28688d7fae39b) +Signed-off-by: Wenchen Fan +--- + .../apache/spark/sql/execution/limit.scala | 2 +- + .../org/apache/spark/sql/SubquerySuite.scala | 24 +++++++++++++++++++ + 2 files changed, 25 insertions(+), 1 deletion(-) + +diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +index 877f6508d963f..77135d21a26ab 100644 +--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala ++++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +@@ -282,7 +282,7 @@ case class TakeOrderedAndProjectExec( + projectList.map(_.toAttribute) + } + +- override def executeCollect(): Array[InternalRow] = { ++ override def executeCollect(): Array[InternalRow] = executeQuery { + val orderingSatisfies = SortOrder.orderingSatisfies(child.outputOrdering, sortOrder) + val ord = new LazilyGeneratedOrdering(sortOrder, child.output) + val limited = if (orderingSatisfies) { +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 +index d235d2a15fea3..a7a0f6156cb1d 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +@@ -2712,4 +2712,28 @@ class SubquerySuite extends QueryTest + expected) + } + } ++ ++ test("SPARK-45584: subquery execution should not fail with ORDER BY and LIMIT") { ++ withTable("t1") { ++ sql( ++ """ ++ |CREATE TABLE t1 USING PARQUET ++ |AS SELECT * FROM VALUES ++ |(1, "a"), ++ |(2, "a"), ++ |(3, "a") t(id, value) ++ |""".stripMargin) ++ val df = sql( ++ """ ++ |WITH t2 AS ( ++ | SELECT * FROM t1 ORDER BY id ++ |) ++ |SELECT *, (SELECT COUNT(*) FROM t2) FROM t2 LIMIT 10 ++ |""".stripMargin) ++ // This should not fail with IllegalArgumentException. ++ checkAnswer( ++ df, ++ Row(1, "a", 3) :: Row(2, "a", 3) :: Row(3, "a", 3) :: Nil) ++ } ++ } + } diff --git a/spark.spec b/spark.spec index 08d9f0bdefc04b854681099223fd66ca06831395..37e8fe19431953f5d29c279748a3324cc9f79108 100644 --- a/spark.spec +++ b/spark.spec @@ -4,7 +4,7 @@ Summary: A unified analytics engine for large-scale data processing. Name: spark Version: 3.5.0 -Release: 4 +Release: 5 License: Apache 2.0 URL: http://spark.apache.org/ Source0: https://github.com/apache/spark/archive/v%{version}.tar.gz @@ -17,6 +17,12 @@ Source6: https://github.com/grpc/grpc-java/archive/refs/tags/v1.56.0.tar.gz Patch0001: 0001-change-mvn-scalafmt.patch Patch0002: 0002-Upgrade-os-maven-plugin-to-1.7.1.patch +Patch0003: 0003-Fix-correctness-issue-with-persist-using-StorageLevel.NONE-on-Dataset.patch +Patch0004: 0004-CommandResultExec-to-override-iterator-methods-to-avoid-triggering-multiple-jobs.patch +Patch0005: 0005-Cache-Invalidation-Issue-with-JDBC-Table.patch +Patch0006: 0006-Fix-flaky-WholeStageCodegenSparkSubmitSuite.patch +Patch0007: 0007-Fix-subquery-execution-failure-with-TakeOrderedAndProjectExec.patch + %ifarch riscv64 BuildRequires: protobuf-devel protobuf-compiler BuildRequires: autoconf automake libtool pkgconfig zlib-devel libstdc++-static gcc-c++ @@ -76,6 +82,11 @@ popd %patch0001 -p1 %patch0002 -p1 +%patch0003 -p1 +%patch0004 -p1 +%patch0005 -p1 +%patch0006 -p1 +%patch0007 -p1 %ifarch riscv64 sed -i -e 's/protoVersion = "3.23.4/protoVersion = "'${PROTOC_VERSION}/'' project/SparkBuild.scala @@ -97,6 +108,13 @@ cp -rf ../%{name}-%{version} %{buildroot}/opt/apache-%{name}-%{version} %changelog +* Fri Aug 16 2024 zhangxingrong - 3.5.0-5 +- Fix correctness issue with persist using StorageLevel.NONE on Dataset +- CommandResultExec to override iterator methods to avoid triggering multiple jobs +- Cache Invalidation Issue with JDBC Table +- Fix flaky WholeStageCodegenSparkSubmitSuite +- Fix subquery execution failure with TakeOrderedAndProjectExec + * Mon Jul 1 2024 Dingli Zhang - 3.5.0-4 - Add riscv64 to ExclusiveArch - Fix build on riscv64