(spark) branch master updated: [SPARK-48974][SQL][SS][ML][MLLIB] Use `SparkSession.implicits` instead of `SQLContext.implicits`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 877c3f2bea92 [SPARK-48974][SQL][SS][ML][MLLIB] Use `SparkSession.implicits` instead of `SQLContext.implicits` 877c3f2bea92 is described below commit 877c3f2bea924ca9f3fd5b7e9c6cbfb0fc3be958 Author: yangjie01 AuthorDate: Wed Jul 24 10:41:07 2024 +0800 [SPARK-48974][SQL][SS][ML][MLLIB] Use `SparkSession.implicits` instead of `SQLContext.implicits` ### What changes were proposed in this pull request? This PR replaces `SQLContext.implicits` with `SparkSession.implicits` in the Spark codebase. ### Why are the changes needed? Reduce the usage of code from `SQLContext` within the internal code of Spark. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #47457 from LuciferYang/use-sparksession-implicits. Lead-authored-by: yangjie01 Co-authored-by: YangJie Signed-off-by: yangjie01 --- .../src/main/scala/org/apache/spark/mllib/util/MLUtils.scala | 2 +- .../apache/spark/ml/classification/FMClassifierSuite.scala | 4 ++-- .../spark/ml/classification/LogisticRegressionSuite.scala| 12 ++-- .../apache/spark/ml/recommendation/CollectTopKSuite.scala| 4 ++-- .../apache/spark/ml/regression/LinearRegressionSuite.scala | 4 ++-- .../test/scala/org/apache/spark/ml/util/MLTestingUtils.scala | 2 +- .../spark/sql/execution/datasources/csv/CSVUtils.scala | 2 +- .../org/apache/spark/sql/SparkSessionExtensionSuite.scala| 8 .../org/apache/spark/sql/streaming/util/BlockingSource.scala | 2 +- .../spark/sql/hive/HiveContextCompatibilitySuite.scala | 4 ++-- .../org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala | 2 +- .../scala/org/apache/spark/sql/hive/ListTablesSuite.scala| 2 +- .../org/apache/spark/sql/hive/execution/HiveQuerySuite.scala | 2 +- .../spark/sql/hive/execution/HiveResolutionSuite.scala | 2 +- .../apache/spark/sql/hive/execution/HiveTableScanSuite.scala | 2 +- .../sql/sources/BucketedWriteWithHiveSupportSuite.scala | 2 +- 16 files changed, 28 insertions(+), 28 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index e23423e4c004..1257d2ccfbfb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -119,7 +119,7 @@ object MLUtils extends Logging { ).resolveRelation(checkFilesExist = false)) .select("value") -import lines.sqlContext.implicits._ +import lines.sparkSession.implicits._ lines.select(trim($"value").as("line")) .filter(not((length($"line") === 0).or($"line".startsWith("#" diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala index 68e83fccf3d1..ff9ce1ca7b9f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala @@ -52,8 +52,8 @@ class FMClassifierSuite extends MLTest with DefaultReadWriteTest { } test("FMClassifier: Predictor, Classifier methods") { -val sqlContext = smallBinaryDataset.sqlContext -import sqlContext.implicits._ +val session = smallBinaryDataset.sparkSession +import session.implicits._ val fm = new FMClassifier() val model = fm.fit(smallBinaryDataset) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 8e54262e2f61..b0e275f5e193 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -550,8 +550,8 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { } test("multinomial logistic regression: Predictor, Classifier methods") { -val sqlContext = smallMultinomialDataset.sqlContext -import sqlContext.implicits._ +val session = smallMultinomialDataset.sparkSession +import session.implicits._ val mlr = new LogisticRegression().setFamily("multinomial") val model = mlr.fit(smallMultinomialDataset) @@ -590,8 +590,8 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTes
(spark) branch master updated: [SPARK-48893][SQL][PYTHON][DOCS] Add some examples for `linearRegression` built-in functions
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new a809740cf6ec [SPARK-48893][SQL][PYTHON][DOCS] Add some examples for `linearRegression` built-in functions a809740cf6ec is described below commit a809740cf6ec039141d416f6fb27a6deb66b3d2c Author: Wei Guo AuthorDate: Tue Jul 23 10:27:21 2024 +0800 [SPARK-48893][SQL][PYTHON][DOCS] Add some examples for `linearRegression` built-in functions ### What changes were proposed in this pull request? This PR aims to add some extra examples for `linearRegression` built-in functions. ### Why are the changes needed? - Align the use examples for this series of functions. - Allow users to better understand the usage of `linearRegression` related methods from sql built-in functions docs(https://spark.apache.org/docs/latest/api/sql/index.html). ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA and Manual testing for new examples. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47343 from wayneguow/regr_series. Authored-by: Wei Guo Signed-off-by: yangjie01 --- python/pyspark/sql/functions/builtin.py| 545 ++--- .../expressions/aggregate/linearRegression.scala | 28 +- .../sql-functions/sql-expression-schema.md | 4 +- 3 files changed, 494 insertions(+), 83 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 5b9d0dd87002..3d094dd38c50 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -3671,16 +3671,59 @@ def regr_avgx(y: "ColumnOrName", x: "ColumnOrName") -> Column: Examples ->>> from pyspark.sql import functions as sf ->>> x = (sf.col("id") % 3).alias("x") ->>> y = (sf.randn(42) + x * 10).alias("y") ->>> spark.range(0, 1000, 1, 1).select(x, y).select( -... sf.regr_avgx("y", "x"), sf.avg("x") -... ).show() +Example 1: All paris are non-null + +>>> import pyspark.sql.functions as sf +>>> df = spark.sql("SELECT * FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x)") +>>> df.select(sf.regr_avgx("y", "x"), sf.avg("x")).show() +---+--+ |regr_avgx(y, x)|avg(x)| +---+--+ -| 0.999| 0.999| +| 2.75| 2.75| ++---+--+ + +Example 2: All paris's x values are null + +>>> import pyspark.sql.functions as sf +>>> df = spark.sql("SELECT * FROM VALUES (1, null) AS tab(y, x)") +>>> df.select(sf.regr_avgx("y", "x"), sf.avg("x")).show() ++---+--+ +|regr_avgx(y, x)|avg(x)| ++---+--+ +| NULL| NULL| ++---+--+ + +Example 3: All paris's y values are null + +>>> import pyspark.sql.functions as sf +>>> df = spark.sql("SELECT * FROM VALUES (null, 1) AS tab(y, x)") +>>> df.select(sf.regr_avgx("y", "x"), sf.avg("x")).show() ++---+--+ +|regr_avgx(y, x)|avg(x)| ++---+--+ +| NULL| 1.0| ++---+--+ + +Example 4: Some paris's x values are null + +>>> import pyspark.sql.functions as sf +>>> df = spark.sql("SELECT * FROM VALUES (1, 2), (2, null), (2, 3), (2, 4) AS tab(y, x)") +>>> df.select(sf.regr_avgx("y", "x"), sf.avg("x")).show() ++---+--+ +|regr_avgx(y, x)|avg(x)| ++---+--+ +|3.0| 3.0| ++---+--+ + +Example 5: Some paris's x or y values are null + +>>> import pyspark.sql.functions as sf +>>> df = spark.sql("SELECT * FROM VALUES (1, 2), (2, null), (null, 3), (2, 4) AS tab(y, x)") +>>> df.select(sf.regr_avgx("y", "x"), sf.avg("x")).show() ++---+--+ +|regr_avgx(y, x)|avg(x)| ++---+--+ +|3.0| 3.0| +---+--+ """ return _invoke_function_over_columns("regr_avgx", y, x) @@ -3708,17 +3751,60 @@ def regr_avgy(y: "ColumnOrName", x: "ColumnOrName") -> Column: Examples ->>&g
(spark) branch master updated: [SPARK-48943][TESTS] Upgrade `h2` to 2.3.230 and enhance the test coverage of behavior changes of `asin` and `acos` complying Standard SQL
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 4cc41ea63f94 [SPARK-48943][TESTS] Upgrade `h2` to 2.3.230 and enhance the test coverage of behavior changes of `asin` and `acos` complying Standard SQL 4cc41ea63f94 is described below commit 4cc41ea63f943b61be8f771f5cd95cfd4ea15c2e Author: Wei Guo AuthorDate: Tue Jul 23 10:21:32 2024 +0800 [SPARK-48943][TESTS] Upgrade `h2` to 2.3.230 and enhance the test coverage of behavior changes of `asin` and `acos` complying Standard SQL ### What changes were proposed in this pull request? This PR aims to upgrade `h2` from 2.2.220 to 2.3.230 and enhance the test coverage of behavior changes of `asin` and `acos` complying Standard SQL. The detail of behavior changes as follows: After this commit( https://github.com/h2database/h2database/commit/186647d4a35d05681febf4f53502b306aa6d511a), the behavior of `asin` and `acos` has changed in h2, complying with Standard SQL, and throwing exceptions directly when the argument is invalid(< -1d || > 1d). ### Why are the changes needed? 2.3.230 is latest version of `h2`, there are a lot of bug fixes and improvements. Full change notes: https://www.h2database.com/html/changelog.html ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Update a exist test case and add a new test case. Pass GA and manually test `JDBCV2Suite`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47414 from wayneguow/upgrade_h2. Authored-by: Wei Guo Signed-off-by: yangjie01 --- connect/server/pom.xml | 2 +- sql/core/pom.xml | 2 +- .../org/apache/spark/sql/jdbc/JDBCV2Suite.scala| 58 +++--- 3 files changed, 42 insertions(+), 20 deletions(-) diff --git a/connect/server/pom.xml b/connect/server/pom.xml index 73a3310c8a38..ecbb22168aa1 100644 --- a/connect/server/pom.xml +++ b/connect/server/pom.xml @@ -254,7 +254,7 @@ com.h2database h2 - 2.2.220 + 2.3.230 test diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 59d798e6e62f..c891763eb4e1 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -166,7 +166,7 @@ com.h2database h2 - 2.2.220 + 2.3.230 test diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala index e1a7971b283c..db06aac7f5e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.connector.expressions.Expression import org.apache.spark.sql.execution.FormattedMode import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation, V1ScanWrapper} import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog -import org.apache.spark.sql.functions.{abs, acos, asin, atan, atan2, avg, ceil, coalesce, cos, cosh, cot, count, count_distinct, degrees, exp, floor, lit, log => logarithm, log10, not, pow, radians, round, signum, sin, sinh, sqrt, sum, tan, tanh, udf, when} +import org.apache.spark.sql.functions.{abs, acos, asin, avg, ceil, coalesce, count, count_distinct, degrees, exp, floor, lit, log => logarithm, log10, not, pow, radians, round, signum, sqrt, sum, udf, when} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{DataType, IntegerType, StringType} @@ -1258,25 +1258,29 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkAnswer(df15, Seq(Row(1, "cathy", 9000, 1200, false), Row(2, "alex", 12000, 1200, false), Row(6, "jen", 12000, 1200, true))) -val df16 = spark.table("h2.test.employee") - .filter(sin($"bonus") < -0.08) - .filter(sinh($"bonus") > 200) - .filter(cos($"bonus") > 0.9) - .filter(cosh($"bonus") > 200) - .filter(tan($"bonus") < -0.08) - .filter(tanh($"bonus") === 1) - .filter(cot($"bonus") < -11) - .filter(asin($"bonus") > 0.1) - .filter(acos($"bonus") > 1.4) - .filter(atan($"bonus") > 1.4) - .filter(atan2($"bonus", $"bonus") > 0.7) +val df16 = sql( + """ +|SELECT * FROM h2.test.employee +|WHERE sin(bonus) < -0.08 +|
(spark) branch master updated: [MINOR][SQL][TESTS] Enable test case `testOrcAPI` in `JavaDataFrameReaderWriterSuite`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 546da0d5522e [MINOR][SQL][TESTS] Enable test case `testOrcAPI` in `JavaDataFrameReaderWriterSuite` 546da0d5522e is described below commit 546da0d5522ec79620bd29563c5ea809386635f5 Author: yangjie01 AuthorDate: Thu Jul 18 15:58:21 2024 +0800 [MINOR][SQL][TESTS] Enable test case `testOrcAPI` in `JavaDataFrameReaderWriterSuite` ### What changes were proposed in this pull request? This PR enabled test case `testOrcAPI` in `JavaDataFrameReaderWriterSuite` because this test no longer depends on Hive classes, we can test it like other test cases in this Suite. ### Why are the changes needed? Enable test case `testOrcAPI` in `JavaDataFrameReaderWriterSuite` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #47400 from LuciferYang/minor-testOrcAPI. Authored-by: yangjie01 Signed-off-by: yangjie01 --- .../test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java| 5 + 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java index 2a0c8c00574a..691fb67bbe90 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java @@ -144,10 +144,7 @@ public class JavaDataFrameReaderWriterSuite { .write().parquet(output); } - /** - * This only tests whether API compiles, but does not run it as orc() - * cannot be run without Hive classes. - */ + @Test public void testOrcAPI() { spark.read().schema(schema).orc(); spark.read().schema(schema).orc(input); - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [MINOR][SQL][TESTS] Fix compilation warning `adaptation of an empty argument list by inserting () is deprecated`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new b6c05259a0b9 [MINOR][SQL][TESTS] Fix compilation warning `adaptation of an empty argument list by inserting () is deprecated` b6c05259a0b9 is described below commit b6c05259a0b98205d2f0fe2476ecd09c8d258b0a Author: panbingkun AuthorDate: Mon Jul 15 17:11:17 2024 +0800 [MINOR][SQL][TESTS] Fix compilation warning `adaptation of an empty argument list by inserting () is deprecated` ### What changes were proposed in this pull request? The pr aims to fix compilation warning: `adaptation of an empty argument list by inserting () is deprecated` ### Why are the changes needed? Fix compilation warning. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually check. Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47350 from panbingkun/ParquetCommitterSuite_deprecated. Authored-by: panbingkun Signed-off-by: yangjie01 --- .../spark/sql/execution/datasources/parquet/ParquetCommitterSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCommitterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCommitterSuite.scala index eadd55bdc320..fb435e3639fd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCommitterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCommitterSuite.scala @@ -116,7 +116,7 @@ class ParquetCommitterSuite extends SparkFunSuite with SQLTestUtils test("SPARK-48804: Fail fast on unloadable or invalid committers") { Seq("invalid", getClass.getName).foreach { committer => val e = intercept[IllegalArgumentException] { -withSQLConf(SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key -> committer)() +withSQLConf(SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key -> committer)(()) } assert(e.getMessage.contains(classOf[OutputCommitter].getName)) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48864][SQL][TESTS] Refactor `HiveQuerySuite` and fix bug
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 4c7edd2a2048 [SPARK-48864][SQL][TESTS] Refactor `HiveQuerySuite` and fix bug 4c7edd2a2048 is described below commit 4c7edd2a20480a8521fcc88a966b22619143aebd Author: panbingkun AuthorDate: Fri Jul 12 15:22:34 2024 +0800 [SPARK-48864][SQL][TESTS] Refactor `HiveQuerySuite` and fix bug ### What changes were proposed in this pull request? The pr aims to refactor `HiveQuerySuite` and `fix` bug, includes: - use `getWorkspaceFilePath` to enable `HiveQuerySuite` to run successfully in the IDE. - make the test `lookup hive UDF in another thread` `independence`, without relying on the previous UT `current_database with multiple sessions`. - enable two test: `non-boolean conditions in a CaseWhen are illegal` and `Dynamic partition folder layout`. ### Why are the changes needed? - Run successfully in the `IDE` Before: https://github.com/apache/spark/assets/15246973/005fd49c-3edf-4e51-8223-097fd7a485bf;> After: https://github.com/apache/spark/assets/15246973/caedec72-be0c-4bb5-bc06-26cceef8b4b8;> - Make UT `lookup hive UDF in another thread` `independence` when `only` running it, it actually failed with the following error: https://github.com/apache/spark/assets/15246973/ef9c260f-8c0d-4821-8233-d4d7ae13802a;> **why ?** Because the previous UT `current_database with multiple sessions` changed `current database` and was not restored after it finished running. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Manually test - Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47293 from panbingkun/refactor_HiveQuerySuite. Authored-by: panbingkun Signed-off-by: yangjie01 --- .../sql/hive/execution/HiveComparisonTest.scala| 5 +- .../spark/sql/hive/execution/HiveQuerySuite.scala | 249 +++-- 2 files changed, 135 insertions(+), 119 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index f0feccb4f494..87e58bb8fa13 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -100,8 +100,9 @@ abstract class HiveComparisonTest extends SparkFunSuite with BeforeAndAfterAll { .map(name => new File(targetDir, s"$suiteName.$name")) /** The local directory with cached golden answer will be stored. */ - protected val answerCache = new File("src" + File.separator + "test" + -File.separator + "resources" + File.separator + "golden") + protected val answerCache = getWorkspaceFilePath( +"sql", "hive", "src", "test", "resources", "golden").toFile + if (!answerCache.exists) { answerCache.mkdir() } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 5ccb7f0d1f84..24d1e24b30c8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.hive.execution import java.io.File -import java.net.URI import java.nio.file.Files import java.sql.Timestamp @@ -679,15 +678,23 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd assert(actual === expected) } - // TODO: adopt this test when Spark SQL has the functionality / framework to report errors. - // See https://github.com/apache/spark/pull/1055#issuecomment-45820167 for a discussion. - ignore("non-boolean conditions in a CaseWhen are illegal") { + test("non-boolean conditions in a CaseWhen are illegal") { checkError( exception = intercept[AnalysisException] { sql("SELECT (CASE WHEN key > 2 THEN 3 WHEN 1 THEN 2 ELSE 0 END) FROM src").collect() }, - errorClass = null, - parameters = Map.empty) + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( +"sqlExpr" -> "\"CASE WHEN (key > 2) THEN 3 WHEN 1 THEN 2 ELSE 0 END\"", +"paramIndex" -> "second", +"inputSql" -
(spark) branch master updated: [SPARK-48866][SQL] Fix hints of valid charset in the error message of INVALID_PARAMETER_VALUE.CHARSET
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 261dbf4a9047 [SPARK-48866][SQL] Fix hints of valid charset in the error message of INVALID_PARAMETER_VALUE.CHARSET 261dbf4a9047 is described below commit 261dbf4a9047bc00271137b547341e02351106ed Author: Kent Yao AuthorDate: Thu Jul 11 18:59:10 2024 +0800 [SPARK-48866][SQL] Fix hints of valid charset in the error message of INVALID_PARAMETER_VALUE.CHARSET ### What changes were proposed in this pull request? This PR fixes hints at the error message of INVALID_PARAMETER_VALUE.CHARSET. The current error message does not enumerate all valid charsets, e.g. UTF-32. This PR parameterizes it to fix this issue. ### Why are the changes needed? Bugfix, the hint w/ charsets missing is not helpful ### Does this PR introduce _any_ user-facing change? Yes, error message changing ### How was this patch tested? modified tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #47295 from yaooqinn/SPARK-48866. Authored-by: Kent Yao Signed-off-by: yangjie01 --- common/utils/src/main/resources/error/error-conditions.json | 2 +- .../scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala | 5 +++-- .../resources/sql-tests/results/ansi/string-functions.sql.out | 8 .../src/test/resources/sql-tests/results/string-functions.sql.out | 8 .../org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala | 8 +--- 5 files changed, 25 insertions(+), 6 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 02d1e63e380a..7f54a77c94a0 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2584,7 +2584,7 @@ }, "CHARSET" : { "message" : [ - "expects one of the charsets 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16', but got ." + "expects one of the , but got ." ] }, "DATETIME_UNIT" : { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index d524742e126e..bdd53219de40 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -42,7 +42,7 @@ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.ValueInterval import org.apache.spark.sql.catalyst.trees.{Origin, TreeNode} -import org.apache.spark.sql.catalyst.util.{sideBySide, DateTimeUtils, FailFastMode, MapData} +import org.apache.spark.sql.catalyst.util.{sideBySide, CharsetProvider, DateTimeUtils, FailFastMode, MapData} import org.apache.spark.sql.connector.catalog.{CatalogNotFoundException, Table, TableProvider} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.connector.expressions.Transform @@ -2742,7 +2742,8 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE messageParameters = Map( "functionName" -> toSQLId(functionName), "parameter" -> toSQLId("charset"), -"charset" -> charset)) +"charset" -> charset, +"charsets" -> CharsetProvider.VALID_CHARSETS.mkString(", "))) } def malformedCharacterCoding(functionName: String, charset: String): RuntimeException = { diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out index da2fa9ca0c18..d4adec22c50f 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out @@ -846,6 +846,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "WINDOWS-1252", +"charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`encode`", "parameter" : "`charset`" } @@ -863,6 +864,7 @@ org.apache.spark.SparkIllegalArgumentException "sql
(spark) branch master updated: [SPARK-48826][BUILD] Upgrade `fasterxml.jackson` to 2.17.2
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 65daff55f556 [SPARK-48826][BUILD] Upgrade `fasterxml.jackson` to 2.17.2 65daff55f556 is described below commit 65daff55f556ab48e06aa1f0536b627a8b479b9b Author: Wei Guo AuthorDate: Tue Jul 9 16:01:27 2024 +0800 [SPARK-48826][BUILD] Upgrade `fasterxml.jackson` to 2.17.2 ### What changes were proposed in this pull request? This PR amis to upgrade `fasterxml.jackson` from 2.17.1 to 2.17.2. ### Why are the changes needed? There are some bug fixes about [Databind](https://github.com/FasterXML/jackson-databind): [#4561](https://github.com/FasterXML/jackson-databind/issues/4561): Issues using jackson-databind 2.17.1 with Reactor (wrt DeserializerCache and ReentrantLock) [#4575](https://github.com/FasterXML/jackson-databind/issues/4575): StdDelegatingSerializer does not consider a Converter that may return null for a non-null input [#4577](https://github.com/FasterXML/jackson-databind/issues/4577): Cannot deserialize value of type java.math.BigDecimal from String "3." (not a valid representation) [#4595](https://github.com/FasterXML/jackson-databind/issues/4595): No way to explicitly disable wrapping in custom annotation processor [#4607](https://github.com/FasterXML/jackson-databind/issues/4607): MismatchedInput: No Object Id found for an instance of X to assign to property 'id' [#4610](https://github.com/FasterXML/jackson-databind/issues/4610): DeserializationFeature.FAIL_ON_UNRESOLVED_OBJECT_IDS does not work when used with Polymorphic type handling The full release note of 2.17.2: https://github.com/FasterXML/jackson/wiki/Jackson-Release-2.17.2 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47241 from wayneguow/upgrade_jackson. Authored-by: Wei Guo Signed-off-by: yangjie01 --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 14 +++--- pom.xml | 4 ++-- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 2c3bee92176b..5ec7cb541ee7 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -102,16 +102,16 @@ icu4j/75.1//icu4j-75.1.jar ini4j/0.5.4//ini4j-0.5.4.jar istack-commons-runtime/3.0.8//istack-commons-runtime-3.0.8.jar ivy/2.5.2//ivy-2.5.2.jar -jackson-annotations/2.17.1//jackson-annotations-2.17.1.jar +jackson-annotations/2.17.2//jackson-annotations-2.17.2.jar jackson-core-asl/1.9.13//jackson-core-asl-1.9.13.jar -jackson-core/2.17.1//jackson-core-2.17.1.jar -jackson-databind/2.17.1//jackson-databind-2.17.1.jar -jackson-dataformat-cbor/2.17.1//jackson-dataformat-cbor-2.17.1.jar -jackson-dataformat-yaml/2.17.1//jackson-dataformat-yaml-2.17.1.jar +jackson-core/2.17.2//jackson-core-2.17.2.jar +jackson-databind/2.17.2//jackson-databind-2.17.2.jar +jackson-dataformat-cbor/2.17.2//jackson-dataformat-cbor-2.17.2.jar +jackson-dataformat-yaml/2.17.2//jackson-dataformat-yaml-2.17.2.jar jackson-datatype-jdk8/2.17.0//jackson-datatype-jdk8-2.17.0.jar -jackson-datatype-jsr310/2.17.1//jackson-datatype-jsr310-2.17.1.jar +jackson-datatype-jsr310/2.17.2//jackson-datatype-jsr310-2.17.2.jar jackson-mapper-asl/1.9.13//jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.13/2.17.1//jackson-module-scala_2.13-2.17.1.jar +jackson-module-scala_2.13/2.17.2//jackson-module-scala_2.13-2.17.2.jar jakarta.annotation-api/2.0.0//jakarta.annotation-api-2.0.0.jar jakarta.inject-api/2.0.1//jakarta.inject-api-2.0.1.jar jakarta.servlet-api/5.0.0//jakarta.servlet-api-5.0.0.jar diff --git a/pom.xml b/pom.xml index 0ebe6ab8c580..b2dd22cb0c0a 100644 --- a/pom.xml +++ b/pom.xml @@ -180,8 +180,8 @@ true true 1.9.13 -2.17.1 - 2.17.1 +2.17.2 + 2.17.2 2.3.1 3.0.2 1.1.10.5 - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48720][SQL] Align the command `ALTER TABLE ... UNSET TBLPROPERTIES ...` in v1 and v2
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 73126b17515a [SPARK-48720][SQL] Align the command `ALTER TABLE ... UNSET TBLPROPERTIES ...` in v1 and v2 73126b17515a is described below commit 73126b17515adc73dbb63f199fd641c330171d02 Author: panbingkun AuthorDate: Mon Jul 8 11:40:21 2024 +0800 [SPARK-48720][SQL] Align the command `ALTER TABLE ... UNSET TBLPROPERTIES ...` in v1 and v2 ### What changes were proposed in this pull request? The pr aims to: - align the command `ALTER TABLE ... UNSET TBLPROPERTIES ...` in v1 and v2. (this means that in the v1, regardless of whether `IF EXISTS` is specified or not, when unset a `non-existent` property, it is `ignored` and no longer `fails`.) - update the description of `ALTER TABLE ... UNSET TBLPROPERTIES ...` in the doc `docs/sql-ref-syntax-ddl-alter-table.md`. - unify v1 and v2 `ALTER TABLE ... UNSET TBLPROPERTIES ...` tests. - Add the following `scenario` for `ALTER TABLE ... SET TBLPROPERTIES ...` testing A.`table to alter does not exist` B.`alter table set reserved properties` ### Why are the changes needed? - align the command `ALTER TABLE ... UNSET TBLPROPERTIES ...` in v1 and v2, avoid confusing end-users. - to improve test coverage. - align with other similar tests, eg: `AlterTableSetTblProperties*` ### Does this PR introduce _any_ user-facing change? Yes, in the `v1`, regardless of whether `IF EXISTS` is specified or not, when unset a `non-existent` property, it is `ignored` and no longer `fails` ### How was this patch tested? Update some UT & Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47097 from panbingkun/alter_unset_table. Authored-by: panbingkun Signed-off-by: yangjie01 --- .../src/main/resources/error/error-conditions.json | 6 - docs/sql-ref-syntax-ddl-alter-table.md | 20 ++- .../spark/sql/errors/QueryCompilationErrors.scala | 10 -- .../spark/sql/catalyst/parser/DDLParserSuite.scala | 19 --- .../apache/spark/sql/execution/command/ddl.scala | 8 -- .../AlterTableSetTblPropertiesSuiteBase.scala | 80 +-- .../AlterTableUnsetTblPropertiesParserSuite.scala | 65 + .../AlterTableUnsetTblPropertiesSuiteBase.scala| 149 + .../sql/execution/command/DDLParserSuite.scala | 12 -- .../spark/sql/execution/command/DDLSuite.scala | 67 - .../v1/AlterTableSetTblPropertiesSuite.scala | 4 + ...ala => AlterTableUnsetTblPropertiesSuite.scala} | 17 ++- .../v2/AlterTableSetTblPropertiesSuite.scala | 4 + ...ala => AlterTableUnsetTblPropertiesSuite.scala} | 10 +- .../spark/sql/hive/execution/HiveDDLSuite.scala| 26 +--- .../AlterTableUnsetTblPropertiesSuite.scala| 27 16 files changed, 353 insertions(+), 171 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 45b922b88063..06f8d3a78252 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4275,12 +4275,6 @@ ], "sqlState" : "42883" }, - "UNSET_NONEXISTENT_PROPERTIES" : { -"message" : [ - "Attempted to unset non-existent properties [] in table ." -], -"sqlState" : "42K0J" - }, "UNSUPPORTED_ADD_FILE" : { "message" : [ "Don't support add file." diff --git a/docs/sql-ref-syntax-ddl-alter-table.md b/docs/sql-ref-syntax-ddl-alter-table.md index 566e73da2151..31eaf659b5c7 100644 --- a/docs/sql-ref-syntax-ddl-alter-table.md +++ b/docs/sql-ref-syntax-ddl-alter-table.md @@ -236,21 +236,29 @@ ALTER TABLE table_identifier DROP [ IF EXISTS ] partition_spec [PURGE] ### SET AND UNSET - SET TABLE PROPERTIES + SET PROPERTIES `ALTER TABLE SET` command is used for setting the table properties. If a particular property was already set, this overrides the old value with the new one. -`ALTER TABLE UNSET` is used to drop the table property. - # Syntax ```sql --- Set Table Properties +-- Set Properties ALTER TABLE table_identifier SET TBLPROPERTIES ( key1 = val1, key2 = val2, ... ) +``` + + UNSET PROPERTIES + +`ALTER TABLE UNSET` command is used to drop the table property. --- Unset Table Properties -ALTER TABLE table_identifier UNSET TBLPROPERTIES [ IF EXISTS ] ( key1, key2, ... ) +**Note:** If the specified property key does not exist, whether specify `IF EXISTS` or not, the command will ignore it and finally succeed.
(spark) branch master updated (f1eca903f5c2 -> 489e32535aad)
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from f1eca903f5c2 [SPARK-48719][SQL] Fix the calculation bug of `RegrSlope` & `RegrIntercept` when the first parameter is null add 489e32535aad [SPARK-48177][BUILD][FOLLOWUP] Update parquet version in `sql-data-sources-parquet.md` doc No new revisions were added by this update. Summary of changes: docs/sql-data-sources-parquet.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48805][SQL][ML][SS][AVRO][EXAMPLES] Replace calls to bridged APIs based on `SparkSession#sqlContext` with `SparkSession` API
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 54b75582506d [SPARK-48805][SQL][ML][SS][AVRO][EXAMPLES] Replace calls to bridged APIs based on `SparkSession#sqlContext` with `SparkSession` API 54b75582506d is described below commit 54b75582506d0e58af7f500b9d284ab7222e98f0 Author: yangjie01 AuthorDate: Thu Jul 4 19:27:22 2024 +0800 [SPARK-48805][SQL][ML][SS][AVRO][EXAMPLES] Replace calls to bridged APIs based on `SparkSession#sqlContext` with `SparkSession` API ### What changes were proposed in this pull request? In the internal code of Spark, there are instances where, despite having a SparkSession instance, the bridged APIs based on SparkSession#sqlContext are still used. Therefore, this PR makes some simplifications in this regard:" 1. `SparkSession#sqlContext#read` -> `SparkSession#read` ```scala /** * Returns a [[DataFrameReader]] that can be used to read non-streaming data in as a * `DataFrame`. * {{{ * sqlContext.read.parquet("/path/to/file.parquet") * sqlContext.read.schema(schema).json("/path/to/file.json") * }}} * * group genericdata * since 1.4.0 */ def read: DataFrameReader = sparkSession.read ``` 2. `SparkSession#sqlContext#setConf` -> `SparkSession#conf#set` ```scala /** * Set the given Spark SQL configuration property. * * group config * since 1.0.0 */ def setConf(key: String, value: String): Unit = { sparkSession.conf.set(key, value) } ``` 3. `SparkSession#sqlContext#getConf` -> `SparkSession#conf#get` ```scala /** * Return the value of Spark SQL configuration property for the given key. * * group config * since 1.0.0 */ def getConf(key: String): String = { sparkSession.conf.get(key) } ``` 4. `SparkSession#sqlContext#createDataFrame` -> `SparkSession#createDataFrame` ```scala /** * Creates a DataFrame from an RDD of Product (e.g. case classes, tuples). * * group dataframes * since 1.3.0 */ def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { sparkSession.createDataFrame(rdd) } ``` 5. `SparkSession#sqlContext#sessionState` -> `SparkSession#sessionState` ```scala private[sql] def sessionState: SessionState = sparkSession.sessionState ``` 6. `SparkSession#sqlContext#sharedState` -> `SparkSession#sharedState` ```scala private[sql] def sharedState: SharedState = sparkSession.sharedState ``` 7. `SparkSession#sqlContext#streams` -> `SparkSession#streams` ```scala /** * Returns a `StreamingQueryManager` that allows managing all the * [[org.apache.spark.sql.streaming.StreamingQuery StreamingQueries]] active on `this` context. * * since 2.0.0 */ def streams: StreamingQueryManager = sparkSession.streams ``` 8. `SparkSession#sqlContext#uncacheTable` -> `SparkSession#catalog#uncacheTable` ```scala /** * Removes the specified table from the in-memory cache. * group cachemgmt * since 1.3.0 */ def uncacheTable(tableName: String): Unit = { sparkSession.catalog.uncacheTable(tableName) } ``` ### Why are the changes needed? Decrease the nesting levels of API calls ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass GitHub Actions - Manually checked `SparkHiveExample` ### Was this patch authored or co-authored using generative AI tooling? No Closes #47210 from LuciferYang/session.sqlContext. Authored-by: yangjie01 Signed-off-by: yangjie01 --- .../src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala | 4 ++-- .../apache/spark/examples/sql/hive/SparkHiveExample.scala| 4 ++-- .../apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala | 2 +- .../sql/execution/streaming/FlatMapGroupsWithStateExec.scala | 12 ++-- .../sql/execution/streaming/TransformWithStateExec.scala | 12 ++-- .../test/scala/org/apache/spark/sql/CachedTableSuite.scala | 4 ++-- .../apache/spark/sql/errors/QueryExecutionErrorsSuite.scala | 2 +- .../apache/spark/sql/hive/HiveParquetMetastoreSuite.scala| 2 +- .../org/apache/spark/sql/hive/HiveUDFDynamicLoadSuite.scala | 2 +- .../spark/sql/hive/PartitionedTablePerfStatsSuite.scala | 2 +- 10 files changed, 23 insertions(+), 23 deletions(-) diff --git a/connector/avro/src/test/scala/org/apa
(spark) branch master updated: [SPARK-48765][DEPLOY] Enhance default value evaluation for SPARK_IDENT_STRING
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new bc16b24c7a32 [SPARK-48765][DEPLOY] Enhance default value evaluation for SPARK_IDENT_STRING bc16b24c7a32 is described below commit bc16b24c7a328cf103b003b1c4a5cf16832cf2bd Author: Cheng Pan AuthorDate: Mon Jul 1 19:49:31 2024 +0800 [SPARK-48765][DEPLOY] Enhance default value evaluation for SPARK_IDENT_STRING ### What changes were proposed in this pull request? This PR follows Hadoop[1] to enhance the `SPARK_IDENT_STRING` default value evaluation. [1] https://github.com/apache/hadoop/blob/rel/release-3.4.0/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh#L893-L896 ### Why are the changes needed? I found in some cases, `$USER` is not available, thus the auto-generated log and pid file names are strange. For example, there is no `$USER` when login to docker ``` $ docker run -t -i ubuntu:latest root1dbeaefd6cd4:/# echo $USER root1dbeaefd6cd4:/# id -nu root root1dbeaefd6cd4:/# exit ``` ### Does this PR introduce _any_ user-facing change? Yes, affects log/pid file names. ### How was this patch tested? Manually tested. ### Was this patch authored or co-authored using generative AI tooling? No Closes #47160 from pan3793/SPARK-48765. Authored-by: Cheng Pan Signed-off-by: yangjie01 --- sbin/spark-daemon.sh | 4 1 file changed, 4 insertions(+) diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 28d205f03e0f..b7233e6e9bf3 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -98,6 +98,10 @@ spark_rotate_log () . "${SPARK_HOME}/bin/load-spark-env.sh" if [ "$SPARK_IDENT_STRING" = "" ]; then + # if for some reason the shell doesn't have $USER defined + # (e.g., ssh'd in to execute a command) + # let's get the effective username and use that + USER=${USER:-$(id -nu)} export SPARK_IDENT_STRING="$USER" fi - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48691][BUILD] Upgrade scalatest related dependencies to the 3.2.19 series
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 7f5f96cad224 [SPARK-48691][BUILD] Upgrade scalatest related dependencies to the 3.2.19 series 7f5f96cad224 is described below commit 7f5f96cad22464e02679ab1a1c6eb08b9da039ef Author: Wei Guo AuthorDate: Wed Jun 26 22:31:08 2024 +0800 [SPARK-48691][BUILD] Upgrade scalatest related dependencies to the 3.2.19 series ### What changes were proposed in this pull request? This PR aims to upgrade: - `scalatest` to 3.2.19 - `mockto` to 5.12.0 - `selenium` to 4.21 - `bytebuddy` to 1.14.17 ### Why are the changes needed? Full release notes: scalatest: https://github.com/scalatest/scalatest/releases/tag/release-3.2.19 mockito: https://github.com/mockito/mockito/releases/tag/v5.12.0 https://github.com/mockito/mockito/releases/tag/v5.11.0 selenium: https://github.com/SeleniumHQ/selenium/compare/selenium-4.17.0...selenium-4.21.0 bytebuddy: https://github.com/raphw/byte-buddy/compare/byte-buddy-1.14.11...byte-buddy-1.14.17 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Pass GitHub Actions - Manual tests: ``` build/sbt clean -Dguava.version=33.0.0-jre -Dspark.test.webdriver.chrome.driver=/opt/homebrew/bin/chromedriver -Dtest.default.exclude.tags="org.apache.spark.tags.ExtendedLevelDBTest" -Phive -Phive-thriftserver \ "core/testOnly *HistoryServerSuite" ``` ``` build/sbt clean -Dguava.version=33.0.0-jre -Dspark.test.webdriver.chrome.driver=/opt/homebrew/bin/chromedriver -Dtest.default.exclude.tags="org.apache.spark.tags.ExtendedLevelDBTest" -Phive -Phive-thriftserver \ "core/testOnly *UISeleniumSuite" ``` ``` build/sbt clean -Dguava.version=33.0.0-jre -Dspark.test.webdriver.chrome.driver=/opt/homebrew/bin/chromedriver -Dtest.default.exclude.tags="org.apache.spark.tags.ExtendedLevelDBTest" -Phive -Phive-thriftserver \ "sql/testOnly *UISeleniumSuite" ``` ``` build/sbt clean -Dguava.version=33.0.0-jre -Dspark.test.webdriver.chrome.driver=/opt/homebrew/bin/chromedriver -Dtest.default.exclude.tags="org.apache.spark.tags.ExtendedLevelDBTest" -Phive -Phive-thriftserver \ "streaming/testOnly *UISeleniumSuite" ``` ``` build/sbt clean -Dguava.version=33.0.0-jre -Dspark.test.webdriver.chrome.driver=/opt/homebrew/bin/chromedriver -Dtest.default.exclude.tags="org.apache.spark.tags.ExtendedLevelDBTest" -Phive -Phive-thriftserver \ "hive-thriftserver/testOnly *UISeleniumSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47065 from wayneguow/upgrade_mockito. Authored-by: Wei Guo Signed-off-by: yangjie01 --- pom.xml | 26 +- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/pom.xml b/pom.xml index b2e7bba3ec75..5e181cc38d31 100644 --- a/pom.xml +++ b/pom.xml @@ -206,8 +206,8 @@ 0.16.0 4.13.1 1.1 -4.17.0 -4.17.0 +4.21.0 +4.21.0 3.1.0 1.1.0 1.8.0 @@ -420,12 +420,12 @@ org.scalatestplus - mockito-5-10_${scala.binary.version} + mockito-5-12_${scala.binary.version} test org.scalatestplus - selenium-4-17_${scala.binary.version} + selenium-4-21_${scala.binary.version} test @@ -1156,25 +1156,25 @@ org.scalatest scalatest_${scala.binary.version} -3.2.18 +3.2.19 test org.scalatestplus scalacheck-1-18_${scala.binary.version} -3.2.18.0 +3.2.19.0 test org.scalatestplus -mockito-5-10_${scala.binary.version} -3.2.18.0 +mockito-5-12_${scala.binary.version} +3.2.19.0 test org.scalatestplus -selenium-4-17_${scala.binary.version} -3.2.18.0 +selenium-4-21_${scala.binary.version} +3.2.19.0 test @@ -1186,19 +1186,19 @@ org.mockito mockito-core -5.10.0 +5.12.0 test net.bytebuddy byte-buddy -1.14.11 +1.14.17 test net.bytebuddy byte-buddy-agent -1.14.11 +1.14.17 test - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48724][SQL][TESTS] Fix incorrect conf settings of `ignoreCorruptFiles` related tests case in `ParquetQuerySuite`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new a474b88aa2eb [SPARK-48724][SQL][TESTS] Fix incorrect conf settings of `ignoreCorruptFiles` related tests case in `ParquetQuerySuite` a474b88aa2eb is described below commit a474b88aa2ebb2af17273975f2f91584c0ce9af1 Author: Wei Guo AuthorDate: Wed Jun 26 19:30:52 2024 +0800 [SPARK-48724][SQL][TESTS] Fix incorrect conf settings of `ignoreCorruptFiles` related tests case in `ParquetQuerySuite` ### What changes were proposed in this pull request? This PR aims to fix incorrect conf settings of `ignoreCorruptFiles` related tests case in `ParquetQuerySuite`. The inner `withSQLConf (SQLConf.IGNORE_CORRUPT_FILES.key -> "false")` will overwrite the outer configuration, making it impossible to test the situation where `sqlConf` is true. ### Why are the changes needed? Fix test coverage logic. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47088 from wayneguow/parquet_query_suite. Authored-by: Wei Guo Signed-off-by: yangjie01 --- .../datasources/parquet/ParquetQuerySuite.scala| 18 -- 1 file changed, 8 insertions(+), 10 deletions(-) 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 index a329d3fdc3cb..4d413efe5043 100644 --- 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 @@ -369,16 +369,14 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS } withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> sqlConf) { -withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") { - val exception = intercept[SparkException] { -testIgnoreCorruptFiles(options) - }.getCause - assert(exception.getMessage().contains("is not a Parquet file")) - val exception2 = intercept[SparkException] { -testIgnoreCorruptFilesWithoutSchemaInfer(options) - }.getCause - assert(exception2.getMessage().contains("is not a Parquet file")) -} +val exception = intercept[SparkException] { + testIgnoreCorruptFiles(options) +}.getCause +assert(exception.getMessage().contains("is not a Parquet file")) +val exception2 = intercept[SparkException] { + testIgnoreCorruptFilesWithoutSchemaInfer(options) +}.getCause +assert(exception2.getMessage().contains("is not a Parquet file")) } } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48692][BUILD] Upgrade `rocksdbjni` to 9.2.1
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 5112e5887714 [SPARK-48692][BUILD] Upgrade `rocksdbjni` to 9.2.1 5112e5887714 is described below commit 5112e58877147c7fb169d2c53845ce00de127866 Author: panbingkun AuthorDate: Tue Jun 25 11:45:42 2024 +0800 [SPARK-48692][BUILD] Upgrade `rocksdbjni` to 9.2.1 ### What changes were proposed in this pull request? The pr aims to upgrade rocksdbjni from `8.11.4` to `9.2.1`. ### Why are the changes needed? The full release notes as follows: https://github.com/facebook/rocksdb/releases/tag/v9.2.1 https://github.com/facebook/rocksdb/releases/tag/v9.1.1 https://github.com/facebook/rocksdb/releases/tag/v9.1.0 https://github.com/facebook/rocksdb/releases/tag/v9.0.1 https://github.com/facebook/rocksdb/releases/tag/v9.0.0 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46146 from panbingkun/test_rocksdbjni_9. Authored-by: panbingkun Signed-off-by: yangjie01 --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- pom.xml| 2 +- ...StoreBasicOperationsBenchmark-jdk21-results.txt | 120 ++--- .../StateStoreBasicOperationsBenchmark-results.txt | 120 ++--- 4 files changed, 122 insertions(+), 122 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index edaf1c494d13..b99ec346e6ab 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -244,7 +244,7 @@ parquet-jackson/1.13.1//parquet-jackson-1.13.1.jar pickle/1.5//pickle-1.5.jar py4j/0.10.9.7//py4j-0.10.9.7.jar remotetea-oncrpc/1.1.2//remotetea-oncrpc-1.1.2.jar -rocksdbjni/8.11.4//rocksdbjni-8.11.4.jar +rocksdbjni/9.2.1//rocksdbjni-9.2.1.jar scala-collection-compat_2.13/2.7.0//scala-collection-compat_2.13-2.7.0.jar scala-compiler/2.13.14//scala-compiler-2.13.14.jar scala-library/2.13.14//scala-library-2.13.14.jar diff --git a/pom.xml b/pom.xml index 85fc2aefdf90..a6dc3a60d89c 100644 --- a/pom.xml +++ b/pom.xml @@ -691,7 +691,7 @@ org.rocksdb rocksdbjni -8.11.4 +9.2.1 ${leveldbjni.group} diff --git a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk21-results.txt b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk21-results.txt index e563e60a8f48..6a42c7b283b7 100644 --- a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk21-results.txt @@ -2,143 +2,143 @@ put rows -OpenJDK 64-Bit Server VM 21.0.3+9-LTS on Linux 6.5.0-1018-azure +OpenJDK 64-Bit Server VM 21.0.3+9-LTS on Linux 6.5.0-1022-azure AMD EPYC 7763 64-Core Processor putting 1 rows (1 rows to overwrite - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms)Rate(M/s) Per Row(ns) Relative --- -In-memory 10 12 1 1.01023.2 1.0X -RocksDB (trackTotalNumberOfRows: true) 42 44 2 0.24197.6 0.2X -RocksDB (trackTotalNumberOfRows: false) 16 17 1 0.61591.7 0.6X +In-memory 10 11 1 1.0 968.0 1.0X +RocksDB (trackTotalNumberOfRows: true) 40 42 2 0.24033.5 0.2X +RocksDB (trackTotalNumberOfRows: false) 15 16 1 0.71502.0 0.6X -OpenJDK 64-Bit Server VM 21.0.3+9-LTS on Linux 6.5.0-1018-azure +OpenJDK 64-Bit Server VM 21.0.3+9-LTS on Linux 6.5.0-1022-azure AMD EPYC 7763 64-Core Processor putting 1 rows (5000 rows to overwrite - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms)Rate(M/s) Per Row(ns) Relative - -In-memory 10 11 1 1.01009.0 1.0X -RocksDB (trackTotalNumberOfRows: true
(spark) branch master updated: [SPARK-48661][BUILD] Upgrade `RoaringBitmap` to 1.1.0
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 67c71874bcb2 [SPARK-48661][BUILD] Upgrade `RoaringBitmap` to 1.1.0 67c71874bcb2 is described below commit 67c71874bcb2ce6fe2f68e0e47cab72e0d37a687 Author: Wei Guo AuthorDate: Fri Jun 21 10:31:43 2024 +0800 [SPARK-48661][BUILD] Upgrade `RoaringBitmap` to 1.1.0 ### What changes were proposed in this pull request? This PR aims to upgrade `RoaringBitmap` to 1.1.0. ### Why are the changes needed? There are some bug fixes in `RoaringBitmap` 1.1.0: Fix RunContainer#contains(BitmapContainer) (https://github.com/RoaringBitmap/RoaringBitmap/issues/721) by LeeWorrall in https://github.com/RoaringBitmap/RoaringBitmap/pull/722 Fix ArrayContainer#contains(RunContainer) (https://github.com/RoaringBitmap/RoaringBitmap/issues/723) by LeeWorrall in https://github.com/RoaringBitmap/RoaringBitmap/pull/724 Full release note: https://github.com/RoaringBitmap/RoaringBitmap/releases/tag/1.1.0 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47020 from wayneguow/upgrade_RoaringBitmap. Authored-by: Wei Guo Signed-off-by: yangjie01 --- core/benchmarks/MapStatusesConvertBenchmark-jdk21-results.txt | 8 core/benchmarks/MapStatusesConvertBenchmark-results.txt | 8 dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- pom.xml | 2 +- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/core/benchmarks/MapStatusesConvertBenchmark-jdk21-results.txt b/core/benchmarks/MapStatusesConvertBenchmark-jdk21-results.txt index 71c13a0fc5ad..a15442496b24 100644 --- a/core/benchmarks/MapStatusesConvertBenchmark-jdk21-results.txt +++ b/core/benchmarks/MapStatusesConvertBenchmark-jdk21-results.txt @@ -2,12 +2,12 @@ MapStatuses Convert Benchmark -OpenJDK 64-Bit Server VM 21.0.3+9-LTS on Linux 6.5.0-1018-azure +OpenJDK 64-Bit Server VM 21.0.3+9-LTS on Linux 6.5.0-1022-azure AMD EPYC 7763 64-Core Processor MapStatuses Convert: Best Time(ms) Avg Time(ms) Stdev(ms)Rate(M/s) Per Row(ns) Relative -Num Maps: 5 Fetch partitions:500664678 14 0.0 664277160.0 1.0X -Num Maps: 5 Fetch partitions:1000 1597 1616 29 0.0 1596794881.0 0.4X -Num Maps: 5 Fetch partitions:1500 2402 2421 18 0.0 2401654923.0 0.3X +Num Maps: 5 Fetch partitions:500674685 12 0.0 673772738.0 1.0X +Num Maps: 5 Fetch partitions:1000 1579 1590 12 0.0 1579383970.0 0.4X +Num Maps: 5 Fetch partitions:1500 2435 2472 37 0.0 2434530380.0 0.3X diff --git a/core/benchmarks/MapStatusesConvertBenchmark-results.txt b/core/benchmarks/MapStatusesConvertBenchmark-results.txt index a7379aa0d4af..b9f36af4a653 100644 --- a/core/benchmarks/MapStatusesConvertBenchmark-results.txt +++ b/core/benchmarks/MapStatusesConvertBenchmark-results.txt @@ -2,12 +2,12 @@ MapStatuses Convert Benchmark -OpenJDK 64-Bit Server VM 17.0.11+9-LTS on Linux 6.5.0-1018-azure +OpenJDK 64-Bit Server VM 17.0.11+9-LTS on Linux 6.5.0-1022-azure AMD EPYC 7763 64-Core Processor MapStatuses Convert: Best Time(ms) Avg Time(ms) Stdev(ms)Rate(M/s) Per Row(ns) Relative -Num Maps: 5 Fetch partitions:500699715 14 0.0 698750825.0 1.0X -Num Maps: 5 Fetch partitions:1000 1653 1676 36 0.0 1653453370.0 0.4X -Num Maps: 5 Fetch partitions:1500 2580 2613 30 0.0 2579900318.0 0.3X +Num Maps: 5 Fetch partitions:500703716 11 0.0 703103575.0 1.0X +Num Maps: 5 Fetch partitions:1000 1707 1723 14 0.0 1707060398.0 0.4X +Num Maps
(spark) branch master updated: [SPARK-47148][SQL][FOLLOWUP] Use broadcast hint to make test more stable
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 58701d811d95 [SPARK-47148][SQL][FOLLOWUP] Use broadcast hint to make test more stable 58701d811d95 is described below commit 58701d811d95918ac4a73d8fb260c46ccbf25bdd Author: Wenchen Fan AuthorDate: Tue Jun 18 16:10:19 2024 +0800 [SPARK-47148][SQL][FOLLOWUP] Use broadcast hint to make test more stable ### What changes were proposed in this pull request? A followup of https://github.com/apache/spark/pull/45234 to make the test more stable by using broadcast hint. ### Why are the changes needed? test improvement ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? N/A ### Was this patch authored or co-authored using generative AI tooling? no Closes #47007 from cloud-fan/follow. Authored-by: Wenchen Fan Signed-off-by: yangjie01 --- .../apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala| 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 4e1e171c8a84..d6fd45269ce6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -961,7 +961,7 @@ class AdaptiveQueryExecSuite spark.range(10).toDF("col1").createTempView("t1") spark.range(5).coalesce(2).toDF("col2").createTempView("t2") spark.range(15).toDF("col3").filter(Symbol("col3") >= 2).createTempView("t3") - sql("SELECT * FROM (SELECT /*+ BROADCAST(t2) */ * FROM t1 " + + sql("SELECT /*+ BROADCAST(t3) */ * FROM (SELECT /*+ BROADCAST(t2) */ * FROM t1 " + "INNER JOIN t2 ON t1.col1 = t2.col2) t JOIN t3 ON t.col1 = t3.col3;") } withTempView("t1", "t2", "t3") { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48585][SQL] Make `built-in` JdbcDialect's method `classifyException` throw out the `original` exception
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new a3feffdff9cd [SPARK-48585][SQL] Make `built-in` JdbcDialect's method `classifyException` throw out the `original` exception a3feffdff9cd is described below commit a3feffdff9cd17e0435ac5620731093f40d1a3bf Author: panbingkun AuthorDate: Tue Jun 18 14:50:34 2024 +0800 [SPARK-48585][SQL] Make `built-in` JdbcDialect's method `classifyException` throw out the `original` exception ### What changes were proposed in this pull request? The pr aims to make `built-in` JdbcDialect's method classifyException throw out the `original` exception. ### Why are the changes needed? As discussed in https://github.com/apache/spark/pull/46912#discussion_r1630876576, the following code: https://github.com/apache/spark/blob/df4156aa3217cf0f58b4c6cbf33c967bb43f7155/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala#L746-L751 have lost the original cause of the error, let's correct it. ### Does this PR introduce _any_ user-facing change? Yes, more accurate error conditions for end users. ### How was this patch tested? - Manually test. - Update existed UT & Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46937 from panbingkun/improve_JDBCTableCatalog. Authored-by: panbingkun Signed-off-by: yangjie01 --- .../org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala | 26 -- .../apache/spark/sql/jdbc/AggregatedDialect.scala | 3 ++- .../org/apache/spark/sql/jdbc/DB2Dialect.scala | 2 +- .../apache/spark/sql/jdbc/DatabricksDialect.scala | 2 +- .../org/apache/spark/sql/jdbc/DerbyDialect.scala | 2 +- .../org/apache/spark/sql/jdbc/H2Dialect.scala | 2 +- .../org/apache/spark/sql/jdbc/JdbcDialects.scala | 17 ++ .../apache/spark/sql/jdbc/MsSqlServerDialect.scala | 2 +- .../org/apache/spark/sql/jdbc/MySQLDialect.scala | 2 +- .../org/apache/spark/sql/jdbc/OracleDialect.scala | 2 +- .../apache/spark/sql/jdbc/PostgresDialect.scala| 3 ++- .../apache/spark/sql/jdbc/SnowflakeDialect.scala | 2 +- .../apache/spark/sql/jdbc/TeradataDialect.scala| 2 +- .../v2/jdbc/JDBCTableCatalogSuite.scala| 16 ++--- 14 files changed, 52 insertions(+), 31 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala index c78e87d0b846..88ba00a8a1ae 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala @@ -83,14 +83,16 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu def testCreateTableWithProperty(tbl: String): Unit = {} - def checkErrorFailedLoadTable(e: AnalysisException, tbl: String): Unit = { -checkError( + private def checkErrorFailedJDBC( + e: AnalysisException, + errorClass: String, + tbl: String): Unit = { +checkErrorMatchPVals( exception = e, - errorClass = "FAILED_JDBC.UNCLASSIFIED", + errorClass = errorClass, parameters = Map( -"url" -> "jdbc:", -"message" -> s"Failed to load table: $tbl" - ) +"url" -> "jdbc:.*", +"tableName" -> s"`$tbl`") ) } @@ -132,7 +134,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu val e = intercept[AnalysisException] { sql(s"ALTER TABLE $catalogName.not_existing_table ADD COLUMNS (C4 STRING)") } -checkErrorFailedLoadTable(e, "not_existing_table") +checkErrorFailedJDBC(e, "FAILED_JDBC.LOAD_TABLE", "not_existing_table") } test("SPARK-33034: ALTER TABLE ... drop column") { @@ -154,7 +156,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu val e = intercept[AnalysisException] { sql(s"ALTER TABLE $catalogName.not_existing_table DROP COLUMN C1") } -checkErrorFailedLoadTable(e, "not_existing_table") +checkErrorFailedJDBC(e, "FAILED_JDBC.LOAD_TABLE", "not_existing_table") } test("SPARK-33034: ALTER TABLE ... update column type") { @@ -170,7 +172,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu val e = intercept[AnalysisException] { sql(s"ALTER TABLE $catalogName.not_existing_table
(spark) branch master updated: [SPARK-48615][SQL] Perf improvement for parsing hex string
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 257a7883f215 [SPARK-48615][SQL] Perf improvement for parsing hex string 257a7883f215 is described below commit 257a7883f2150e037eb05f8c7a84103ad9a1 Author: Kent Yao AuthorDate: Mon Jun 17 09:56:05 2024 +0800 [SPARK-48615][SQL] Perf improvement for parsing hex string ### What changes were proposed in this pull request? Currently, we use two heximal string parsing functions. One uses Apache Codecs Hex for X-prefixed lit parsing, and the other use builtin impl for unhex function. I did a benchmark for them comparing with the `java.util.HexFormat` which was introduced in JDK17. ``` OpenJDK 64-Bit Server VM 17.0.10+0 on Mac OS X 14.5 Apple M2 Max Cardinality 100: Best Time(ms) Avg Time(ms) Stdev(ms)Rate(M/s) Per Row(ns) Relative Apache 5050 5100 86 0.25050.1 1.0X Spark 3822 3840 30 0.33821.6 1.3X Java 2462 2522 87 0.42462.1 2.1X OpenJDK 64-Bit Server VM 17.0.10+0 on Mac OS X 14.5 Apple M2 Max Cardinality 200: Best Time(ms) Avg Time(ms) Stdev(ms)Rate(M/s) Per Row(ns) Relative Apache10020 10828 1154 0.25010.1 1.0X Spark 6875 6966 144 0.33437.7 1.5X Java 4999 5092 89 0.42499.3 2.0X OpenJDK 64-Bit Server VM 17.0.10+0 on Mac OS X 14.5 Apple M2 Max Cardinality 400: Best Time(ms) Avg Time(ms) Stdev(ms)Rate(M/s) Per Row(ns) Relative Apache20090 20433 433 0.25022.5 1.0X Spark 13389 13620 229 0.33347.2 1.5X Java 10023 10069 42 0.42505.6 2.0X OpenJDK 64-Bit Server VM 17.0.10+0 on Mac OS X 14.5 Apple M2 Max Cardinality 800: Best Time(ms) Avg Time(ms) Stdev(ms)Rate(M/s) Per Row(ns) Relative Apache40277 43453 2755 0.25034.7 1.0X Spark 27145 27380 311 0.33393.1 1.5X Java 19980 21198 1473 0.42497.5 2.0X ``` The results indicate that the speed is Apache Codecs < builtin < Java, increasing by ~50%. In this PR, we replace these two with the Java 17 API ### Why are the changes needed? performance enhance ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? benchmarking existing unit tests in org.apache.spark.sql.catalyst.expressions.MathExpressionsSuite ### Was this patch authored or co-authored using generative AI tooling? no Closes #46972 from yaooqinn/SPARK-48615. Authored-by: Kent Yao Signed-off-by: yangjie01 --- .../benchmarks/HexBenchmark-jdk21-results.txt | 14 sql/catalyst/benchmarks/HexBenchmark-results.txt | 14 .../sql/catalyst/expressions/mathExpressions.scala | 94 +- .../spark/sql/catalyst/parser/AstBuilder.scala | 7 +- .../sql/catalyst/expressions/HexBenchmark.scala| 90 + 5 files changed, 158 insertions(+), 61 deletions(-) diff --git a/sql/catalyst/benchmarks/HexBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/HexBenchmark-jdk21-results.txt new file mode 100644 index ..afa3efa7a919 --- /dev/null +++ b/sql/ca
(spark) branch master updated: [SPARK-48626][CORE] Change the scope of object LogKeys as private in Spark
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 878de0014a37 [SPARK-48626][CORE] Change the scope of object LogKeys as private in Spark 878de0014a37 is described below commit 878de0014a3782187180c40158f0805e51335cb5 Author: Gengliang Wang AuthorDate: Fri Jun 14 15:08:41 2024 +0800 [SPARK-48626][CORE] Change the scope of object LogKeys as private in Spark ### What changes were proposed in this pull request? Change the scope of object LogKeys as private in Spark. ### Why are the changes needed? LogKeys are internal and developing. Making it private can avoid future confusion or compiling failures. This is suggested by pan3793 in https://github.com/apache/spark/pull/46947#issuecomment-2167164424 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing UT ### Was this patch authored or co-authored using generative AI tooling? No Closes #46983 from gengliangwang/changeScope. Authored-by: Gengliang Wang Signed-off-by: yangjie01 --- common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala index b8b63382fe4c..ec621c4f84ce 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala @@ -57,7 +57,7 @@ trait LogKey { * Various keys used for mapped diagnostic contexts(MDC) in logging. All structured logging keys * should be defined here for standardization. */ -object LogKeys { +private[spark] object LogKeys { case object ACCUMULATOR_ID extends LogKey case object ACL_ENABLED extends LogKey case object ACTUAL_NUM_FILES extends LogKey - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48612][SQL][SS] Cleanup deprecated api usage related to commons-pool2
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 157b1e3ba5d5 [SPARK-48612][SQL][SS] Cleanup deprecated api usage related to commons-pool2 157b1e3ba5d5 is described below commit 157b1e3ba5d5d5e75eb79805eaa3ea14fa876f5b Author: yangjie01 AuthorDate: Fri Jun 14 12:36:22 2024 +0800 [SPARK-48612][SQL][SS] Cleanup deprecated api usage related to commons-pool2 ### What changes were proposed in this pull request? This pr make the following changes - o.a.c.pool2.impl.BaseObjectPoolConfig#setMinEvictableIdleTime -> o.a.c.pool2.impl.BaseObjectPoolConfig#setMinEvictableIdleDuration - o.a.c.pool2.impl.BaseObjectPoolConfig#setSoftMinEvictableIdleTime -> o.a.c.pool2.impl.BaseObjectPoolConfig#setSoftMinEvictableIdleDuration to fix the following compilation warnings related to 'commons-pool2': ``` [WARNING] [Warn] /Users/yangjie01/SourceCode/git/spark-mine-13/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/InternalKafkaConsumerPool.scala:186: method setMinEvictableIdleTime in class BaseObjectPoolConfig is deprecated Applicable -Wconf / nowarn filters for this warning: msg=, cat=deprecation, site=org.apache.spark.sql.kafka010.consumer.InternalKafkaConsumerPool.PoolConfig.init, origin=org.apache.commons.pool2.impl.BaseObjectPoolConfig.setMinEvictableIdleTime [WARNING] [Warn] /Users/yangjie01/SourceCode/git/spark-mine-13/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/InternalKafkaConsumerPool.scala:187: method setSoftMinEvictableIdleTime in class BaseObjectPoolConfig is deprecated Applicable -Wconf / nowarn filters for this warning: msg=, cat=deprecation, site=org.apache.spark.sql.kafka010.consumer.InternalKafkaConsumerPool.PoolConfig.init, origin=org.apache.commons.pool2.impl.BaseObjectPoolConfig.setSoftMinEvictableIdleTime ``` The fix refers to: - https://github.com/apache/commons-pool/blob/e5c44f5184a55a58fef4a1efec8124d162a348bd/src/main/java/org/apache/commons/pool2/impl/BaseObjectPoolConfig.java#L765-L789 - https://github.com/apache/commons-pool/blob/e5c44f5184a55a58fef4a1efec8124d162a348bd/src/main/java/org/apache/commons/pool2/impl/BaseObjectPoolConfig.java#L815-L839 ```java /** * Sets the value for the {code minEvictableIdleTime} configuration attribute for pools created with this configuration instance. * * param minEvictableIdleTime The new setting of {code minEvictableIdleTime} for this configuration instance * see GenericObjectPool#getMinEvictableIdleDuration() * see GenericKeyedObjectPool#getMinEvictableIdleDuration() * since 2.10.0 * deprecated Use {link #setMinEvictableIdleDuration(Duration)}. */ Deprecated public void setMinEvictableIdleTime(final Duration minEvictableIdleTime) { this.minEvictableIdleDuration = PoolImplUtils.nonNull(minEvictableIdleTime, DEFAULT_MIN_EVICTABLE_IDLE_TIME); } /** * Sets the value for the {code minEvictableIdleTime} configuration attribute for pools created with this configuration instance. * * param minEvictableIdleTime The new setting of {code minEvictableIdleTime} for this configuration instance * see GenericObjectPool#getMinEvictableIdleDuration() * see GenericKeyedObjectPool#getMinEvictableIdleDuration() * since 2.12.0 */ public void setMinEvictableIdleDuration(final Duration minEvictableIdleTime) { this.minEvictableIdleDuration = PoolImplUtils.nonNull(minEvictableIdleTime, DEFAULT_MIN_EVICTABLE_IDLE_TIME); } /** * Sets the value for the {code softMinEvictableIdleTime} configuration attribute for pools created with this configuration instance. * * param softMinEvictableIdleTime The new setting of {code softMinEvictableIdleTime} for this configuration instance * see GenericObjectPool#getSoftMinEvictableIdleDuration() * see GenericKeyedObjectPool#getSoftMinEvictableIdleDuration() * since 2.10.0 * deprecated Use {link #setSoftMinEvictableIdleDuration(Duration)}. */ Deprecated public void setSoftMinEvictableIdleTime(final Duration softMinEvictableIdleTime) { this.softMinEvictableIdleDuration = PoolImplUtils.nonNull(softMinEvictableIdleTime, DEFAULT_SOFT_MIN_EVICTABLE_IDLE_TIME); } /** * Sets the value for the {code softMinEvictableIdleTime} configuration attribute for pools created with this configuration instance. * * param softMinEvictableIdleTime The new setting of {code softMinEvictableId
(spark) branch master updated: [SPARK-45685][SQL][FOLLOWUP] Add handling for `Stream` where `LazyList.force` is called
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 75fff90d2618 [SPARK-45685][SQL][FOLLOWUP] Add handling for `Stream` where `LazyList.force` is called 75fff90d2618 is described below commit 75fff90d2618617a66b9a3311792c8b16e8e Author: yangjie01 AuthorDate: Fri Jun 14 12:30:44 2024 +0800 [SPARK-45685][SQL][FOLLOWUP] Add handling for `Stream` where `LazyList.force` is called ### What changes were proposed in this pull request? Refer to the suggestion of https://github.com/apache/spark/pull/43563#pullrequestreview-2114900378, this pr add handling for Stream where LazyList.force is called ### Why are the changes needed? Even though `Stream` is deprecated in 2.13, it is not _removed_ and thus is is possible that some parts of Spark / Catalyst (or third-party code) might continue to pass around `Stream` instances. Hence, we should restore the call to `Stream.force` where `.force` is called on `LazyList`, to avoid losing the eager materialization for Streams that happen to flow to these call sites. This is also a guarantee of compatibility. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Add some new tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #46970 from LuciferYang/SPARK-45685-FOLLOWUP. Authored-by: yangjie01 Signed-off-by: yangjie01 --- .../spark/sql/catalyst/plans/QueryPlan.scala | 4 +++- .../apache/spark/sql/catalyst/trees/TreeNode.scala | 13 --- .../sql/catalyst/plans/LogicalPlanSuite.scala | 22 ++ .../spark/sql/catalyst/trees/TreeNodeSuite.scala | 27 ++ .../sql/execution/WholeStageCodegenExec.scala | 4 +++- .../apache/spark/sql/execution/PlannerSuite.scala | 8 +++ .../sql/execution/WholeStageCodegenSuite.scala | 10 7 files changed, 83 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index bc0ca31dc635..c9c8fdb676b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -226,12 +226,14 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] } } +@scala.annotation.nowarn("cat=deprecation") def recursiveTransform(arg: Any): AnyRef = arg match { case e: Expression => transformExpression(e) case Some(value) => Some(recursiveTransform(value)) case m: Map[_, _] => m case d: DataType => d // Avoid unpacking Structs - case stream: LazyList[_] => stream.map(recursiveTransform).force + case stream: Stream[_] => stream.map(recursiveTransform).force + case lazyList: LazyList[_] => lazyList.map(recursiveTransform).force case seq: Iterable[_] => seq.map(recursiveTransform) case other: AnyRef => other case null => null diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 23d26854a767..6683f2dbfb39 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.trees import java.util.UUID +import scala.annotation.nowarn import scala.collection.{mutable, Map} import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag @@ -378,12 +379,16 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] case nonChild: AnyRef => nonChild case null => null } +@nowarn("cat=deprecation") val newArgs = mapProductIterator { case s: StructType => s // Don't convert struct types to some other type of Seq[StructField] // Handle Seq[TreeNode] in TreeNode parameters. - case s: LazyList[_] => -// LazyList is lazy so we need to force materialization + case s: Stream[_] => +// Stream is lazy so we need to force materialization s.map(mapChild).force + case l: LazyList[_] => +// LazyList is lazy so we need to force materialization +l.map(mapChild).force case s: Seq[_] => s.map(mapChild) case m: Map[_, _] => @@ -801,6 +806,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] case other => other } +@nowarn("cat=deprecation") val newArgs = mapP
(spark) branch master updated: [SPARK-48604][SQL] Replace deprecated `new ArrowType.Decimal(precision, scale)` method call
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 08e741b92b8f [SPARK-48604][SQL] Replace deprecated `new ArrowType.Decimal(precision, scale)` method call 08e741b92b8f is described below commit 08e741b92b8fc9e43c838d0849317916218414ce Author: Wei Guo AuthorDate: Thu Jun 13 18:11:30 2024 +0800 [SPARK-48604][SQL] Replace deprecated `new ArrowType.Decimal(precision, scale)` method call ### What changes were proposed in this pull request? This pr replaces deprecated classes and methods of `arrow-vector` called in Spark: - `Decimal(int precision, int scale)` -> `Decimal( JsonProperty("precision") int precision, JsonProperty("scale") int scale, JsonProperty("bitWidth") int bitWidth )` All `arrow-vector` related Spark classes, I made a double check, only in `ArrowUtils` there is a deprecated method call. ### Why are the changes needed? Clean up deprecated API usage. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Passed GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46961 from wayneguow/deprecated_arrow. Authored-by: Wei Guo Signed-off-by: yangjie01 --- sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala index d9bd3b0e612b..6852fe09ef96 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala @@ -51,7 +51,7 @@ private[sql] object ArrowUtils { case BinaryType if !largeVarTypes => ArrowType.Binary.INSTANCE case _: StringType if largeVarTypes => ArrowType.LargeUtf8.INSTANCE case BinaryType if largeVarTypes => ArrowType.LargeBinary.INSTANCE -case DecimalType.Fixed(precision, scale) => new ArrowType.Decimal(precision, scale) +case DecimalType.Fixed(precision, scale) => new ArrowType.Decimal(precision, scale, 8 * 16) case DateType => new ArrowType.Date(DateUnit.DAY) case TimestampType if timeZoneId == null => throw SparkException.internalError("Missing timezoneId where it is mandatory.") - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (78fd4e3301ff -> b8c7aee12f02)
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 78fd4e3301ff [SPARK-48584][SQL][FOLLOWUP] Improve the unescapePathName add b8c7aee12f02 [SPARK-48609][BUILD] Upgrade `scala-xml` to 2.3.0 No new revisions were added by this update. Summary of changes: dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48583][SQL][TESTS] Replace deprecated classes and methods of `commons-io` called in Spark
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new fd045c9887fe [SPARK-48583][SQL][TESTS] Replace deprecated classes and methods of `commons-io` called in Spark fd045c9887fe is described below commit fd045c9887feabc37c0f15fa41c860847f5fffa0 Author: Wei Guo AuthorDate: Thu Jun 13 11:03:45 2024 +0800 [SPARK-48583][SQL][TESTS] Replace deprecated classes and methods of `commons-io` called in Spark ### What changes were proposed in this pull request? This pr replaces deprecated classes and methods of `commons-io` called in Spark: - `writeStringToFile(final File file, final String data)` -> `writeStringToFile(final File file, final String data, final Charset charset)` - `CountingInputStream` -> `BoundedInputStream` ### Why are the changes needed? Clean up deprecated API usage. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Passed related test cases in `UDFXPathUtilSuite` and `XmlSuite`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46935 from wayneguow/deprecated. Authored-by: Wei Guo Signed-off-by: yangjie01 --- .../spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala | 3 ++- .../spark/sql/execution/datasources/xml/XmlInputFormat.scala | 10 ++ 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala index a8dc2b20f56d..8351e94c0c36 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.xml +import java.nio.charset.StandardCharsets import javax.xml.xpath.XPathConstants.STRING import org.w3c.dom.Node @@ -85,7 +86,7 @@ class UDFXPathUtilSuite extends SparkFunSuite { tempFile.deleteOnExit() val fname = tempFile.getAbsolutePath -FileUtils.writeStringToFile(tempFile, secretValue) +FileUtils.writeStringToFile(tempFile, secretValue, StandardCharsets.UTF_8) val xml = s""" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlInputFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlInputFormat.scala index 4359ac02f5f5..6169cec6f821 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlInputFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlInputFormat.scala @@ -20,7 +20,7 @@ import java.io.{InputStream, InputStreamReader, IOException, Reader} import java.nio.ByteBuffer import java.nio.charset.Charset -import org.apache.commons.io.input.CountingInputStream +import org.apache.commons.io.input.BoundedInputStream import org.apache.hadoop.fs.Seekable import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.io.compress._ @@ -67,7 +67,7 @@ private[xml] class XmlRecordReader extends RecordReader[LongWritable, Text] { private var end: Long = _ private var reader: Reader = _ private var filePosition: Seekable = _ - private var countingIn: CountingInputStream = _ + private var countingIn: BoundedInputStream = _ private var readerLeftoverCharFn: () => Boolean = _ private var readerByteBuffer: ByteBuffer = _ private var decompressor: Decompressor = _ @@ -117,7 +117,9 @@ private[xml] class XmlRecordReader extends RecordReader[LongWritable, Text] { } } else { fsin.seek(start) - countingIn = new CountingInputStream(fsin) + countingIn = BoundedInputStream.builder() +.setInputStream(fsin) +.get() in = countingIn // don't use filePosition in this case. We have to count bytes read manually } @@ -156,7 +158,7 @@ private[xml] class XmlRecordReader extends RecordReader[LongWritable, Text] { if (filePosition != null) { filePosition.getPos } else { - start + countingIn.getByteCount - + start + countingIn.getCount - readerByteBuffer.remaining() - (if (readerLeftoverCharFn()) 1 else 0) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (53d65fd12dd9 -> 452c1b64b625)
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 53d65fd12dd9 [SPARK-48565][UI] Fix thread dump display in UI add 452c1b64b625 [SPARK-48551][SQL] Perf improvement for escapePathName No new revisions were added by this update. Summary of changes: .../EscapePathBenchmark-jdk21-results.txt} | 9 +-- .../benchmarks/EscapePathBenchmark-results.txt}| 9 +-- .../catalyst/catalog/ExternalCatalogUtils.scala| 42 .../spark/sql/catalyst/EscapePathBenchmark.scala | 74 ++ .../catalog/ExternalCatalogUtilsSuite.scala| 42 5 files changed, 154 insertions(+), 22 deletions(-) copy sql/{core/benchmarks/HashedRelationMetricsBenchmark-results.txt => catalyst/benchmarks/EscapePathBenchmark-jdk21-results.txt} (51%) copy sql/{core/benchmarks/HashedRelationMetricsBenchmark-results.txt => catalyst/benchmarks/EscapePathBenchmark-results.txt} (51%) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/EscapePathBenchmark.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtilsSuite.scala - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48563][BUILD] Upgrade `pickle` to 1.5
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 3fe6abde125b [SPARK-48563][BUILD] Upgrade `pickle` to 1.5 3fe6abde125b is described below commit 3fe6abde125b7c34437a3f72d17ee97d9653c218 Author: yangjie01 AuthorDate: Tue Jun 11 10:36:32 2024 +0800 [SPARK-48563][BUILD] Upgrade `pickle` to 1.5 ### What changes were proposed in this pull request? This pr aims upgrade `pickle` from 1.3 to 1.5. ### Why are the changes needed? The new version include a new fix related to [empty bytes object construction](https://github.com/irmen/pickle/commit/badc8fe08c9e47b87df66b8a16c67010e3614e35) All changes from 1.3 to 1.5 are as follows: - https://github.com/irmen/pickle/compare/pickle-1.3...pickle-1.5 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #46913 from LuciferYang/pickle-1.5. Authored-by: yangjie01 Signed-off-by: yangjie01 --- core/pom.xml | 2 +- dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 7413ad0d3393..adb1b3034b42 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -399,7 +399,7 @@ net.razorvine pickle - 1.3 + 1.5 net.sf.py4j diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 8ab76b5787b8..4585b534e908 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -241,7 +241,7 @@ parquet-encoding/1.13.1//parquet-encoding-1.13.1.jar parquet-format-structures/1.13.1//parquet-format-structures-1.13.1.jar parquet-hadoop/1.13.1//parquet-hadoop-1.13.1.jar parquet-jackson/1.13.1//parquet-jackson-1.13.1.jar -pickle/1.3//pickle-1.3.jar +pickle/1.5//pickle-1.5.jar py4j/0.10.9.7//py4j-0.10.9.7.jar remotetea-oncrpc/1.1.2//remotetea-oncrpc-1.1.2.jar rocksdbjni/8.11.4//rocksdbjni-8.11.4.jar - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48522][BUILD] Update Stream Library to 2.9.8 and attach its NOTICE
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 8b88f5ae10cc [SPARK-48522][BUILD] Update Stream Library to 2.9.8 and attach its NOTICE 8b88f5ae10cc is described below commit 8b88f5ae10cc676a9778c186b12c691fa913088d Author: Kent Yao AuthorDate: Tue Jun 4 21:33:01 2024 +0800 [SPARK-48522][BUILD] Update Stream Library to 2.9.8 and attach its NOTICE ### What changes were proposed in this pull request? Update Stream Library to 2.9.8 and attach its NOTICE ### Why are the changes needed? update dep and notice file ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? passing ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #46861 from yaooqinn/SPARK-48522. Authored-by: Kent Yao Signed-off-by: yangjie01 --- NOTICE-binary | 9 + dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- pom.xml | 2 +- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/NOTICE-binary b/NOTICE-binary index c82d0b52f31c..c4cfe0e9f8b3 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -33,11 +33,12 @@ services. // Version 2.0, in this case for // -- -Hive Beeline -Copyright 2016 The Apache Software Foundation +=== NOTICE FOR com.clearspring.analytics:streams === +stream-api +Copyright 2016 AddThis -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). +This product includes software developed by AddThis. +=== END OF NOTICE FOR com.clearspring.analytics:streams === Apache Avro Copyright 2009-2014 The Apache Software Foundation diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 3d8ffee05d3a..acb236e1c4e0 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -262,7 +262,7 @@ spire-platform_2.13/0.18.0//spire-platform_2.13-0.18.0.jar spire-util_2.13/0.18.0//spire-util_2.13-0.18.0.jar spire_2.13/0.18.0//spire_2.13-0.18.0.jar stax-api/1.0.1//stax-api-1.0.1.jar -stream/2.9.6//stream-2.9.6.jar +stream/2.9.8//stream-2.9.8.jar super-csv/2.2.0//super-csv-2.2.0.jar threeten-extra/1.7.1//threeten-extra-1.7.1.jar tink/1.13.0//tink-1.13.0.jar diff --git a/pom.xml b/pom.xml index ce3b4041ae57..bd384e42b0ec 100644 --- a/pom.xml +++ b/pom.xml @@ -806,7 +806,7 @@ com.clearspring.analytics stream -2.9.6 +2.9.8 - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48506][CORE] Compression codec short names are case insensitive except for event logging
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new f4afa2215a1a [SPARK-48506][CORE] Compression codec short names are case insensitive except for event logging f4afa2215a1a is described below commit f4afa2215a1a390d9f099a26155fbefc5beefbe9 Author: Kent Yao AuthorDate: Tue Jun 4 20:33:51 2024 +0800 [SPARK-48506][CORE] Compression codec short names are case insensitive except for event logging ### What changes were proposed in this pull request? Compression codec short names, e.g. map statuses, broadcasts, shuffle, parquet/orc/avro outputs, are case insensitive except for event logging. Calling `org.apache.spark.io.CompressionCodec.getShortName` causes this issue. In this PR, we make `CompressionCodec.getShortName` handle case sensitivity correctly. ### Why are the changes needed? Feature parity ### Does this PR introduce _any_ user-facing change? Yes, spark.eventLog.compression.codec now accepts not only the lowercased form of lz4, lzf, snappy, and zstd, but also forms with any of the characters to be upcased。 ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #46847 from yaooqinn/SPARK-48506. Authored-by: Kent Yao Signed-off-by: yangjie01 --- .../main/scala/org/apache/spark/io/CompressionCodec.scala | 5 +++-- .../scala/org/apache/spark/io/CompressionCodecSuite.scala | 15 +++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 7d5a86d1a81d..233228a9c6d4 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -101,8 +101,9 @@ private[spark] object CompressionCodec { * If it is already a short name, just return it. */ def getShortName(codecName: String): String = { -if (shortCompressionCodecNames.contains(codecName)) { - codecName +val lowercasedCodec = codecName.toLowerCase(Locale.ROOT) +if (shortCompressionCodecNames.contains(lowercasedCodec)) { + lowercasedCodec } else { shortCompressionCodecNames .collectFirst { case (k, v) if v == codecName => k } diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala index 729fcecff120..5c09a1f965b9 100644 --- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.io import java.io.{ByteArrayInputStream, ByteArrayOutputStream} +import java.util.Locale import com.google.common.io.ByteStreams @@ -160,4 +161,18 @@ class CompressionCodecSuite extends SparkFunSuite { ByteStreams.readFully(concatenatedBytes, decompressed) assert(decompressed.toSeq === (0 to 127)) } + + test("SPARK-48506: CompressionCodec getShortName is case insensitive for short names") { +CompressionCodec.shortCompressionCodecNames.foreach { case (shortName, codecClass) => + assert(CompressionCodec.getShortName(shortName) === shortName) + assert(CompressionCodec.getShortName(shortName.toUpperCase(Locale.ROOT)) === shortName) + assert(CompressionCodec.getShortName(codecClass) === shortName) + checkError( +exception = intercept[SparkIllegalArgumentException] { + CompressionCodec.getShortName(codecClass.toUpperCase(Locale.ROOT)) +}, +errorClass = "CODEC_SHORT_NAME_NOT_FOUND", +parameters = Map("codecName" -> codecClass.toUpperCase(Locale.ROOT))) +} + } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48519][BUILD] Upgrade jetty to 11.0.21
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new d273fdf37bc2 [SPARK-48519][BUILD] Upgrade jetty to 11.0.21 d273fdf37bc2 is described below commit d273fdf37bc291aadf8677305bda2a91b593219f Author: yangjie01 AuthorDate: Tue Jun 4 19:08:40 2024 +0800 [SPARK-48519][BUILD] Upgrade jetty to 11.0.21 ### What changes were proposed in this pull request? This pr aims to upgrade jetty from 11.0.20 to 11.0.21. ### Why are the changes needed? The new version bring some bug fix like [Reduce ByteBuffer churning in HttpOutput](https://github.com/jetty/jetty.project/commit/fe94c9f8a40df49021b28280f708448870c5b420). The full release notes as follows: - https://github.com/jetty/jetty.project/releases/tag/jetty-11.0.21 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #46843 from LuciferYang/jetty-11.0.21. Authored-by: yangjie01 Signed-off-by: yangjie01 --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 4 ++-- pom.xml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 65e627b1854f..3d8ffee05d3a 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -137,8 +137,8 @@ jersey-container-servlet/3.0.12//jersey-container-servlet-3.0.12.jar jersey-hk2/3.0.12//jersey-hk2-3.0.12.jar jersey-server/3.0.12//jersey-server-3.0.12.jar jettison/1.5.4//jettison-1.5.4.jar -jetty-util-ajax/11.0.20//jetty-util-ajax-11.0.20.jar -jetty-util/11.0.20//jetty-util-11.0.20.jar +jetty-util-ajax/11.0.21//jetty-util-ajax-11.0.21.jar +jetty-util/11.0.21//jetty-util-11.0.21.jar jline/2.14.6//jline-2.14.6.jar jline/3.25.1//jline-3.25.1.jar jna/5.14.0//jna-5.14.0.jar diff --git a/pom.xml b/pom.xml index ded8cc2405fd..ce3b4041ae57 100644 --- a/pom.xml +++ b/pom.xml @@ -140,7 +140,7 @@ 1.13.1 2.0.1 shaded-protobuf -11.0.20 +11.0.21 5.0.0 4.0.1 - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (8d534c048866 -> 9270931221d4)
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 8d534c048866 [SPARK-48487][INFRA] Update License & Notice according to the dependency changes add 9270931221d4 [SPARK-48433][BUILD] Upgrade `checkstyle` to 10.17.0 No new revisions were added by this update. Summary of changes: pom.xml | 2 +- project/plugins.sbt | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch branch-3.4 updated: [SPARK-48484][SQL] Fix: V2Write use the same TaskAttemptId for different task attempts
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch branch-3.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.4 by this push: new 11c06fcbf2e6 [SPARK-48484][SQL] Fix: V2Write use the same TaskAttemptId for different task attempts 11c06fcbf2e6 is described below commit 11c06fcbf2e62e870c758cedcd386ba2d539352d Author: jackylee-ch AuthorDate: Fri May 31 22:37:49 2024 +0800 [SPARK-48484][SQL] Fix: V2Write use the same TaskAttemptId for different task attempts ### What changes were proposed in this pull request? After #40064 , we always get the same TaskAttemptId for different task attempts which has the same partitionId. This would lead different task attempts write to the same directory. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? GA ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46811 from jackylee-ch/fix_v2write_use_same_directories_for_different_task_attempts. Lead-authored-by: jackylee-ch Co-authored-by: Kent Yao Signed-off-by: yangjie01 (cherry picked from commit 67d11b1992aaa100d0e1fa30b0e5c33684c93a89) Signed-off-by: yangjie01 --- .../datasources/v2/FileWriterFactory.scala | 8 ++-- .../datasources/v2/FileWriterFactorySuite.scala| 48 ++ 2 files changed, 53 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala index 4b1a099d3bac..f18424b4bcb8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala @@ -38,7 +38,7 @@ case class FileWriterFactory ( @transient private lazy val jobId = SparkHadoopWriterUtils.createJobID(jobTrackerID, 0) override def createWriter(partitionId: Int, realTaskId: Long): DataWriter[InternalRow] = { -val taskAttemptContext = createTaskAttemptContext(partitionId) +val taskAttemptContext = createTaskAttemptContext(partitionId, realTaskId.toInt & Int.MaxValue) committer.setupTask(taskAttemptContext) if (description.partitionColumns.isEmpty) { new SingleDirectoryDataWriter(description, taskAttemptContext, committer) @@ -47,9 +47,11 @@ case class FileWriterFactory ( } } - private def createTaskAttemptContext(partitionId: Int): TaskAttemptContextImpl = { + private def createTaskAttemptContext( + partitionId: Int, + realTaskId: Int): TaskAttemptContextImpl = { val taskId = new TaskID(jobId, TaskType.MAP, partitionId) -val taskAttemptId = new TaskAttemptID(taskId, 0) +val taskAttemptId = new TaskAttemptID(taskId, realTaskId) // Set up the configuration object val hadoopConf = description.serializableHadoopConf.value hadoopConf.set("mapreduce.job.id", jobId.toString) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactorySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactorySuite.scala new file mode 100644 index ..bd2030797441 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactorySuite.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.mockito.Mockito._ +import org.scalatest.PrivateMethodTester + +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.io.FileCommitProtocol +import org.apache.spark.sql.execution.datasources.WriteJobDescription +import org.apache.spark.util.SerializableConfiguration + +class FileWriterFactorySuite extends SparkFunSuite with PrivateMethodTester
(spark) branch branch-3.5 updated: [SPARK-48484][SQL] Fix: V2Write use the same TaskAttemptId for different task attempts
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 7d39000f809a [SPARK-48484][SQL] Fix: V2Write use the same TaskAttemptId for different task attempts 7d39000f809a is described below commit 7d39000f809a117d2ef9e73e46697704e45ba262 Author: jackylee-ch AuthorDate: Fri May 31 22:37:49 2024 +0800 [SPARK-48484][SQL] Fix: V2Write use the same TaskAttemptId for different task attempts ### What changes were proposed in this pull request? After #40064 , we always get the same TaskAttemptId for different task attempts which has the same partitionId. This would lead different task attempts write to the same directory. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? GA ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46811 from jackylee-ch/fix_v2write_use_same_directories_for_different_task_attempts. Lead-authored-by: jackylee-ch Co-authored-by: Kent Yao Signed-off-by: yangjie01 (cherry picked from commit 67d11b1992aaa100d0e1fa30b0e5c33684c93a89) Signed-off-by: yangjie01 --- .../datasources/v2/FileWriterFactory.scala | 8 ++-- .../datasources/v2/FileWriterFactorySuite.scala| 48 ++ 2 files changed, 53 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala index 4b1a099d3bac..f18424b4bcb8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala @@ -38,7 +38,7 @@ case class FileWriterFactory ( @transient private lazy val jobId = SparkHadoopWriterUtils.createJobID(jobTrackerID, 0) override def createWriter(partitionId: Int, realTaskId: Long): DataWriter[InternalRow] = { -val taskAttemptContext = createTaskAttemptContext(partitionId) +val taskAttemptContext = createTaskAttemptContext(partitionId, realTaskId.toInt & Int.MaxValue) committer.setupTask(taskAttemptContext) if (description.partitionColumns.isEmpty) { new SingleDirectoryDataWriter(description, taskAttemptContext, committer) @@ -47,9 +47,11 @@ case class FileWriterFactory ( } } - private def createTaskAttemptContext(partitionId: Int): TaskAttemptContextImpl = { + private def createTaskAttemptContext( + partitionId: Int, + realTaskId: Int): TaskAttemptContextImpl = { val taskId = new TaskID(jobId, TaskType.MAP, partitionId) -val taskAttemptId = new TaskAttemptID(taskId, 0) +val taskAttemptId = new TaskAttemptID(taskId, realTaskId) // Set up the configuration object val hadoopConf = description.serializableHadoopConf.value hadoopConf.set("mapreduce.job.id", jobId.toString) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactorySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactorySuite.scala new file mode 100644 index ..bd2030797441 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactorySuite.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.mockito.Mockito._ +import org.scalatest.PrivateMethodTester + +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.io.FileCommitProtocol +import org.apache.spark.sql.execution.datasources.WriteJobDescription +import org.apache.spark.util.SerializableConfiguration + +class FileWriterFactorySuite extends SparkFunSuite with PrivateMethodTester
(spark) branch master updated: [SPARK-48484][SQL] Fix: V2Write use the same TaskAttemptId for different task attempts
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 67d11b1992aa [SPARK-48484][SQL] Fix: V2Write use the same TaskAttemptId for different task attempts 67d11b1992aa is described below commit 67d11b1992aaa100d0e1fa30b0e5c33684c93a89 Author: jackylee-ch AuthorDate: Fri May 31 22:37:49 2024 +0800 [SPARK-48484][SQL] Fix: V2Write use the same TaskAttemptId for different task attempts ### What changes were proposed in this pull request? After #40064 , we always get the same TaskAttemptId for different task attempts which has the same partitionId. This would lead different task attempts write to the same directory. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? GA ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46811 from jackylee-ch/fix_v2write_use_same_directories_for_different_task_attempts. Lead-authored-by: jackylee-ch Co-authored-by: Kent Yao Signed-off-by: yangjie01 --- .../datasources/v2/FileWriterFactory.scala | 8 ++-- .../datasources/v2/FileWriterFactorySuite.scala| 48 ++ 2 files changed, 53 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala index 4b1a099d3bac..f18424b4bcb8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala @@ -38,7 +38,7 @@ case class FileWriterFactory ( @transient private lazy val jobId = SparkHadoopWriterUtils.createJobID(jobTrackerID, 0) override def createWriter(partitionId: Int, realTaskId: Long): DataWriter[InternalRow] = { -val taskAttemptContext = createTaskAttemptContext(partitionId) +val taskAttemptContext = createTaskAttemptContext(partitionId, realTaskId.toInt & Int.MaxValue) committer.setupTask(taskAttemptContext) if (description.partitionColumns.isEmpty) { new SingleDirectoryDataWriter(description, taskAttemptContext, committer) @@ -47,9 +47,11 @@ case class FileWriterFactory ( } } - private def createTaskAttemptContext(partitionId: Int): TaskAttemptContextImpl = { + private def createTaskAttemptContext( + partitionId: Int, + realTaskId: Int): TaskAttemptContextImpl = { val taskId = new TaskID(jobId, TaskType.MAP, partitionId) -val taskAttemptId = new TaskAttemptID(taskId, 0) +val taskAttemptId = new TaskAttemptID(taskId, realTaskId) // Set up the configuration object val hadoopConf = description.serializableHadoopConf.value hadoopConf.set("mapreduce.job.id", jobId.toString) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactorySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactorySuite.scala new file mode 100644 index ..bd2030797441 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactorySuite.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.mockito.Mockito._ +import org.scalatest.PrivateMethodTester + +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.io.FileCommitProtocol +import org.apache.spark.sql.execution.datasources.WriteJobDescription +import org.apache.spark.util.SerializableConfiguration + +class FileWriterFactorySuite extends SparkFunSuite with PrivateMethodTester { + + test("SPARK-48484: V2Write uses different TaskAttemptIds for different task attempts&quo
(spark) branch master updated: [SPARK-47361][SQL] Derby: Calculate suitable precision and scale for DECIMAL type
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 69afd4be9c93 [SPARK-47361][SQL] Derby: Calculate suitable precision and scale for DECIMAL type 69afd4be9c93 is described below commit 69afd4be9c93cb31a840b969ed1984c0b6b92f8e Author: Kent Yao AuthorDate: Thu May 30 17:31:28 2024 +0800 [SPARK-47361][SQL] Derby: Calculate suitable precision and scale for DECIMAL type ### What changes were proposed in this pull request? When storing `decimal(p, s)` to derby, if `p > 31`, `s` is wrongly hardcoded to `5` which is the assumed default scale of derby decimal. Actually, 0 is the default scale, 5 is the default precision https://db.apache.org/derby/docs/10.13/ref/rrefsqlj15260.html This PR calculates a suitable scale to make room for precision. ### Why are the changes needed? avoid precision loss ### Does this PR introduce _any_ user-facing change? Yes, but derby is rare in production environments, and the new mapping are compatible for most usecases ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #46776 from yaooqinn/SPARK-48439. Authored-by: Kent Yao Signed-off-by: yangjie01 --- .../main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala | 12 +--- .../datasources/v2/jdbc/DerbyTableCatalogSuite.scala | 8 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala index 36af0e6aeaf1..23da4dbb60a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala @@ -48,9 +48,15 @@ private case class DerbyDialect() extends JdbcDialect { case ByteType => Option(JdbcType("SMALLINT", java.sql.Types.SMALLINT)) case ShortType => Option(JdbcType("SMALLINT", java.sql.Types.SMALLINT)) case BooleanType => Option(JdbcType("BOOLEAN", java.sql.Types.BOOLEAN)) -// 31 is the maximum precision and 5 is the default scale for a Derby DECIMAL -case t: DecimalType if t.precision > 31 => - Option(JdbcType("DECIMAL(31,5)", java.sql.Types.DECIMAL)) +// 31 is the maximum precision +// https://db.apache.org/derby/docs/10.13/ref/rrefsqlj15260.html +case t: DecimalType => + val (p, s) = if (t.precision > 31) { +(31, math.max(t.scale - (t.precision - 31), 0)) + } else { +(t.precision, t.scale) + } + Option(JdbcType(s"DECIMAL($p,$s)", java.sql.Types.DECIMAL)) case _ => None } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DerbyTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DerbyTableCatalogSuite.scala index e3714e604495..d793ef526c47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DerbyTableCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DerbyTableCatalogSuite.scala @@ -51,4 +51,12 @@ class DerbyTableCatalogSuite extends QueryTest with SharedSparkSession { checkAnswer(sql(s"SHOW TABLES IN derby.test1"), Row("test1", "TABLE2", false)) } } + + test("SPARK-48439: Calculate suitable precision and scale for DECIMAL type") { +withTable("derby.test1.table1") { + sql("CREATE TABLE derby.test1.table1 (c1 decimal(38, 18))") + sql("INSERT INTO derby.test1.table1 VALUES (1.123456789123456789)") + checkAnswer(sql("SELECT * FROM derby.test1.table1"), Row(1.12345678912)) +} + } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48420][BUILD] Upgrade netty to `4.1.110.Final`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new a88cc1ad9319 [SPARK-48420][BUILD] Upgrade netty to `4.1.110.Final` a88cc1ad9319 is described below commit a88cc1ad9319bd0f4a14e2d6094865229449c8cb Author: panbingkun AuthorDate: Tue May 28 13:09:39 2024 +0800 [SPARK-48420][BUILD] Upgrade netty to `4.1.110.Final` ### What changes were proposed in this pull request? The pr aims to upgrade `netty` from `4.1.109.Final` to `4.1.110.Final`. ### Why are the changes needed? - https://netty.io/news/2024/05/22/4-1-110-Final.html This version has brought some bug fixes and improvements, such as: Fix Zstd throws Exception on read-only volumes (https://github.com/netty/netty/pull/13982) Add unix domain socket transport in netty 4.x via JDK16+ ([#13965](https://github.com/netty/netty/pull/13965)) Backport #13075: Add the AdaptivePoolingAllocator ([#13976](https://github.com/netty/netty/pull/13976)) Add no-value key handling only for form body ([#13998](https://github.com/netty/netty/pull/13998)) Add support for specifying SecureRandom in SSLContext initialization ([#14058](https://github.com/netty/netty/pull/14058)) - https://github.com/netty/netty/issues?q=milestone%3A4.1.110.Final+is%3Aclosed ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46744 from panbingkun/SPARK-48420. Authored-by: panbingkun Signed-off-by: yangjie01 --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 38 +-- pom.xml | 2 +- 2 files changed, 20 insertions(+), 20 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 10d812c9fd8a..e854bd0e804a 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -197,31 +197,31 @@ metrics-jmx/4.2.25//metrics-jmx-4.2.25.jar metrics-json/4.2.25//metrics-json-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar minlog/1.3.0//minlog-1.3.0.jar -netty-all/4.1.109.Final//netty-all-4.1.109.Final.jar -netty-buffer/4.1.109.Final//netty-buffer-4.1.109.Final.jar -netty-codec-http/4.1.109.Final//netty-codec-http-4.1.109.Final.jar -netty-codec-http2/4.1.109.Final//netty-codec-http2-4.1.109.Final.jar -netty-codec-socks/4.1.109.Final//netty-codec-socks-4.1.109.Final.jar -netty-codec/4.1.109.Final//netty-codec-4.1.109.Final.jar -netty-common/4.1.109.Final//netty-common-4.1.109.Final.jar -netty-handler-proxy/4.1.109.Final//netty-handler-proxy-4.1.109.Final.jar -netty-handler/4.1.109.Final//netty-handler-4.1.109.Final.jar -netty-resolver/4.1.109.Final//netty-resolver-4.1.109.Final.jar +netty-all/4.1.110.Final//netty-all-4.1.110.Final.jar +netty-buffer/4.1.110.Final//netty-buffer-4.1.110.Final.jar +netty-codec-http/4.1.110.Final//netty-codec-http-4.1.110.Final.jar +netty-codec-http2/4.1.110.Final//netty-codec-http2-4.1.110.Final.jar +netty-codec-socks/4.1.110.Final//netty-codec-socks-4.1.110.Final.jar +netty-codec/4.1.110.Final//netty-codec-4.1.110.Final.jar +netty-common/4.1.110.Final//netty-common-4.1.110.Final.jar +netty-handler-proxy/4.1.110.Final//netty-handler-proxy-4.1.110.Final.jar +netty-handler/4.1.110.Final//netty-handler-4.1.110.Final.jar +netty-resolver/4.1.110.Final//netty-resolver-4.1.110.Final.jar netty-tcnative-boringssl-static/2.0.65.Final/linux-aarch_64/netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar netty-tcnative-boringssl-static/2.0.65.Final/linux-x86_64/netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar netty-tcnative-boringssl-static/2.0.65.Final/osx-aarch_64/netty-tcnative-boringssl-static-2.0.65.Final-osx-aarch_64.jar netty-tcnative-boringssl-static/2.0.65.Final/osx-x86_64/netty-tcnative-boringssl-static-2.0.65.Final-osx-x86_64.jar netty-tcnative-boringssl-static/2.0.65.Final/windows-x86_64/netty-tcnative-boringssl-static-2.0.65.Final-windows-x86_64.jar netty-tcnative-classes/2.0.65.Final//netty-tcnative-classes-2.0.65.Final.jar -netty-transport-classes-epoll/4.1.109.Final//netty-transport-classes-epoll-4.1.109.Final.jar -netty-transport-classes-kqueue/4.1.109.Final//netty-transport-classes-kqueue-4.1.109.Final.jar -netty-transport-native-epoll/4.1.109.Final/linux-aarch_64/netty-transport-native-epoll-4.1.109.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.109.Final/linux-riscv64/netty-transport-native-epoll-4.1.109.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.109.Final/linux-x86_64/netty-transport-native-epoll-4.1.109.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.109.Final/osx-aarch_64/netty-transport
(spark) branch master updated (3346afd4b250 -> ef43bbbc1163)
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 3346afd4b250 [SPARK-46090][SQL][FOLLOWUP] Add DeveloperApi import add ef43bbbc1163 [SPARK-48384][BUILD] Exclude `io.netty:netty-tcnative-boringssl-static` from `zookeeper` No new revisions were added by this update. Summary of changes: dev/deps/spark-deps-hadoop-3-hive-2.3 | 1 - pom.xml | 4 2 files changed, 4 insertions(+), 1 deletion(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48386][TESTS] Replace JVM assert with JUnit Assert in tests
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 5df9a0866ae6 [SPARK-48386][TESTS] Replace JVM assert with JUnit Assert in tests 5df9a0866ae6 is described below commit 5df9a0866ae60a42d78136a21a82a0b6e58daefa Author: panbingkun AuthorDate: Thu May 23 10:46:08 2024 +0800 [SPARK-48386][TESTS] Replace JVM assert with JUnit Assert in tests ### What changes were proposed in this pull request? The pr aims to replace `JVM assert` with `JUnit Assert` in tests. ### Why are the changes needed? assert() statements do not produce as useful errors when they fail, and, if they were somehow disabled, would fail to test anything. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Manually test. - Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46698 from panbingkun/minor_assert. Authored-by: panbingkun Signed-off-by: yangjie01 --- .../protocol/EncryptedMessageWithHeaderSuite.java | 2 +- .../shuffle/RetryingBlockTransferorSuite.java | 8 +++--- .../apache/spark/util/SparkLoggerSuiteBase.java| 30 -- .../apache/spark/sql/TestStatefulProcessor.java| 10 +--- .../sql/TestStatefulProcessorWithInitialState.java | 4 ++- .../JavaAdvancedDataSourceV2WithV2Filter.java | 14 +- 6 files changed, 38 insertions(+), 30 deletions(-) diff --git a/common/network-common/src/test/java/org/apache/spark/network/protocol/EncryptedMessageWithHeaderSuite.java b/common/network-common/src/test/java/org/apache/spark/network/protocol/EncryptedMessageWithHeaderSuite.java index 7478fa1db711..2865d411bf67 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/protocol/EncryptedMessageWithHeaderSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/protocol/EncryptedMessageWithHeaderSuite.java @@ -116,7 +116,7 @@ public class EncryptedMessageWithHeaderSuite { // Validate we read data correctly assertEquals(bodyResult.readableBytes(), chunkSize); - assert(bodyResult.readableBytes() < (randomData.length - readIndex)); + assertTrue(bodyResult.readableBytes() < (randomData.length - readIndex)); while (bodyResult.readableBytes() > 0) { assertEquals(bodyResult.readByte(), randomData[readIndex++]); } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockTransferorSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockTransferorSuite.java index 3725973ae733..84c8b1b3353f 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockTransferorSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockTransferorSuite.java @@ -288,7 +288,7 @@ public class RetryingBlockTransferorSuite { verify(listener, timeout(5000)).onBlockTransferSuccess("b0", block0); verify(listener).getTransferType(); verifyNoMoreInteractions(listener); -assert(_retryingBlockTransferor.getRetryCount() == 0); +assertEquals(0, _retryingBlockTransferor.getRetryCount()); } @Test @@ -310,7 +310,7 @@ public class RetryingBlockTransferorSuite { verify(listener, timeout(5000)).onBlockTransferFailure("b0", saslTimeoutException); verify(listener, times(3)).getTransferType(); verifyNoMoreInteractions(listener); -assert(_retryingBlockTransferor.getRetryCount() == MAX_RETRIES); +assertEquals(MAX_RETRIES, _retryingBlockTransferor.getRetryCount()); } @Test @@ -339,7 +339,7 @@ public class RetryingBlockTransferorSuite { // This should be equal to 1 because after the SASL exception is retried, // retryCount should be set back to 0. Then after that b1 encounters an // exception that is retried. -assert(_retryingBlockTransferor.getRetryCount() == 1); +assertEquals(1, _retryingBlockTransferor.getRetryCount()); } @Test @@ -368,7 +368,7 @@ public class RetryingBlockTransferorSuite { verify(listener, timeout(5000)).onBlockTransferFailure("b0", saslExceptionFinal); verify(listener, atLeastOnce()).getTransferType(); verifyNoMoreInteractions(listener); -assert(_retryingBlockTransferor.getRetryCount() == MAX_RETRIES); +assertEquals(MAX_RETRIES, _retryingBlockTransferor.getRetryCount()); } @Test diff --git a/common/utils/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java b/common/utils/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java index 46bfe3415080..0869f9827324 100644 --- a/common/utils/src/test/java/org/apache/spark/util/SparkLogg
(spark) branch master updated: [SPARK-48238][BUILD][YARN] Replace YARN AmIpFilter with a forked implementation
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 4fc2910f92d1 [SPARK-48238][BUILD][YARN] Replace YARN AmIpFilter with a forked implementation 4fc2910f92d1 is described below commit 4fc2910f92d1b5f7e0dd5f803e822668f23c21c5 Author: Cheng Pan AuthorDate: Mon May 20 20:42:57 2024 +0800 [SPARK-48238][BUILD][YARN] Replace YARN AmIpFilter with a forked implementation ### What changes were proposed in this pull request? This PR replaces AmIpFilter with a forked implementation, and removes the dependency `hadoop-yarn-server-web-proxy` ### Why are the changes needed? SPARK-47118 upgraded Spark built-in Jetty from 10 to 11, and migrated from `javax.servlet` to `jakarta.servlet`, which breaks the Spark on YARN. ``` Caused by: java.lang.IllegalStateException: class org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter is not a jakarta.servlet.Filter at org.sparkproject.jetty.servlet.FilterHolder.doStart(FilterHolder.java:99) at org.sparkproject.jetty.util.component.AbstractLifeCycle.start(AbstractLifeCycle.java:93) at org.sparkproject.jetty.servlet.ServletHandler.lambda$initialize$2(ServletHandler.java:724) at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625) at java.base/java.util.stream.Streams$ConcatSpliterator.forEachRemaining(Streams.java:734) at java.base/java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:762) at org.sparkproject.jetty.servlet.ServletHandler.initialize(ServletHandler.java:749) ... 38 more ``` During the investigation, I found a comment here https://github.com/apache/spark/pull/31642#issuecomment-786257114 > Agree that in the long term we should either: 1) consider to re-implement the logic in Spark which allows us to get away from server-side dependency in Hadoop ... This should be a simple and clean way to address the exact issue, then we don't need to wait for Hadoop `jakarta.servlet` migration, and it also strips a Hadoop dependency. ### Does this PR introduce _any_ user-facing change? No, this recovers the bootstrap of the Spark application on YARN mode, keeping the same behavior with Spark 3.5 and earlier versions. ### How was this patch tested? UTs are added. (refer to `org.apache.hadoop.yarn.server.webproxy.amfilter.TestAmFilter`) I tested it in a YARN cluster. Spark successfully started. ``` roothadoop-master1:/opt/spark-SPARK-48238# JAVA_HOME=/opt/openjdk-17 bin/spark-sql --conf spark.yarn.appMasterEnv.JAVA_HOME=/opt/openjdk-17 --conf spark.executorEnv.JAVA_HOME=/opt/openjdk-17 WARNING: Using incubator modules: jdk.incubator.vector Setting default log level to "WARN". To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel). 2024-05-18 04:11:36 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable 2024-05-18 04:11:44 WARN Client: Neither spark.yarn.jars nor spark.yarn.archive} is set, falling back to uploading libraries under SPARK_HOME. Spark Web UI available at http://hadoop-master1.orb.local:4040 Spark master: yarn, Application Id: application_1716005503866_0001 spark-sql (default)> select version(); 4.0.0 4ddc2303c7cbabee12a3de9f674aaacad3f5eb01 Time taken: 1.707 seconds, Fetched 1 row(s) spark-sql (default)> ``` When access `http://hadoop-master1.orb.local:4040`, it redirects to `http://hadoop-master1.orb.local:8088/proxy/redirect/application_1716005503866_0001/`, and the UI looks correct. https://github.com/apache/spark/assets/26535726/8500fc83-48c5-4603-8d05-37855f0308ae;> ### Was this patch authored or co-authored using generative AI tooling? No Closes #46611 from pan3793/SPARK-48238. Authored-by: Cheng Pan Signed-off-by: yangjie01 --- assembly/pom.xml | 4 - dev/deps/spark-deps-hadoop-3-hive-2.3 | 1 - pom.xml| 77 - .../org/apache/spark/deploy/yarn/AmIpFilter.java | 239 ++ .../apache/spark/deploy/yarn/AmIpPrincipal.java| 35 +++ .../deploy/yarn/AmIpServletRequestWrapper.java | 54 .../org/apache/spark/deploy/yarn/ProxyUtils.java | 126 .../spark/deploy/yarn/ApplicationMaster.scala | 2 +- .../apache/spark/deploy/yarn/AmIpFilterSuite.scala | 342 + .../org/apache/spark/streaming/Checkpoint.scala| 2 +- 10 files changed, 798 insertions(+), 84 deletions(-) diff --git a/assem
(spark) branch master updated: [SPARK-48242][BUILD] Upgrade extra-enforcer-rules to 1.8.0
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 2eea28b5efd4 [SPARK-48242][BUILD] Upgrade extra-enforcer-rules to 1.8.0 2eea28b5efd4 is described below commit 2eea28b5efd4ae30d7962c92b2f9851cf3938b5e Author: panbingkun AuthorDate: Mon May 20 17:23:59 2024 +0800 [SPARK-48242][BUILD] Upgrade extra-enforcer-rules to 1.8.0 ### What changes were proposed in this pull request? The pr aims to upgrade `extra-enforcer-rules to 1.8.0` from `1.7.0` to `1.8.0`. ### Why are the changes needed? The full release notes: https://github.com/mojohaus/extra-enforcer-rules/releases/tag/1.8.0 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Manually test. ``` sh dev/test-dependencies.sh ``` - Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46538 from panbingkun/SPARK-48242. Authored-by: panbingkun Signed-off-by: yangjie01 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 611e82f343d8..5811e5b7716d 100644 --- a/pom.xml +++ b/pom.xml @@ -3008,7 +3008,7 @@ org.codehaus.mojo extra-enforcer-rules - 1.7.0 + 1.8.0 - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48274][BUILD] Upgrade GenJavadoc to `0.19`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 7441bd0197e6 [SPARK-48274][BUILD] Upgrade GenJavadoc to `0.19` 7441bd0197e6 is described below commit 7441bd0197e6442c4f98481bf2fb23b49b5f75cf Author: panbingkun AuthorDate: Wed May 15 10:22:34 2024 +0800 [SPARK-48274][BUILD] Upgrade GenJavadoc to `0.19` ### What changes were proposed in this pull request? This PR upgrades `GenJavadoc` plugin from `0.18` to `0.19`. ### Why are the changes needed? 1.The full release notes: https://github.com/lightbend/genjavadoc/releases/tag/v0.19 2.The latest version supports scala `2.13.14`, which is a `prerequisite` for us to upgrade spark's scala `2.13.14`. https://mvnrepository.com/artifact/com.typesafe.genjavadoc/genjavadoc-plugin 3.The last upgrade occurred 3 years ago https://github.com/apache/spark/pull/33383 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Built the doc: ``` ./build/sbt -Phadoop-3 -Pkubernetes -Pkinesis-asl -Phive-thriftserver -Pdocker-integration-tests -Pyarn -Phadoop-cloud -Pspark-ganglia-lgpl -Phive -Pjvm-profiler unidoc ``` https://github.com/apache/spark/assets/15246973/58d3fac8-c968-44e0-83f3-84cf00a5084f;> ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46579 from panbingkun/unidocGenjavadocVersion_0_19. Authored-by: panbingkun Signed-off-by: yangjie01 --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5bb7745d77bf..d1b0ed953e30 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -266,7 +266,7 @@ object SparkBuild extends PomBuild { .orElse(sys.props.get("java.home").map { p => new File(p).getParentFile().getAbsolutePath() }) .map(file), publishMavenStyle := true, -unidocGenjavadocVersion := "0.18", +unidocGenjavadocVersion := "0.19", // Override SBT's default resolvers: resolvers := Seq( - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48257][BUILD] Polish POM for Hive dependencies
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c4c9ccbdf562 [SPARK-48257][BUILD] Polish POM for Hive dependencies c4c9ccbdf562 is described below commit c4c9ccbdf562b5da6066d6cd0517ab27bf9de3fa Author: Cheng Pan AuthorDate: Mon May 13 21:59:14 2024 +0800 [SPARK-48257][BUILD] Polish POM for Hive dependencies ### What changes were proposed in this pull request? 1. `org.apache.hive` and `${hive.group}` co-exists in `pom.xml`, this PR unifies them to `${hive.group}` 2. `hive23.version`, `hive.version.short`, `` were used in Spark 3.0 period to distinguish hive 1.2 and hive 2.3, which are useless today, this PR removes those outdated definitions. 3. update/remove some outdated comments. e.g. remove the comment for Hive LOG4J exclusion because Spark already switched to LOG4J2, generalize the comments for Hive Parquet/Jetty exclusion ### Why are the changes needed? Cleanup POM. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass CI. ### Was this patch authored or co-authored using generative AI tooling? No Closes #46558 from pan3793/SPARK-48257. Authored-by: Cheng Pan Signed-off-by: yangjie01 --- pom.xml | 48 ++-- sql/core/pom.xml | 2 +- sql/hive/pom.xml | 4 ++-- 3 files changed, 21 insertions(+), 33 deletions(-) diff --git a/pom.xml b/pom.xml index ad6e9391b68c..12d20f4f0736 100644 --- a/pom.xml +++ b/pom.xml @@ -133,9 +133,6 @@ core 2.3.10 -2.3.10 - -2.3 3.7.0 @@ -2112,7 +2109,6 @@ commons-logging commons-logging - + org.eclipse.jetty.aggregate jetty-all - org.apache.logging.log4j * @@ -2139,10 +2134,9 @@ -org.apache.hive +${hive.group} hive-storage-api - @@ -2261,7 +2255,6 @@ org.json json - ${hive.group} @@ -2276,7 +2269,6 @@ org.apache.calcite.avatica avatica - org.apache.logging.log4j * @@ -2297,7 +2289,6 @@ net.hydromatic aggdesigner-algorithm - @@ -2410,7 +2401,6 @@ org.slf4j slf4j-log4j12 - org.apache.hbase @@ -2420,7 +2410,6 @@ co.cask.tephra * - @@ -2478,12 +2467,14 @@ org.codehaus.groovy groovy-all - ${hive.group} hive-service-rpc - + org.apache.parquet parquet-hadoop-bundle @@ -2497,7 +2488,6 @@ tomcat jasper-runtime - @@ -2574,30 +2564,28 @@ org.codehaus.groovy groovy-all - org.apache.logging.log4j log4j-slf4j-impl - -org.apache.hive +${hive.group} hive-llap-common -${hive23.version} +${hive.version} ${hive.deps.scope} -org.apache.hive +${hive.group} hive-common -org.apache.hive +${hive.group} hive-serde @@ -2608,21 +2596,21 @@ -org.apache.hive +${hive.group} hive-llap-client -${hive23.version} +${hive.version} test -org.apache.hive +${hive.group} hive-common -org.apache.hive +${hive.group} hive-serde -org.apache.hive +${hive.group} hive-llap-common @@ -2683,7 +2671,7 @@ hadoop-client-api -org.apache.hive +${hive.group} hive-storage-api @@ -2713,7 +2701,7 @@ orc-core -org.apache.hive +${hive.group} hive-storage-api @@ -2902,7
(spark) branch master updated (cae2248bc13d -> acc37531deb9)
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from cae2248bc13d [MINOR][PYTHON][TESTS] Move test `test_named_arguments_negative` to `test_arrow_python_udf` add acc37531deb9 [SPARK-47993][PYTHON][FOLLOW-UP] Update migration guide about Python 3.8 dropped No new revisions were added by this update. Summary of changes: python/docs/source/migration_guide/pyspark_upgrade.rst | 1 + 1 file changed, 1 insertion(+) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48138][CONNECT][TESTS] Disable a flaky `SparkSessionE2ESuite.interrupt tag` test
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 8294c5962feb [SPARK-48138][CONNECT][TESTS] Disable a flaky `SparkSessionE2ESuite.interrupt tag` test 8294c5962feb is described below commit 8294c5962febe53eebdff79f65f5f293d93a1997 Author: Dongjoon Hyun AuthorDate: Mon May 6 13:45:54 2024 +0800 [SPARK-48138][CONNECT][TESTS] Disable a flaky `SparkSessionE2ESuite.interrupt tag` test ### What changes were proposed in this pull request? This PR aims to disable a flaky test, `SparkSessionE2ESuite.interrupt tag`, temporarily. To re-enable this, SPARK-48139 is created as a blocker issue for 4.0.0. ### Why are the changes needed? This test case was added at `Apache Spark 3.5.0` but has been unstable unfortunately until now. - #42009 We tried to stabilize this test case before `Apache Spark 4.0.0-preview`. - #45173 - #46374 However, it's still flaky. - https://github.com/apache/spark/actions/runs/8962353911/job/24611130573 (Master, 2024-05-05) - https://github.com/apache/spark/actions/runs/8948176536/job/24581022674 (Master, 2024-05-04) This PR aims to stablize CI first and to focus this flaky issue as a blocker level before going on `Spark Connect GA` in SPARK-48139 before Apache Spark 4.0.0. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46396 from dongjoon-hyun/SPARK-48138. Authored-by: Dongjoon Hyun Signed-off-by: yangjie01 --- .../jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala index d1015d55b1df..f56085191f87 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala @@ -108,7 +108,8 @@ class SparkSessionE2ESuite extends RemoteSparkSession { assert(interrupted.length == 2, s"Interrupted operations: $interrupted.") } - test("interrupt tag") { + // TODO(SPARK-48139): Re-enable `SparkSessionE2ESuite.interrupt tag` + ignore("interrupt tag") { val session = spark import session.implicits._ - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-48021][ML][BUILD][FOLLOWUP] add `--add-modules=jdk.incubator.vector` to maven compile args
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 1f9e09ce2148 [SPARK-48021][ML][BUILD][FOLLOWUP] add `--add-modules=jdk.incubator.vector` to maven compile args 1f9e09ce2148 is described below commit 1f9e09ce2148dfc5e0fd9f3e43e5ceef8133414b Author: panbingkun AuthorDate: Sun Apr 28 16:24:43 2024 +0800 [SPARK-48021][ML][BUILD][FOLLOWUP] add `--add-modules=jdk.incubator.vector` to maven compile args ### What changes were proposed in this pull request? The pr is following up https://github.com/apache/spark/pull/46246 The pr aims to add `--add-modules=jdk.incubator.vector` to maven `compile args`. ### Why are the changes needed? As commented by LuciferYang , we need to be consistent in `maven` compile. https://github.com/apache/spark/pull/46246#issuecomment-2081298219 https://github.com/apache/spark/assets/15246973/26163da2-f27d-4ec2-893f-d9282b68aec1;> ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46259 from panbingkun/SPARK-48021. Authored-by: panbingkun Signed-off-by: yangjie01 --- pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/pom.xml b/pom.xml index b916659fdbfa..efbf93856333 100644 --- a/pom.xml +++ b/pom.xml @@ -304,6 +304,7 @@ -XX:+IgnoreUnrecognizedVMOptions + --add-modules=jdk.incubator.vector --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47928][SQL][TEST] Speed up test "Add jar support Ivy URI in SQL"
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 61dc9d991373 [SPARK-47928][SQL][TEST] Speed up test "Add jar support Ivy URI in SQL" 61dc9d991373 is described below commit 61dc9d991373c01d449a8ed26d9bfd7eb93f9301 Author: Cheng Pan AuthorDate: Mon Apr 22 18:36:49 2024 +0800 [SPARK-47928][SQL][TEST] Speed up test "Add jar support Ivy URI in SQL" ### What changes were proposed in this pull request? `SQLQuerySuite`/"SPARK-33084: Add jar support Ivy URI in SQL" uses Hive deps to test `ADD JAR` which pulls tons of transitive deps, this PR replaces it with light jars but covers all semantics to speed up the UT. ### Why are the changes needed? Speed up the test, and reduce unnecessary relationships with Hive. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Run UT locally. Before ``` [info] - SPARK-33084: Add jar support Ivy URI in SQL (16 minutes, 55 seconds) ``` After ``` [info] - SPARK-33084: Add jar support Ivy URI in SQL (17 seconds, 783 milliseconds) ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46150 from pan3793/SPARK-47928. Authored-by: Cheng Pan Signed-off-by: yangjie01 --- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 25 +++--- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index f81369bbad36..78d4b91088a6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -3748,22 +3748,21 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark test("SPARK-33084: Add jar support Ivy URI in SQL") { val sc = spark.sparkContext -val hiveVersion = "2.3.9" // transitive=false, only download specified jar -sql(s"ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:$hiveVersion?transitive=false") -assert(sc.listJars() - .exists(_.contains(s"org.apache.hive.hcatalog_hive-hcatalog-core-$hiveVersion.jar"))) +sql(s"ADD JAR ivy://org.springframework:spring-core:6.1.6?transitive=false") + assert(sc.listJars().exists(_.contains("org.springframework_spring-core-6.1.6.jar"))) + assert(!sc.listJars().exists(_.contains("org.springframework_spring-jcl-6.1.6.jar"))) // default transitive=true, test download ivy URL jar return multiple jars -sql("ADD JAR ivy://org.scala-js:scalajs-test-interface_2.12:1.2.0") -assert(sc.listJars().exists(_.contains("scalajs-library_2.12"))) -assert(sc.listJars().exists(_.contains("scalajs-test-interface_2.12"))) - -sql(s"ADD JAR ivy://org.apache.hive:hive-contrib:$hiveVersion" + - "?exclude=org.pentaho:pentaho-aggdesigner-algorithm=true") - assert(sc.listJars().exists(_.contains(s"org.apache.hive_hive-contrib-$hiveVersion.jar"))) - assert(sc.listJars().exists(_.contains(s"org.apache.hive_hive-exec-$hiveVersion.jar"))) - assert(!sc.listJars().exists(_.contains("org.pentaho.pentaho_aggdesigner-algorithm"))) +sql("ADD JAR ivy://org.awaitility:awaitility:4.2.1") + assert(sc.listJars().exists(_.contains("org.awaitility_awaitility-4.2.1.jar"))) +assert(sc.listJars().exists(_.contains("org.hamcrest_hamcrest-2.1.jar"))) + +sql("ADD JAR ivy://org.junit.jupiter:junit-jupiter:5.10.2" + + "?exclude=org.junit.jupiter:junit-jupiter-engine=true") + assert(sc.listJars().exists(_.contains("org.junit.jupiter_junit-jupiter-api-5.10.2.jar"))) + assert(sc.listJars().exists(_.contains("org.junit.jupiter_junit-jupiter-params-5.10.2.jar"))) + assert(!sc.listJars().exists(_.contains("org.junit.jupiter_junit-jupiter-engine-5.10.2.jar"))) } test("SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (458f70bd5213 -> 2d0b56c3eac6)
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 458f70bd5213 [SPARK-47902][SQL] Making Compute Current Time* expressions foldable add 2d0b56c3eac6 [SPARK-47932][SQL][TESTS] Avoid using legacy commons-lang No new revisions were added by this update. Summary of changes: sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47901][BUILD] Upgrade common-text 1.12.0
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 06b12fc31709 [SPARK-47901][BUILD] Upgrade common-text 1.12.0 06b12fc31709 is described below commit 06b12fc317093d5a45c7c76e6617a9917f98b10d Author: yangjie01 AuthorDate: Fri Apr 19 13:21:37 2024 +0800 [SPARK-47901][BUILD] Upgrade common-text 1.12.0 ### What changes were proposed in this pull request? This pr aims to upgrade Apache common-text from 1.11.0 to 1.12.0 ### Why are the changes needed? The new version bring 2 bug fix: - [TEXT-232](https://issues.apache.org/jira/browse/TEXT-232): WordUtils.containsAllWords?() may throw PatternSyntaxException - [TEXT-175](https://issues.apache.org/jira/browse/TEXT-175): Fix regression for determining whitespace in WordUtils The full release notes as follows: - https://github.com/apache/commons-text/blob/rel/commons-text-1.12.0/RELEASE-NOTES.txt ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #46127 from LuciferYang/commons-text-1.12.0. Authored-by: yangjie01 Signed-off-by: yangjie01 --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 45a4d499e513..770a7522e9f7 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -48,7 +48,7 @@ commons-lang/2.6//commons-lang-2.6.jar commons-lang3/3.14.0//commons-lang3-3.14.0.jar commons-math3/3.6.1//commons-math3-3.6.1.jar commons-pool/1.5.4//commons-pool-1.5.4.jar -commons-text/1.11.0//commons-text-1.11.0.jar +commons-text/1.12.0//commons-text-1.12.0.jar compress-lzf/1.1.2//compress-lzf-1.1.2.jar curator-client/5.6.0//curator-client-5.6.0.jar curator-framework/5.6.0//curator-framework-5.6.0.jar diff --git a/pom.xml b/pom.xml index 682365d9704a..74a2a61d6e09 100644 --- a/pom.xml +++ b/pom.xml @@ -606,7 +606,7 @@ org.apache.commons commons-text -1.11.0 +1.12.0 commons-lang - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47850][SQL] Support `spark.sql.hive.convertInsertingUnpartitionedTable`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 52a2e63dd714 [SPARK-47850][SQL] Support `spark.sql.hive.convertInsertingUnpartitionedTable` 52a2e63dd714 is described below commit 52a2e63dd7147e2701c9c26667fe5bd9fdc3f14c Author: Cheng Pan AuthorDate: Thu Apr 18 15:05:15 2024 +0800 [SPARK-47850][SQL] Support `spark.sql.hive.convertInsertingUnpartitionedTable` ### What changes were proposed in this pull request? This PR introduced a new configuration `spark.sql.hive.convertInsertingUnpartitionedTable` alongside the existing `spark.sql.hive.convertInsertingPartitionedTable` to allow fine grain switching from Hive Serde to Data Source on inserting Parquet/ORC Hive tables. ### Why are the changes needed? In the practice of hybrid workload (Hive tables may be read/written by Hive, Spark, Impala, etc.), we usually use DataSource for reading Parquet/ORC tables but Hive Serde for writing, the current configuration combination allows us to achieve that except for unpartitioned tables. ### Does this PR introduce _any_ user-facing change? No. The new added configuration `spark.sql.hive.convertInsertingUnpartitionedTable` default value is `true`, which keeps the existing behavior. ### How was this patch tested? New UT is added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46052 from pan3793/SPARK-47850. Authored-by: Cheng Pan Signed-off-by: yangjie01 --- .../plans/logical/basicLogicalOperators.scala | 1 + .../apache/spark/sql/execution/command/views.scala | 1 + .../org/apache/spark/sql/hive/HiveStrategies.scala | 10 -- .../org/apache/spark/sql/hive/HiveUtils.scala | 10 ++ .../execution/CreateHiveTableAsSelectCommand.scala | 5 ++- .../sql/hive/execution/InsertIntoHiveTable.scala | 7 .../spark/sql/hive/orc/HiveOrcQuerySuite.scala | 37 ++ 7 files changed, 67 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 1c8f7a97dd7f..7c36e3bc79af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -783,6 +783,7 @@ object View { "spark.sql.hive.convertMetastoreParquet", "spark.sql.hive.convertMetastoreOrc", "spark.sql.hive.convertInsertingPartitionedTable", +"spark.sql.hive.convertInsertingUnpartitionedTable", "spark.sql.hive.convertMetastoreCtas" ).contains(key) || key.startsWith("spark.sql.catalog.") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index d71d0d43683c..cb5e7e7f42d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -360,6 +360,7 @@ object ViewHelper extends SQLConfHelper with Logging { "spark.sql.hive.convertMetastoreParquet", "spark.sql.hive.convertMetastoreOrc", "spark.sql.hive.convertInsertingPartitionedTable", +"spark.sql.hive.convertInsertingUnpartitionedTable", "spark.sql.hive.convertMetastoreCtas", SQLConf.ADDITIONAL_REMOTE_REPOSITORIES.key) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 5972a9df78ec..e74cc088a1f6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.execution.command.{CreateTableCommand, DDLUtils, Ins import org.apache.spark.sql.execution.datasources.{CreateTable, DataSourceStrategy, HadoopFsRelation, InsertIntoHadoopFsRelationCommand, LogicalRelation} import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.hive.execution.HiveScriptTransformationExec +import org.apache.spark.sql.hive.execution.InsertIntoHiveTable.BY_CTAS import org.apache.spark.sql.internal.HiveSerDe @@ -194,6 +195,8 @@ object HiveAnalysis extends Rule[LogicalPlan] { * - When writing to non-partitioned Hive-serde Parquet/Orc tables * - When writing to partitioned Hive-serde Parquet/Orc tables when
(spark) branch master updated: [SPARK-47770][INFRA] Fix `GenerateMIMAIgnore.isPackagePrivateModule` to return `false` instead of failing
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 08c49637795f [SPARK-47770][INFRA] Fix `GenerateMIMAIgnore.isPackagePrivateModule` to return `false` instead of failing 08c49637795f is described below commit 08c49637795fd56ef550a509648f0890ff22a948 Author: Dongjoon Hyun AuthorDate: Tue Apr 9 11:14:49 2024 +0800 [SPARK-47770][INFRA] Fix `GenerateMIMAIgnore.isPackagePrivateModule` to return `false` instead of failing ### What changes were proposed in this pull request? This PR aims to fix `GenerateMIMAIgnore.isPackagePrivateModule` to work correctly. For example, `Metadata` is a case class inside package private `DefaultParamsReader` class. Currently, MIMA fails at this class analysis. https://github.com/apache/spark/blob/f8e652e88320528a70e605a6a3cf986725e153a5/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala#L474-L485 The root cause is `isPackagePrivateModule` fails due to `scala.ScalaReflectionException`. We can simply make `isPackagePrivateModule` return `false` instead of failing. ``` Error instrumenting class:org.apache.spark.ml.util.DefaultParamsReader$Metadata Exception in thread "main" scala.ScalaReflectionException: type Serializable is not a class at scala.reflect.api.Symbols$SymbolApi.asClass(Symbols.scala:284) at scala.reflect.api.Symbols$SymbolApi.asClass$(Symbols.scala:284) at scala.reflect.internal.Symbols$SymbolContextApiImpl.asClass(Symbols.scala:99) at scala.reflect.runtime.JavaMirrors$JavaMirror.classToScala1(JavaMirrors.scala:1085) at scala.reflect.runtime.JavaMirrors$JavaMirror.$anonfun$classToScala$1(JavaMirrors.scala:1040) at scala.reflect.runtime.JavaMirrors$JavaMirror.$anonfun$toScala$1(JavaMirrors.scala:150) at scala.reflect.runtime.TwoWayCaches$TwoWayCache.toScala(TwoWayCaches.scala:50) at scala.reflect.runtime.JavaMirrors$JavaMirror.toScala(JavaMirrors.scala:148) at scala.reflect.runtime.JavaMirrors$JavaMirror.classToScala(JavaMirrors.scala:1040) at scala.reflect.runtime.JavaMirrors$JavaMirror.typeToScala(JavaMirrors.scala:1148) at scala.reflect.runtime.JavaMirrors$JavaMirror$FromJavaClassCompleter.$anonfun$completeRest$2(JavaMirrors.scala:816) at scala.reflect.runtime.JavaMirrors$JavaMirror$FromJavaClassCompleter.$anonfun$completeRest$1(JavaMirrors.scala:816) at scala.reflect.runtime.JavaMirrors$JavaMirror$FromJavaClassCompleter.completeRest(JavaMirrors.scala:810) at scala.reflect.runtime.JavaMirrors$JavaMirror$FromJavaClassCompleter.complete(JavaMirrors.scala:806) at scala.reflect.internal.Symbols$Symbol.completeInfo(Symbols.scala:1575) at scala.reflect.internal.Symbols$Symbol.info(Symbols.scala:1538) at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anon$13.scala$reflect$runtime$SynchronizedSymbols$SynchronizedSymbol$$super$info(SynchronizedSymbols.scala:221) at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol.info(SynchronizedSymbols.scala:158) at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol.info$(SynchronizedSymbols.scala:158) at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anon$13.info(SynchronizedSymbols.scala:221) at scala.reflect.internal.Symbols$Symbol.initialize(Symbols.scala:1733) at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol.privateWithin(SynchronizedSymbols.scala:109) at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol.privateWithin$(SynchronizedSymbols.scala:107) at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anon$13.privateWithin(SynchronizedSymbols.scala:221) at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anon$13.privateWithin(SynchronizedSymbols.scala:221) at org.apache.spark.tools.GenerateMIMAIgnore$.isPackagePrivateModule(GenerateMIMAIgnore.scala:48) at org.apache.spark.tools.GenerateMIMAIgnore$.$anonfun$privateWithin$1(GenerateMIMAIgnore.scala:67) at scala.collection.immutable.List.foreach(List.scala:334) at org.apache.spark.tools.GenerateMIMAIgnore$.privateWithin(GenerateMIMAIgnore.scala:61) at org.apache.spark.tools.GenerateMIMAIgnore$.main(GenerateMIMAIgnore.scala:125) at org.apache.spark.tools.GenerateMIMAIgnore.main(GenerateMIMAIgnore.scala) ``` ### Why are the changes needed? **BEFORE** ``` $ dev/mima | grep org.apache.spark.ml.util.DefaultParamsReader Using SPARK_LOCAL_IP=localhost Using SPARK_LOCAL_IP
(spark) branch master updated (a598f654066d -> 03f4e45cd7e9)
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from a598f654066d [SPARK-47664][PYTHON][CONNECT][TESTS][FOLLOW-UP] Add more tests add 03f4e45cd7e9 [SPARK-47685][SQL] Restore the support for `Stream` type in `Dataset#groupBy` No new revisions were added by this update. Summary of changes: .../scala/org/apache/spark/sql/RelationalGroupedDataset.scala | 4 +++- .../test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala | 8 +++- 2 files changed, 10 insertions(+), 2 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch branch-3.5 updated: [SPARK-45593][BUILD][3.5] Correct relocation connect guava dependency
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 2da520e88266 [SPARK-45593][BUILD][3.5] Correct relocation connect guava dependency 2da520e88266 is described below commit 2da520e88266530b2283ef3c9ac90bdc806b7556 Author: yikaifei AuthorDate: Mon Apr 1 15:35:23 2024 +0800 [SPARK-45593][BUILD][3.5] Correct relocation connect guava dependency ### What changes were proposed in this pull request? This PR amins to correct relocation connect guava dependency and remove duplicate connect-common from SBT build jars. This PR cherry-pick from https://github.com/apache/spark/pull/43436 and https://github.com/apache/spark/pull/44801 as a backport to 3.5 branch. ### Why are the changes needed? Bugfix ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Follow the steps described at https://github.com/apache/spark/pull/43195#issue-1921234067 to test manually. In addition, will continue to observe the GA situation in recent days. ### Was this patch authored or co-authored using generative AI tooling? No Closes #45775 from Yikf/branch-3.5. Authored-by: yikaifei Signed-off-by: yangjie01 --- assembly/pom.xml | 6 ++ connector/connect/client/jvm/pom.xml | 22 ++ connector/connect/common/pom.xml | 33 + connector/connect/server/pom.xml | 1 + project/SparkBuild.scala | 6 +- 5 files changed, 51 insertions(+), 17 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index d1ef9b24afda..21330058f77d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -159,6 +159,12 @@ org.apache.spark spark-connect_${scala.binary.version} ${project.version} + + + org.apache.spark + spark-connect-common_${scala.binary.version} + + org.apache.spark diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index 53ff0b0147e0..6febc5ee6bd6 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -51,9 +51,14 @@ ${project.version} + + com.google.protobuf + protobuf-java + compile + com.google.guava guava @@ -61,8 +66,9 @@ compile - com.google.protobuf - protobuf-java + com.google.guava + failureaccess + ${guava.failureaccess.version} compile @@ -108,6 +114,7 @@ true + com.google.guava:* com.google.android:* com.google.api.grpc:* com.google.code.findbugs:* @@ -127,6 +134,13 @@ + + com.google.common + ${spark.shade.packageName}.connect.guava + +com.google.common.** + + io.grpc ${spark.shade.packageName}.io.grpc @@ -138,7 +152,7 @@ com.google ${spark.shade.packageName}.com.google - + com.google.common.** diff --git a/connector/connect/common/pom.xml b/connector/connect/common/pom.xml index 7ce0aa6615d3..3c07b63c50a5 100644 --- a/connector/connect/common/pom.xml +++ b/connector/connect/common/pom.xml @@ -47,18 +47,6 @@ com.google.protobuf protobuf-java - -com.google.guava -guava -${connect.guava.version} -compile - - -com.google.guava -failureaccess -${guava.failureaccess.version} -compile - io.grpc grpc-netty @@ -152,6 +140,27 @@ + +org.apache.maven.plugins +maven-shade-plugin + +false + + +org.spark-project.spark:unused + org.apache.tomcat:annotations-api + + + + + +package + +shade + + + + diff --git a/connector/connect/server/pom.xml b/connector/connect
(spark) branch master updated: [SPARK-47629][INFRA] Add `common/variant` and `connector/kinesis-asl` to maven daily test module list
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new a8b247e9a50a [SPARK-47629][INFRA] Add `common/variant` and `connector/kinesis-asl` to maven daily test module list a8b247e9a50a is described below commit a8b247e9a50ae0450360e76bc69b2c6cdf5ea6f8 Author: yangjie01 AuthorDate: Fri Mar 29 13:26:40 2024 +0800 [SPARK-47629][INFRA] Add `common/variant` and `connector/kinesis-asl` to maven daily test module list ### What changes were proposed in this pull request? This pr add `common/variant` and `connector/kinesis-asl` to maven daily test module list. ### Why are the changes needed? Synchronize the modules to be tested in Maven daily test ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Monitor GA after merge ### Was this patch authored or co-authored using generative AI tooling? No Closes #45754 from LuciferYang/SPARK-47629. Authored-by: yangjie01 Signed-off-by: yangjie01 --- .github/workflows/maven_test.yml | 15 --- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/.github/workflows/maven_test.yml b/.github/workflows/maven_test.yml index 34fa9a8b7768..b01f08a23e47 100644 --- a/.github/workflows/maven_test.yml +++ b/.github/workflows/maven_test.yml @@ -62,7 +62,7 @@ jobs: - hive2.3 modules: - >- - core,launcher,common#unsafe,common#kvstore,common#network-common,common#network-shuffle,common#sketch,common#utils + core,launcher,common#unsafe,common#kvstore,common#network-common,common#network-shuffle,common#sketch,common#utils,common#variant - >- graphx,streaming,hadoop-cloud - >- @@ -70,7 +70,7 @@ jobs: - >- repl,sql#hive-thriftserver - >- - connector#kafka-0-10,connector#kafka-0-10-sql,connector#kafka-0-10-token-provider,connector#spark-ganglia-lgpl,connector#protobuf,connector#avro + connector#kafka-0-10,connector#kafka-0-10-sql,connector#kafka-0-10-token-provider,connector#spark-ganglia-lgpl,connector#protobuf,connector#avro,connector#kinesis-asl - >- sql#api,sql#catalyst,resource-managers#yarn,resource-managers#kubernetes#core # Here, we split Hive and SQL tests into some of slow ones and the rest of them. @@ -188,20 +188,21 @@ jobs: export MAVEN_OPTS="-Xss64m -Xmx4g -Xms4g -XX:ReservedCodeCacheSize=128m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" export MAVEN_CLI_OPTS="--no-transfer-progress" export JAVA_VERSION=${{ matrix.java }} + export ENABLE_KINESIS_TESTS=0 # Replace with the real module name, for example, connector#kafka-0-10 -> connector/kafka-0-10 export TEST_MODULES=`echo "$MODULES_TO_TEST" | sed -e "s%#%/%g"` - ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pspark-ganglia-lgpl -Djava.version=${JAVA_VERSION/-ea} clean install + ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pspark-ganglia-lgpl -Pkinesis-asl -Djava.version=${JAVA_VERSION/-ea} clean install if [[ "$INCLUDED_TAGS" != "" ]]; then -./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pspark-ganglia-lgpl -Djava.version=${JAVA_VERSION/-ea} -Dtest.include.tags="$INCLUDED_TAGS" test -fae +./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pspark-ganglia-lgpl -Pkinesis-asl -Djava.version=${JAVA_VERSION/-ea} -Dtest.include.tags="$INCLUDED_TAGS" test -fae elif [[ "$MODULES_TO_TEST" == "connect" ]]; then ./build/mvn $MAVEN_CLI_OPTS -Dtest.exclude.tags="$EXCLUDED_TAGS" -Djava.version=${JAVA_VERSION/-ea} -pl connector/connect/client/jvm,connector/connect/common,connector/connect/server test -fae elif [[ "$EXCLUDED_TAGS" != "" ]]; then -./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pspark-ganglia-lgpl -Djava.version=${JAVA_VERSION/-ea} -Dtest.exclude.tags="$EXCLUDED_TAGS" test -fae +./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pspark-ganglia-lgpl -Pkinesis-asl -Djava.version=${JAVA_VERSION/-ea} -Dtes
(spark) branch master updated: [SPARK-47610][CORE] Always set `io.netty.tryReflectionSetAccessible=true`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 5f392a219de2 [SPARK-47610][CORE] Always set `io.netty.tryReflectionSetAccessible=true` 5f392a219de2 is described below commit 5f392a219de29b0856884fb95ff3e313f1047013 Author: Cheng Pan AuthorDate: Wed Mar 27 13:16:13 2024 +0800 [SPARK-47610][CORE] Always set `io.netty.tryReflectionSetAccessible=true` ### What changes were proposed in this pull request? Always set `io.netty.tryReflectionSetAccessible=true` ### Why are the changes needed? Arrow requires `-Dio.netty.tryReflectionSetAccessible=true` for JDK9+, see details in ARROW-7223. SPARK-29924 (fixed in 3.0.0) added document to guide users to add such JavaOpts manually, as Arrow is a Spark built-in component, and later we added such Java options to the building system(Maven, SBT, and PySpark test suite) manually. Now Spark requires JDK 17+, I think we can add such Java options by default to reduce disturbing users. ### Does this PR introduce _any_ user-facing change? Yes, no impacts for those users who manually added `io.netty.tryReflectionSetAccessible=true`, but makes life easier for new Spark users. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45733 from pan3793/SPARK-47610. Authored-by: Cheng Pan Signed-off-by: yangjie01 --- docs/index.md| 2 -- .../src/main/java/org/apache/spark/launcher/JavaModuleOptions.java | 3 ++- pom.xml | 5 +++-- project/SparkBuild.scala | 4 ++-- python/docs/source/getting_started/install.rst | 5 + python/run-tests.py | 2 +- sql/hive/pom.xml | 2 +- 7 files changed, 10 insertions(+), 13 deletions(-) diff --git a/docs/index.md b/docs/index.md index 12c53c40c8f7..57f701316bd0 100644 --- a/docs/index.md +++ b/docs/index.md @@ -38,8 +38,6 @@ Spark runs on Java 17/21, Scala 2.13, Python 3.8+, and R 3.5+. When using the Scala API, it is necessary for applications to use the same version of Scala that Spark was compiled for. For example, when using Scala 2.13, use Spark compiled for 2.13, and compile code/applications for Scala 2.13 as well. -Setting `-Dio.netty.tryReflectionSetAccessible=true` is required for the Apache Arrow library. This prevents the `java.lang.UnsupportedOperationException: sun.misc.Unsafe or java.nio.DirectByteBuffer.(long, int) not available` error when Apache Arrow uses Netty internally. - # Running the Examples and Shell Spark comes with several sample programs. Python, Scala, Java, and R examples are in the diff --git a/launcher/src/main/java/org/apache/spark/launcher/JavaModuleOptions.java b/launcher/src/main/java/org/apache/spark/launcher/JavaModuleOptions.java index 8893f4bcb85a..3a8fa6c42d47 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/JavaModuleOptions.java +++ b/launcher/src/main/java/org/apache/spark/launcher/JavaModuleOptions.java @@ -42,7 +42,8 @@ public class JavaModuleOptions { "--add-opens=java.base/sun.security.action=ALL-UNNAMED", "--add-opens=java.base/sun.util.calendar=ALL-UNNAMED", "--add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED", - "-Djdk.reflect.useDirectMethodHandle=false"}; + "-Djdk.reflect.useDirectMethodHandle=false", + "-Dio.netty.tryReflectionSetAccessible=true"}; /** * Returns the default Java options related to `--add-opens' and diff --git a/pom.xml b/pom.xml index 79f8745e01f8..ffa4b5df36cb 100644 --- a/pom.xml +++ b/pom.xml @@ -316,6 +316,7 @@ --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false + -Dio.netty.tryReflectionSetAccessible=true @@ -3109,7 +3110,7 @@ **/*Suite.java ${project.build.directory}/surefire-reports --ea -Xmx4g -Xss4m -XX:MaxMetaspaceSize=2g -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} -Dio.netty.tryReflectionSetAccessible=true +-ea -Xmx4g -Xss4m -XX:MaxMetaspaceSize=2g -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} - -da -Xmx4g -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} -Dio.netty.tryReflectionSetAccessible=true + -da -Xmx4g -XX
(spark) branch master updated: [SPARK-47474][CORE] Revert SPARK-47461 and add some comments
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new bee1bceb [SPARK-47474][CORE] Revert SPARK-47461 and add some comments bee1bceb is described below commit bee1bcebdad218a4151ad192d4893ff0fed9 Author: yangjie01 AuthorDate: Thu Mar 21 13:58:39 2024 +0800 [SPARK-47474][CORE] Revert SPARK-47461 and add some comments ### What changes were proposed in this pull request? This pr revert the change of SPARK-47461 and add some comments to `ExecutorAllocationManager#totalRunningTasksPerResourceProfile` to clarify that the tests in `ExecutorAllocationManagerSuite` need to call `listener.totalRunningTasksPerResourceProfile` with `synchronized`. ### Why are the changes needed? `ExecutorAllocationManagerSuite` need to call `listener.totalRunningTasksPerResourceProfile` with `synchronized`. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #45602 from LuciferYang/SPARK-47474. Authored-by: yangjie01 Signed-off-by: yangjie01 --- .../src/main/scala/org/apache/spark/ExecutorAllocationManager.scala | 6 ++ .../scala/org/apache/spark/ExecutorAllocationManagerSuite.scala | 4 +++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index cdd1aecf4a22..94927caff1d7 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -320,6 +320,12 @@ private[spark] class ExecutorAllocationManager( } } + // Please do not delete this function, the tests in `ExecutorAllocationManagerSuite` + // need to access `listener.totalRunningTasksPerResourceProfile` with `synchronized`. + private def totalRunningTasksPerResourceProfile(id: Int): Int = synchronized { +listener.totalRunningTasksPerResourceProfile(id) + } + /** * This is called at a fixed interval to regulate the number of pending executor requests * and number of executors running. diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index aeb3cf53ff1a..e1da2b6dd9d6 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -1934,6 +1934,8 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { PrivateMethod[Map[Int, Map[String, Int]]](Symbol("rpIdToHostToLocalTaskCount")) private val _onSpeculativeTaskSubmitted = PrivateMethod[Unit](Symbol("onSpeculativeTaskSubmitted")) + private val _totalRunningTasksPerResourceProfile = +PrivateMethod[Int](Symbol("totalRunningTasksPerResourceProfile")) private val defaultProfile = ResourceProfile.getOrCreateDefaultProfile(new SparkConf) @@ -2041,7 +2043,7 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { } private def totalRunningTasksPerResourceProfile(manager: ExecutorAllocationManager): Int = { -manager.listener.totalRunningTasksPerResourceProfile(defaultProfile.id) +manager invokePrivate _totalRunningTasksPerResourceProfile(defaultProfile.id) } private def hostToLocalTaskCount( - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (bb0867f54d43 -> 5d3845f2942a)
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from bb0867f54d43 [MINOR][CORE] Fix a comment typo `slf4j-to-jul` to `jul-to-slf4j` add 5d3845f2942a [SPARK-46920][YARN] Improve executor exit error message on YARN No new revisions were added by this update. Summary of changes: .../apache/spark/deploy/yarn/YarnAllocator.scala | 28 -- 1 file changed, 16 insertions(+), 12 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch branch-3.4 updated: [SPARK-47455][BUILD] Fix resource leak during the initialization of `scalaStyleOnCompileConfig` in `SparkBuild.scala`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch branch-3.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.4 by this push: new 142677bcd203 [SPARK-47455][BUILD] Fix resource leak during the initialization of `scalaStyleOnCompileConfig` in `SparkBuild.scala` 142677bcd203 is described below commit 142677bcd203caf2b6d07bf41d654e123d910ee8 Author: yangjie01 AuthorDate: Wed Mar 20 15:19:33 2024 +0800 [SPARK-47455][BUILD] Fix resource leak during the initialization of `scalaStyleOnCompileConfig` in `SparkBuild.scala` ### What changes were proposed in this pull request? https://github.com/apache/spark/blob/e01ed0da22f24204fe23143032ff39be7f4b56af/project/SparkBuild.scala#L157-L173 `Source.fromFile(in)` opens a `BufferedSource` resource handle, but it does not close it, this pr fix this issue. ### Why are the changes needed? Close resource after used. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #45582 from LuciferYang/SPARK-47455. Authored-by: yangjie01 Signed-off-by: yangjie01 (cherry picked from commit 85bf7615f85eea3e9192a7684ef711cf44042e05) Signed-off-by: yangjie01 --- project/SparkBuild.scala | 23 ++- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 31263eaa4c8d..31516c8c6ffe 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -159,16 +159,21 @@ object SparkBuild extends PomBuild { val replacements = Map( """customId="println" level="error"""" -> """customId="println" level="warn"""" ) -var contents = Source.fromFile(in).getLines.mkString("\n") -for ((k, v) <- replacements) { - require(contents.contains(k), s"Could not rewrite '$k' in original scalastyle config.") - contents = contents.replace(k, v) -} -new PrintWriter(out) { - write(contents) - close() +val source = Source.fromFile(in) +try { + var contents = source.getLines.mkString("\n") + for ((k, v) <- replacements) { +require(contents.contains(k), s"Could not rewrite '$k' in original scalastyle config.") +contents = contents.replace(k, v) + } + new PrintWriter(out) { +write(contents) +close() + } + out +} finally { + source.close() } -out } // Return a cached scalastyle task for a given configuration (usually Compile or Test) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch branch-3.5 updated: [SPARK-47455][BUILD] Fix resource leak during the initialization of `scalaStyleOnCompileConfig` in `SparkBuild.scala`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 8fcd9a1b0024 [SPARK-47455][BUILD] Fix resource leak during the initialization of `scalaStyleOnCompileConfig` in `SparkBuild.scala` 8fcd9a1b0024 is described below commit 8fcd9a1b0024d24e3622b1948123e7f239a734a5 Author: yangjie01 AuthorDate: Wed Mar 20 15:19:33 2024 +0800 [SPARK-47455][BUILD] Fix resource leak during the initialization of `scalaStyleOnCompileConfig` in `SparkBuild.scala` ### What changes were proposed in this pull request? https://github.com/apache/spark/blob/e01ed0da22f24204fe23143032ff39be7f4b56af/project/SparkBuild.scala#L157-L173 `Source.fromFile(in)` opens a `BufferedSource` resource handle, but it does not close it, this pr fix this issue. ### Why are the changes needed? Close resource after used. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #45582 from LuciferYang/SPARK-47455. Authored-by: yangjie01 Signed-off-by: yangjie01 (cherry picked from commit 85bf7615f85eea3e9192a7684ef711cf44042e05) Signed-off-by: yangjie01 --- project/SparkBuild.scala | 23 ++- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 79b58deafde5..dfadfea172d8 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -160,16 +160,21 @@ object SparkBuild extends PomBuild { val replacements = Map( """customId="println" level="error"""" -> """customId="println" level="warn"""" ) -var contents = Source.fromFile(in).getLines.mkString("\n") -for ((k, v) <- replacements) { - require(contents.contains(k), s"Could not rewrite '$k' in original scalastyle config.") - contents = contents.replace(k, v) -} -new PrintWriter(out) { - write(contents) - close() +val source = Source.fromFile(in) +try { + var contents = source.getLines.mkString("\n") + for ((k, v) <- replacements) { +require(contents.contains(k), s"Could not rewrite '$k' in original scalastyle config.") +contents = contents.replace(k, v) + } + new PrintWriter(out) { +write(contents) +close() + } + out +} finally { + source.close() } -out } // Return a cached scalastyle task for a given configuration (usually Compile or Test) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (c3a04fa59ce1 -> 85bf7615f85e)
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from c3a04fa59ce1 [SPARK-47447][SQL] Allow reading Parquet TimestampLTZ as TimestampNTZ add 85bf7615f85e [SPARK-47455][BUILD] Fix resource leak during the initialization of `scalaStyleOnCompileConfig` in `SparkBuild.scala` No new revisions were added by this update. Summary of changes: project/SparkBuild.scala | 23 ++- 1 file changed, 14 insertions(+), 9 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch branch-3.4 updated: [SPARK-47305][SQL][TESTS][FOLLOWUP][3.4] Fix the compilation error related to `PropagateEmptyRelationSuite`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch branch-3.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.4 by this push: new 8b4316461e2b [SPARK-47305][SQL][TESTS][FOLLOWUP][3.4] Fix the compilation error related to `PropagateEmptyRelationSuite` 8b4316461e2b is described below commit 8b4316461e2bc3ca3b72170648ca6b6e36537a65 Author: yangjie01 AuthorDate: Fri Mar 8 16:44:54 2024 +0800 [SPARK-47305][SQL][TESTS][FOLLOWUP][3.4] Fix the compilation error related to `PropagateEmptyRelationSuite` ### What changes were proposed in this pull request? https://github.com/apache/spark/pull/45406 has been backported to branch-3.4, where the newly added test case in `PropagateEmptyRelationSuite` uses `DataTypeUtils`, but `DataTypeUtils` is a utility class added in Apache Spark 3.5(SPARK-44475), so this triggered a compilation failure in branch-3.4: - https://github.com/apache/spark/actions/runs/8183755511/job/22377119069 ``` [error] /home/runner/work/spark/spark/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala:229:27: not found: value DataTypeUtils [error] val schemaForStream = DataTypeUtils.fromAttributes(outputForStream) [error] ^ [error] /home/runner/work/spark/spark/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala:233:26: not found: value DataTypeUtils [error] val schemaForBatch = DataTypeUtils.fromAttributes(outputForBatch) [error] ^ [info] done compiling [info] compiling 1 Scala source to /home/runner/work/spark/spark/connector/connect/common/target/scala-2.12/test-classes ... [info] compiling 25 Scala sources and 1 Java source to /home/runner/work/spark/spark/connector/connect/client/jvm/target/scala-2.12/classes ... [info] done compiling [error] two errors found ``` Therefore, this PR changes to use the `StructType.fromAttributes` function to fix the compilation failure." ### Why are the changes needed? Fix the compilation failure in branch-3.4 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #45428 from LuciferYang/SPARK-47305-FOLLOWUP-34. Authored-by: yangjie01 Signed-off-by: yangjie01 --- .../spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala| 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala index a1132eadcc6f..91b62ae953f0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala @@ -226,11 +226,11 @@ class PropagateEmptyRelationSuite extends PlanTest { val data = Seq(Row(1)) val outputForStream = Seq($"a".int) -val schemaForStream = DataTypeUtils.fromAttributes(outputForStream) +val schemaForStream = StructType.fromAttributes(outputForStream) val converterForStream = CatalystTypeConverters.createToCatalystConverter(schemaForStream) val outputForBatch = Seq($"b".int) -val schemaForBatch = DataTypeUtils.fromAttributes(outputForBatch) +val schemaForBatch = StructType.fromAttributes(outputForBatch) val converterForBatch = CatalystTypeConverters.createToCatalystConverter(schemaForBatch) val streamingRelation = LocalRelation( - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (3aa16e8193cf -> 433c9b064a3f)
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 3aa16e8193cf [MINOR] Update outdated comments for class `o.a.s.s.functions` add 433c9b064a3f [SPARK-47246][SQL] Replace `InternalRow.fromSeq` with `new GenericInternalRow` to save a collection conversion No new revisions were added by this update. Summary of changes: .../org/apache/spark/sql/catalyst/expressions/literals.scala | 5 ++--- .../spark/sql/catalyst/expressions/objects/objects.scala | 2 +- .../org/apache/spark/sql/catalyst/xml/StaxXmlParser.scala| 12 ++-- .../spark/sql/execution/columnar/InMemoryRelation.scala | 4 ++-- .../datasources/v2/state/metadata/StateMetadataSource.scala | 5 +++-- .../scala/org/apache/spark/sql/hive/HiveInspectors.scala | 2 +- .../apache/spark/sql/hive/execution/HiveTableScanExec.scala | 2 +- 7 files changed, 16 insertions(+), 16 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated (c8d293dff595 -> 7e7ba4eaf071)
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from c8d293dff595 [SPARK-47147][PYTHON][SQL] Fix PySpark collated string conversion error add 7e7ba4eaf071 [MINOR][SQL] Remove out-of-dated comment in `CollectLimitExec` No new revisions were added by this update. Summary of changes: sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala | 5 - 1 file changed, 5 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47100][BUILD] Upgrade `netty` to 4.1.107.Final and `netty-tcnative` to 2.0.62.Final
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new fb1e7872a3e6 [SPARK-47100][BUILD] Upgrade `netty` to 4.1.107.Final and `netty-tcnative` to 2.0.62.Final fb1e7872a3e6 is described below commit fb1e7872a3e64eab6127f9c2b3ffa42b63162f6c Author: Dongjoon Hyun AuthorDate: Tue Feb 20 17:04:41 2024 +0800 [SPARK-47100][BUILD] Upgrade `netty` to 4.1.107.Final and `netty-tcnative` to 2.0.62.Final ### What changes were proposed in this pull request? This PR aims to upgrade `netty` to 4.1.107.Final and `netty-tcnative` to 2.0.62.Final. ### Why are the changes needed? To bring the latest bug fixes. - https://netty.io/news/2024/02/13/4-1-107-Final.html ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45178 from dongjoon-hyun/SPARK-47100. Authored-by: Dongjoon Hyun Signed-off-by: yangjie01 --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 50 +-- pom.xml | 4 +-- 2 files changed, 27 insertions(+), 27 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index dbbddbc54c11..cc0145e004a0 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -192,32 +192,32 @@ metrics-jmx/4.2.25//metrics-jmx-4.2.25.jar metrics-json/4.2.25//metrics-json-4.2.25.jar metrics-jvm/4.2.25//metrics-jvm-4.2.25.jar minlog/1.3.0//minlog-1.3.0.jar -netty-all/4.1.106.Final//netty-all-4.1.106.Final.jar -netty-buffer/4.1.106.Final//netty-buffer-4.1.106.Final.jar -netty-codec-http/4.1.106.Final//netty-codec-http-4.1.106.Final.jar -netty-codec-http2/4.1.106.Final//netty-codec-http2-4.1.106.Final.jar -netty-codec-socks/4.1.106.Final//netty-codec-socks-4.1.106.Final.jar -netty-codec/4.1.106.Final//netty-codec-4.1.106.Final.jar -netty-common/4.1.106.Final//netty-common-4.1.106.Final.jar -netty-handler-proxy/4.1.106.Final//netty-handler-proxy-4.1.106.Final.jar -netty-handler/4.1.106.Final//netty-handler-4.1.106.Final.jar -netty-resolver/4.1.106.Final//netty-resolver-4.1.106.Final.jar +netty-all/4.1.107.Final//netty-all-4.1.107.Final.jar +netty-buffer/4.1.107.Final//netty-buffer-4.1.107.Final.jar +netty-codec-http/4.1.107.Final//netty-codec-http-4.1.107.Final.jar +netty-codec-http2/4.1.107.Final//netty-codec-http2-4.1.107.Final.jar +netty-codec-socks/4.1.107.Final//netty-codec-socks-4.1.107.Final.jar +netty-codec/4.1.107.Final//netty-codec-4.1.107.Final.jar +netty-common/4.1.107.Final//netty-common-4.1.107.Final.jar +netty-handler-proxy/4.1.107.Final//netty-handler-proxy-4.1.107.Final.jar +netty-handler/4.1.107.Final//netty-handler-4.1.107.Final.jar +netty-resolver/4.1.107.Final//netty-resolver-4.1.107.Final.jar netty-tcnative-boringssl-static/2.0.61.Final//netty-tcnative-boringssl-static-2.0.61.Final.jar -netty-tcnative-boringssl-static/2.0.61.Final/linux-aarch_64/netty-tcnative-boringssl-static-2.0.61.Final-linux-aarch_64.jar -netty-tcnative-boringssl-static/2.0.61.Final/linux-x86_64/netty-tcnative-boringssl-static-2.0.61.Final-linux-x86_64.jar -netty-tcnative-boringssl-static/2.0.61.Final/osx-aarch_64/netty-tcnative-boringssl-static-2.0.61.Final-osx-aarch_64.jar -netty-tcnative-boringssl-static/2.0.61.Final/osx-x86_64/netty-tcnative-boringssl-static-2.0.61.Final-osx-x86_64.jar -netty-tcnative-boringssl-static/2.0.61.Final/windows-x86_64/netty-tcnative-boringssl-static-2.0.61.Final-windows-x86_64.jar -netty-tcnative-classes/2.0.61.Final//netty-tcnative-classes-2.0.61.Final.jar -netty-transport-classes-epoll/4.1.106.Final//netty-transport-classes-epoll-4.1.106.Final.jar -netty-transport-classes-kqueue/4.1.106.Final//netty-transport-classes-kqueue-4.1.106.Final.jar -netty-transport-native-epoll/4.1.106.Final/linux-aarch_64/netty-transport-native-epoll-4.1.106.Final-linux-aarch_64.jar -netty-transport-native-epoll/4.1.106.Final/linux-riscv64/netty-transport-native-epoll-4.1.106.Final-linux-riscv64.jar -netty-transport-native-epoll/4.1.106.Final/linux-x86_64/netty-transport-native-epoll-4.1.106.Final-linux-x86_64.jar -netty-transport-native-kqueue/4.1.106.Final/osx-aarch_64/netty-transport-native-kqueue-4.1.106.Final-osx-aarch_64.jar -netty-transport-native-kqueue/4.1.106.Final/osx-x86_64/netty-transport-native-kqueue-4.1.106.Final-osx-x86_64.jar -netty-transport-native-unix-common/4.1.106.Final//netty-transport-native-unix-common-4.1.106.Final.jar -netty-transport/4.1.106.Final//netty-transport-4.1.106.Final.jar +netty-tcnative-boringssl-static/2.0.62.Final/linux-aarch_64/netty-tcnative-boringssl-static-2.0.62.Final-linux-aarch_64.jar +netty
(spark) branch master updated: [SPARK-47084][BUILD] Upgrade joda-time to 2.12.7
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 85108b0cb605 [SPARK-47084][BUILD] Upgrade joda-time to 2.12.7 85108b0cb605 is described below commit 85108b0cb6059e9a5301b63ab266084defd0ddf2 Author: panbingkun AuthorDate: Mon Feb 19 10:15:37 2024 +0800 [SPARK-47084][BUILD] Upgrade joda-time to 2.12.7 ### What changes were proposed in this pull request? The pr aims to upgrade `joda-time` from `2.12.6` to `2.12.7`. ### Why are the changes needed? The version `DateTimeZone` data updated to version `2024agtz`. The full release notes: https://www.joda.org/joda-time/changes-report.html#a2.12.7 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45153 from panbingkun/SPARK-47084. Authored-by: panbingkun Signed-off-by: yangjie01 --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 0b619a249e96..5aabe0e4aef1 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -139,7 +139,7 @@ jetty-util/10.0.19//jetty-util-10.0.19.jar jline/2.14.6//jline-2.14.6.jar jline/3.22.0//jline-3.22.0.jar jna/5.13.0//jna-5.13.0.jar -joda-time/2.12.6//joda-time-2.12.6.jar +joda-time/2.12.7//joda-time-2.12.7.jar jodd-core/3.5.2//jodd-core-3.5.2.jar jpam/1.1//jpam-1.1.jar json/1.8//json-1.8.jar diff --git a/pom.xml b/pom.xml index a14f2d255a90..64931dd73282 100644 --- a/pom.xml +++ b/pom.xml @@ -208,7 +208,7 @@ Because it transitions Jakarta REST API from javax to jakarta package. --> 2.41 -2.12.6 +2.12.7 3.5.2 3.0.0 0.12.0 - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [MINOR][INFRA][DOCS] Remove undated comment in build_and_test.yml
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 7c60fe21a29d [MINOR][INFRA][DOCS] Remove undated comment in build_and_test.yml 7c60fe21a29d is described below commit 7c60fe21a29dd852de01da214c84e6a3deb38e31 Author: Hyukjin Kwon AuthorDate: Mon Feb 19 10:13:52 2024 +0800 [MINOR][INFRA][DOCS] Remove undated comment in build_and_test.yml ### What changes were proposed in this pull request? This PR removes outdated comment. We don't use branch-3.3 anymore ### Why are the changes needed? To remove obsolete information ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? N/A ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45149 from HyukjinKwon/minor-ga. Authored-by: Hyukjin Kwon Signed-off-by: yangjie01 --- .github/workflows/build_and_test.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index c578d5079be8..bad34fd746ba 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -285,7 +285,6 @@ jobs: infra-image: name: "Base image build" needs: precondition -# Currently, enable docker build from cache for `master` and branch (since 3.4) jobs if: >- fromJson(needs.precondition.outputs.required).pyspark == 'true' || fromJson(needs.precondition.outputs.required).lint == 'true' || - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47073][BUILD] Upgrade several Maven plugins to the latest versions
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 2abd3a2f445e [SPARK-47073][BUILD] Upgrade several Maven plugins to the latest versions 2abd3a2f445e is described below commit 2abd3a2f445e86337ad94da19f301cb2b8bc232f Author: Dongjoon Hyun AuthorDate: Fri Feb 16 22:10:41 2024 +0800 [SPARK-47073][BUILD] Upgrade several Maven plugins to the latest versions ### What changes were proposed in this pull request? This PR aims to upgrade several maven plugins to the latest versions for Apache Spark 4.0.0. ### Why are the changes needed? To bring the latest bug fixes. - `versions-maven-plugin` from 2.16.0 to 2.16.2. - `maven-enforcer-plugin` from 3.3.0 to 3.4.1. - `maven-compiler-plugin` from 3.11.0 to 3.12.1. - `maven-surefire-plugin` from 3.1.2 to 3.2.5. - `maven-clean-plugin` from 3.3.1 to 3.3.2. - `maven-javadoc-plugin` from 3.5.0 to 3.6.3. - `maven-shade-plugin` from 3.5.0 to 3.5.1. - `maven-dependency-plugin` from 3.6.0 to 3.6.1. - `maven-checkstyle-plugin` from 3.3.0 to 3.3.1. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs and manual. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45136 from dongjoon-hyun/SPARK-47073. Authored-by: Dongjoon Hyun Signed-off-by: yangjie01 --- pom.xml | 18 +- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index b83378af30ff..cd669dd02b6d 100644 --- a/pom.xml +++ b/pom.xml @@ -179,7 +179,7 @@ 4.7.1 false -2.16.0 +2.16.2 true true @@ -2852,7 +2852,7 @@ org.apache.maven.plugins maven-enforcer-plugin - 3.3.0 + 3.4.1 enforce-versions @@ -3035,7 +3035,7 @@ org.apache.maven.plugins maven-compiler-plugin - 3.11.0 + 3.12.1 ${java.version} ${java.version} @@ -3052,7 +3052,7 @@ org.apache.maven.plugins maven-surefire-plugin - 3.1.2 + 3.2.5 @@ -3189,7 +3189,7 @@ org.apache.maven.plugins maven-clean-plugin - 3.3.1 + 3.3.2 @@ -3216,7 +3216,7 @@ org.apache.maven.plugins maven-javadoc-plugin - 3.5.0 + 3.6.3 -Xdoclint:all @@ -3272,7 +3272,7 @@ org.apache.maven.plugins maven-shade-plugin - 3.5.0 + 3.5.1 org.ow2.asm @@ -3299,7 +3299,7 @@ org.apache.maven.plugins maven-dependency-plugin - 3.6.0 + 3.6.1 default-cli @@ -3439,7 +3439,7 @@ org.apache.maven.plugins maven-checkstyle-plugin -3.3.0 +3.3.1 false true - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47025][BUILD][TESTS] Upgrade `Guava` dependency in `docker-integration-tests` test module
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new d260f5753e9d [SPARK-47025][BUILD][TESTS] Upgrade `Guava` dependency in `docker-integration-tests` test module d260f5753e9d is described below commit d260f5753e9db00b84d85c34d1ebd21e36a98ac1 Author: Dongjoon Hyun AuthorDate: Tue Feb 13 08:41:00 2024 +0800 [SPARK-47025][BUILD][TESTS] Upgrade `Guava` dependency in `docker-integration-tests` test module ### What changes were proposed in this pull request? This PR aims to update `docker-integration-tests` test module to use the latest `Guava` test dependency. Specifically, - Switch from `provided` dependency to `test` dependency - Upgrade from version `19.0` to `33.0.0-jre`. ### Why are the changes needed? Previously, `docker-integration-tests` uses `Guava 19.0` dependency as `provided` scope because `docker-java-core` is using `Guava 19.0` still. - https://mvnrepository.com/artifact/com.github.docker-java/docker-java-core/3.3.4 ### Does this PR introduce _any_ user-facing change? No, `docker-integration-tests` is an integration test module. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45088 from dongjoon-hyun/SPARK-47025. Authored-by: Dongjoon Hyun Signed-off-by: yangjie01 --- connector/docker-integration-tests/pom.xml | 3 ++- project/SparkBuild.scala | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index 4cca3ef12ae5..f9430da052be 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -49,7 +49,8 @@ com.google.guava guava - 19.0 + 33.0.0-jre + test org.apache.spark diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 45b51cb0ff5b..24e2c814f99f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -951,7 +951,7 @@ object Unsafe { object DockerIntegrationTests { // This serves to override the override specified in DependencyOverrides: lazy val settings = Seq( -dependencyOverrides += "com.google.guava" % "guava" % "19.0" +dependencyOverrides += "com.google.guava" % "guava" % "33.0.0-jre" ) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-46615][CONNECT] Support s.c.immutable.ArraySeq in ArrowDeserializers
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 729fc8ec95e0 [SPARK-46615][CONNECT] Support s.c.immutable.ArraySeq in ArrowDeserializers 729fc8ec95e0 is described below commit 729fc8ec95e017bd6eead283c0b660b9c57a174d Author: panbingkun AuthorDate: Thu Feb 8 14:57:13 2024 +0800 [SPARK-46615][CONNECT] Support s.c.immutable.ArraySeq in ArrowDeserializers ### What changes were proposed in this pull request? The pr aims to support s.c.immutable.ArraySeq as customCollectionCls in ArrowDeserializers. ### Why are the changes needed? Because s.c.immutable.ArraySeq is a commonly used type in Scala 2.13, we should support it. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Update existed UT (SQLImplicitsTestSuite). ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44618 from panbingkun/SPARK-46615. Authored-by: panbingkun Signed-off-by: yangjie01 --- .../scala/org/apache/spark/sql/SQLImplicitsTestSuite.scala| 11 +++ .../spark/sql/connect/client/arrow/ArrowDeserializer.scala| 9 + .../spark/sql/connect/client/arrow/ArrowEncoderUtils.scala| 2 ++ 3 files changed, 22 insertions(+) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SQLImplicitsTestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SQLImplicitsTestSuite.scala index b2c13850a13a..3e4704b6ab8e 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SQLImplicitsTestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SQLImplicitsTestSuite.scala @@ -52,6 +52,7 @@ class SQLImplicitsTestSuite extends ConnectFunSuite with BeforeAndAfterAll { test("test implicit encoder resolution") { val spark = session +import org.apache.spark.util.ArrayImplicits._ import spark.implicits._ def testImplicit[T: Encoder](expected: T): Unit = { val encoder = encoderFor[T] @@ -84,6 +85,7 @@ class SQLImplicitsTestSuite extends ConnectFunSuite with BeforeAndAfterAll { testImplicit(booleans) testImplicit(booleans.toSeq) testImplicit(booleans.toSeq)(newBooleanSeqEncoder) +testImplicit(booleans.toImmutableArraySeq) val bytes = Array(76.toByte, 59.toByte, 121.toByte) testImplicit(bytes.head) @@ -91,6 +93,7 @@ class SQLImplicitsTestSuite extends ConnectFunSuite with BeforeAndAfterAll { testImplicit(bytes) testImplicit(bytes.toSeq) testImplicit(bytes.toSeq)(newByteSeqEncoder) +testImplicit(bytes.toImmutableArraySeq) val shorts = Array(21.toShort, (-213).toShort, 14876.toShort) testImplicit(shorts.head) @@ -98,6 +101,7 @@ class SQLImplicitsTestSuite extends ConnectFunSuite with BeforeAndAfterAll { testImplicit(shorts) testImplicit(shorts.toSeq) testImplicit(shorts.toSeq)(newShortSeqEncoder) +testImplicit(shorts.toImmutableArraySeq) val ints = Array(4, 6, 5) testImplicit(ints.head) @@ -105,6 +109,7 @@ class SQLImplicitsTestSuite extends ConnectFunSuite with BeforeAndAfterAll { testImplicit(ints) testImplicit(ints.toSeq) testImplicit(ints.toSeq)(newIntSeqEncoder) +testImplicit(ints.toImmutableArraySeq) val longs = Array(System.nanoTime(), System.currentTimeMillis()) testImplicit(longs.head) @@ -112,6 +117,7 @@ class SQLImplicitsTestSuite extends ConnectFunSuite with BeforeAndAfterAll { testImplicit(longs) testImplicit(longs.toSeq) testImplicit(longs.toSeq)(newLongSeqEncoder) +testImplicit(longs.toImmutableArraySeq) val floats = Array(3f, 10.9f) testImplicit(floats.head) @@ -119,6 +125,7 @@ class SQLImplicitsTestSuite extends ConnectFunSuite with BeforeAndAfterAll { testImplicit(floats) testImplicit(floats.toSeq) testImplicit(floats.toSeq)(newFloatSeqEncoder) +testImplicit(floats.toImmutableArraySeq) val doubles = Array(23.78d, -329.6d) testImplicit(doubles.head) @@ -126,22 +133,26 @@ class SQLImplicitsTestSuite extends ConnectFunSuite with BeforeAndAfterAll { testImplicit(doubles) testImplicit(doubles.toSeq) testImplicit(doubles.toSeq)(newDoubleSeqEncoder) +testImplicit(doubles.toImmutableArraySeq) val strings = Array("foo", "baz", "bar") testImplicit(strings.head) testImplicit(strings) testImplicit(strings.toSeq) testImplicit(strings.toSeq)(newStringSeqEncoder) +testImplicit(strings.toImmutableArraySeq) val myTypes = Array(MyType(12L, Math.E, Math.PI), MyType(0, 0, 0)) testImplicit(myTypes.head) testImplicit(myTypes) testImplicit(myType
(spark) branch branch-3.5 updated: [SPARK-46400][CORE][SQL][3.5] When there are corrupted files in the local maven repo, skip this cache and try again
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 77f8b38a1091 [SPARK-46400][CORE][SQL][3.5] When there are corrupted files in the local maven repo, skip this cache and try again 77f8b38a1091 is described below commit 77f8b38a1091aa51af32dc790b61ae54ac47a2c2 Author: panbingkun AuthorDate: Thu Feb 8 14:41:51 2024 +0800 [SPARK-46400][CORE][SQL][3.5] When there are corrupted files in the local maven repo, skip this cache and try again ### What changes were proposed in this pull request? The pr aims to - fix potential bug(ie: https://github.com/apache/spark/pull/44208) and enhance user experience. - make the code more compliant with standards Backport above to branch 3.5. Master branch pr: https://github.com/apache/spark/pull/44343 ### Why are the changes needed? We use the local maven repo as the first-level cache in ivy. The original intention was to reduce the time required to parse and obtain the ar, but when there are corrupted files in the local maven repo,The above mechanism will be directly interrupted and the prompt is very unfriendly, which will greatly confuse the user. Based on the original intention, we should skip the cache directly in similar situations. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45017 from panbingkun/branch-3.5_SPARK-46400. Authored-by: panbingkun Signed-off-by: yangjie01 --- .../org/apache/spark/deploy/SparkSubmit.scala | 116 + .../sql/hive/client/IsolatedClientLoader.scala | 4 + 2 files changed, 98 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index af35f451e370..0f0d8b6c07c0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -41,7 +41,7 @@ import org.apache.ivy.Ivy import org.apache.ivy.core.LogOptions import org.apache.ivy.core.module.descriptor._ import org.apache.ivy.core.module.id.{ArtifactId, ModuleId, ModuleRevisionId} -import org.apache.ivy.core.report.ResolveReport +import org.apache.ivy.core.report.{DownloadStatus, ResolveReport} import org.apache.ivy.core.resolve.ResolveOptions import org.apache.ivy.core.retrieve.RetrieveOptions import org.apache.ivy.core.settings.IvySettings @@ -1226,7 +1226,7 @@ private[spark] object SparkSubmitUtils extends Logging { s"be whitespace. The artifactId provided is: ${splits(1)}") require(splits(2) != null && splits(2).trim.nonEmpty, s"The version cannot be null or " + s"be whitespace. The version provided is: ${splits(2)}") - new MavenCoordinate(splits(0), splits(1), splits(2)) + MavenCoordinate(splits(0), splits(1), splits(2)) } } @@ -1241,21 +1241,27 @@ private[spark] object SparkSubmitUtils extends Logging { } /** - * Extracts maven coordinates from a comma-delimited string + * Create a ChainResolver used by Ivy to search for and resolve dependencies. + * * @param defaultIvyUserDir The default user path for Ivy + * @param useLocalM2AsCache Whether to use the local maven repo as a cache * @return A ChainResolver used by Ivy to search for and resolve dependencies. */ - def createRepoResolvers(defaultIvyUserDir: File): ChainResolver = { + def createRepoResolvers( + defaultIvyUserDir: File, + useLocalM2AsCache: Boolean = true): ChainResolver = { // We need a chain resolver if we want to check multiple repositories val cr = new ChainResolver cr.setName("spark-list") -val localM2 = new IBiblioResolver -localM2.setM2compatible(true) -localM2.setRoot(m2Path.toURI.toString) -localM2.setUsepoms(true) -localM2.setName("local-m2-cache") -cr.add(localM2) +if (useLocalM2AsCache) { + val localM2 = new IBiblioResolver + localM2.setM2compatible(true) + localM2.setRoot(m2Path.toURI.toString) + localM2.setUsepoms(true) + localM2.setName("local-m2-cache") + cr.add(localM2) +} val localIvy = new FileSystemResolver val localIvyRoot = new File(defaultIvyUserDir, "local") @@ -1351,18 +1357,23 @@ private[spark] object SparkSubmitUtils extends Logging { /** * Build Ivy Settings using options with default resolvers + * * @param remoteRepos Comma-delimited string of remote repositories other than maven central * @param ivyPath The path to the local i
(spark) branch master updated: [MINOR][PYTHON][SQL][TESTS] Don't load Python Data Source when Python executable is not available even for testing
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new ab7fcacca41d [MINOR][PYTHON][SQL][TESTS] Don't load Python Data Source when Python executable is not available even for testing ab7fcacca41d is described below commit ab7fcacca41dad0ec2334b5d990bf36522fb5c82 Author: Hyukjin Kwon AuthorDate: Thu Feb 8 14:19:43 2024 +0800 [MINOR][PYTHON][SQL][TESTS] Don't load Python Data Source when Python executable is not available even for testing ### What changes were proposed in this pull request? This PR proposes to don't load Python Data Source Python executable is not available even for testing ### Why are the changes needed? Whether if we're in test or not, it can't work loading Python Data Sources anyway. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Manually tested. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45067 from HyukjinKwon/minor-checking. Authored-by: Hyukjin Kwon Signed-off-by: yangjie01 --- .../org/apache/spark/sql/execution/datasources/DataSourceManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceManager.scala index f63157b91efb..1b396675d909 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceManager.scala @@ -98,7 +98,7 @@ object DataSourceManager extends Logging { private def normalize(name: String): String = name.toLowerCase(Locale.ROOT) private def initialStaticDataSourceBuilders: Map[String, UserDefinedPythonDataSource] = { -if (Utils.isTesting || shouldLoadPythonDataSources) this.synchronized { +if (shouldLoadPythonDataSources) this.synchronized { if (dataSourceBuilders.isEmpty) { val maybeResult = try { Some(UserDefinedPythonDataSource.lookupAllDataSourcesInPython()) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47005][PYTHON][DOCS] Refine docstring of `asc_nulls_first/asc_nulls_last/desc_nulls_first/desc_nulls_last`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 45956f72d864 [SPARK-47005][PYTHON][DOCS] Refine docstring of `asc_nulls_first/asc_nulls_last/desc_nulls_first/desc_nulls_last` 45956f72d864 is described below commit 45956f72d864701cd84635e9cac0a29592c08b1c Author: yangjie01 AuthorDate: Thu Feb 8 14:09:06 2024 +0800 [SPARK-47005][PYTHON][DOCS] Refine docstring of `asc_nulls_first/asc_nulls_last/desc_nulls_first/desc_nulls_last` ### What changes were proposed in this pull request? This pr refine docstring of `asc_nulls_first/asc_nulls_last/desc_nulls_first/desc_nulls_last` and add some new examples. ### Why are the changes needed? To improve PySpark documentation ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #45066 from LuciferYang/sort-funcs. Authored-by: yangjie01 Signed-off-by: yangjie01 --- python/pyspark/sql/functions/builtin.py | 148 +++- 1 file changed, 128 insertions(+), 20 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 110006df4317..6320f9b922ee 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -2889,7 +2889,7 @@ def getbit(col: "ColumnOrName", pos: "ColumnOrName") -> Column: @_try_remote_functions def asc_nulls_first(col: "ColumnOrName") -> Column: """ -Returns a sort expression based on the ascending order of the given +Sort Function: Returns a sort expression based on the ascending order of the given column name, and null values return before non-null values. .. versionadded:: 2.4.0 @@ -2909,10 +2909,11 @@ def asc_nulls_first(col: "ColumnOrName") -> Column: Examples ->>> df1 = spark.createDataFrame([(1, "Bob"), -... (0, None), -... (2, "Alice")], ["age", "name"]) ->>> df1.sort(asc_nulls_first(df1.name)).show() +Example 1: Sorting a DataFrame with null values in ascending order + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([(1, "Bob"), (0, None), (2, "Alice")], ["age", "name"]) +>>> df.sort(sf.asc_nulls_first(df.name)).show() +---+-+ |age| name| +---+-+ @@ -2921,6 +2922,32 @@ def asc_nulls_first(col: "ColumnOrName") -> Column: | 1| Bob| +---+-+ +Example 2: Sorting a DataFrame with multiple columns, null values in ascending order + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame( +... [(1, "Bob", None), (0, None, "Z"), (2, "Alice", "Y")], ["age", "name", "grade"]) +>>> df.sort(sf.asc_nulls_first(df.name), sf.asc_nulls_first(df.grade)).show() ++---+-+-+ +|age| name|grade| ++---+-+-+ +| 0| NULL|Z| +| 2|Alice|Y| +| 1| Bob| NULL| ++---+-+-+ + +Example 3: Sorting a DataFrame with null values in ascending order using column name string + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([(1, "Bob"), (0, None), (2, "Alice")], ["age", "name"]) +>>> df.sort(sf.asc_nulls_first("name")).show() ++---+-+ +|age| name| ++---+-+ +| 0| NULL| +| 2|Alice| +| 1| Bob| ++---+-+ """ return ( col.asc_nulls_first() @@ -2932,7 +2959,7 @@ def asc_nulls_first(col: "ColumnOrName") -> Column: @_try_remote_functions def asc_nulls_last(col: "ColumnOrName") -> Column: """ -Returns a sort expression based on the ascending order of the given +Sort Function: Returns a sort expression based on the ascending order of the given column name, and null values appear after non-null values. .. versionadded:: 2.4.0 @@ -2952,10 +2979,11 @@ def asc_nulls_last(col: "ColumnOrName") -> Column: Examples ->>> df1 = spark.createDataFrame([(0, None), -... (1, "Bob"), -... (2, "Alice")], ["age", "name"]) ->>> df1
(spark) branch master updated: [SPARK-46987][CONNECT] `ProtoUtils.abbreviate` avoid unnecessary `setField` operation
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new a95aa7a7dda1 [SPARK-46987][CONNECT] `ProtoUtils.abbreviate` avoid unnecessary `setField` operation a95aa7a7dda1 is described below commit a95aa7a7dda1a5a2cfee69b3c132c524c0e01c7d Author: Ruifeng Zheng AuthorDate: Wed Feb 7 10:26:34 2024 +0800 [SPARK-46987][CONNECT] `ProtoUtils.abbreviate` avoid unnecessary `setField` operation ### What changes were proposed in this pull request? `ProtoUtils.abbreviate` avoid unnecessary `setField` operation ### Why are the changes needed? according to the [API reference](https://protobuf.dev/reference/java/api-docs/com/google/protobuf/Message.html#toBuilder--): > Message.Builder toBuilder() Constructs a builder initialized with the current message. Use this to derive a new message from the current one. the builder we used already has all the fields, so we only need to update the truncated fields. ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #45045 from zhengruifeng/connect_redaction_nit. Authored-by: Ruifeng Zheng Signed-off-by: yangjie01 --- .../scala/org/apache/spark/sql/connect/common/ProtoUtils.scala| 8 +--- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoUtils.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoUtils.scala index 44de2350b9fd..2f31b63acf87 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoUtils.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoUtils.scala @@ -43,8 +43,6 @@ private[connect] object ProtoUtils { val threshold = thresholds.getOrElse(STRING, MAX_STRING_SIZE) if (size > threshold) { builder.setField(field, createString(string.take(threshold), size)) -} else { - builder.setField(field, string) } case (field: FieldDescriptor, byteString: ByteString) @@ -57,8 +55,6 @@ private[connect] object ProtoUtils { byteString .substring(0, threshold) .concat(createTruncatedByteString(size))) -} else { - builder.setField(field, byteString) } case (field: FieldDescriptor, byteArray: Array[Byte]) @@ -71,8 +67,6 @@ private[connect] object ProtoUtils { ByteString .copyFrom(byteArray, 0, threshold) .concat(createTruncatedByteString(size))) -} else { - builder.setField(field, byteArray) } // TODO(SPARK-43117): should also support 1, repeated msg; 2, map @@ -80,7 +74,7 @@ private[connect] object ProtoUtils { if field.getJavaType == FieldDescriptor.JavaType.MESSAGE && msg != null => builder.setField(field, abbreviate(msg, thresholds)) - case (field: FieldDescriptor, value: Any) => builder.setField(field, value) + case _ => } builder.build() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-46895][CORE] Replace Timer with single thread scheduled executor
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 5d5b3a54b7b5 [SPARK-46895][CORE] Replace Timer with single thread scheduled executor 5d5b3a54b7b5 is described below commit 5d5b3a54b7b5fb4308fe40da696ba805c72983fc Author: beliefer AuthorDate: Tue Feb 6 17:23:03 2024 +0800 [SPARK-46895][CORE] Replace Timer with single thread scheduled executor ### What changes were proposed in this pull request? This PR propose to replace `Timer` with single thread scheduled executor. ### Why are the changes needed? The javadoc recommends `ScheduledThreadPoolExecutor` instead of `Timer`. ![屏幕快照 2024-01-12 下午12 47 57](https://github.com/apache/spark/assets/8486025/4fc5ed61-6bb9-4768-915a-ad919a067d04) This change based on the following two points. **System time sensitivity** Timer scheduling is based on the absolute time of the operating system and is sensitive to the operating system's time. Once the operating system's time changes, Timer scheduling is no longer precise. The scheduled Thread Pool Executor scheduling is based on relative time and is not affected by changes in operating system time. **Are anomalies captured** Timer does not capture exceptions thrown by Timer Tasks, and in addition, Timer is single threaded. Once a scheduling task encounters an exception, the entire thread will terminate and other tasks that need to be scheduled will no longer be executed. The scheduled Thread Pool Executor implements scheduling functions based on a thread pool. After a task throws an exception, other tasks can still execute normally. ### Does this PR introduce _any_ user-facing change? 'No'. ### How was this patch tested? GA tests. ### Was this patch authored or co-authored using generative AI tooling? 'No'. Closes #44718 from beliefer/replace-timer-with-threadpool. Authored-by: beliefer Signed-off-by: yangjie01 --- .../main/scala/org/apache/spark/BarrierCoordinator.scala | 11 +++ .../main/scala/org/apache/spark/BarrierTaskContext.scala | 14 ++ .../org/apache/spark/scheduler/TaskSchedulerImpl.scala | 15 --- .../scala/org/apache/spark/ui/ConsoleProgressBar.scala | 11 --- .../main/scala/org/apache/spark/util/ThreadUtils.scala | 16 ++-- .../java/org/apache/spark/launcher/LauncherServer.java | 8 6 files changed, 47 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala index 9bc7ade2e5ad..942242107e22 100644 --- a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala @@ -17,8 +17,8 @@ package org.apache.spark -import java.util.{Timer, TimerTask} -import java.util.concurrent.ConcurrentHashMap +import java.util.TimerTask +import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import java.util.function.Consumer import scala.collection.mutable.{ArrayBuffer, HashSet} @@ -26,6 +26,7 @@ import scala.collection.mutable.{ArrayBuffer, HashSet} import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler.{LiveListenerBus, SparkListener, SparkListenerStageCompleted} +import org.apache.spark.util.ThreadUtils /** * For each barrier stage attempt, only at most one barrier() call can be active at any time, thus @@ -51,7 +52,8 @@ private[spark] class BarrierCoordinator( // TODO SPARK-25030 Create a Timer() in the mainClass submitted to SparkSubmit makes it unable to // fetch result, we shall fix the issue. - private lazy val timer = new Timer("BarrierCoordinator barrier epoch increment timer") + private lazy val timer = ThreadUtils.newSingleThreadScheduledExecutor( +"BarrierCoordinator barrier epoch increment timer") // Listen to StageCompleted event, clear corresponding ContextBarrierState. private val listener = new SparkListener { @@ -77,6 +79,7 @@ private[spark] class BarrierCoordinator( states.forEachValue(1, clearStateConsumer) states.clear() listenerBus.removeListener(listener) + ThreadUtils.shutdown(timer) } finally { super.onStop() } @@ -168,7 +171,7 @@ private[spark] class BarrierCoordinator( // we may timeout for the sync. if (requesters.isEmpty) { initTimerTask(this) - timer.schedule(timerTask, timeoutInSecs * 1000) + timer.schedule(timerTask, timeoutInSecs, TimeUnit.SECONDS) } // Add the requester to array of RPCCallContexts pending for reply.
(spark) branch master updated (0154c059cddb -> fd476c1c855a)
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 0154c059cddb [MINOR][DOCS] Remove Java 8/11 at `IgnoreUnrecognizedVMOptions` description add fd476c1c855a [SPARK-46969][SQL][TESTS] Recover `to_timestamp('366', 'DD')` test case of `datetime-parsing-invalid.sql` No new revisions were added by this update. Summary of changes: .../ansi/datetime-parsing-invalid.sql.out| 7 +++ .../analyzer-results/datetime-parsing-invalid.sql.out| 7 +++ .../sql-tests/inputs/datetime-parsing-invalid.sql| 3 +-- .../results/ansi/datetime-parsing-invalid.sql.out| 16 .../sql-tests/results/datetime-parsing-invalid.sql.out | 8 5 files changed, 39 insertions(+), 2 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-46918][YARN] Replace self-defined variables with Hadoop ContainerExitStatus
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e32a8249c6dd [SPARK-46918][YARN] Replace self-defined variables with Hadoop ContainerExitStatus e32a8249c6dd is described below commit e32a8249c6ddb15e01d2307964f2978f4a10ad56 Author: Cheng Pan AuthorDate: Tue Jan 30 20:17:11 2024 +0800 [SPARK-46918][YARN] Replace self-defined variables with Hadoop ContainerExitStatus ### What changes were proposed in this pull request? Replace the Spark self-defined `VMEM_EXCEEDED_EXIT_CODE` and `PMEM_EXCEEDED_EXIT_CODE` with Hadoop defined `ContainerExitStatus.KILLED_EXCEEDED_VMEM` and `ContainerExitStatus.KILLED_EXCEEDED_PMEM` which were introduced in YARN-2091(since Hadoop 2.5.0) ### Why are the changes needed? Minor code clean-up ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44950 from pan3793/SPARK-46918. Authored-by: Cheng Pan Signed-off-by: yangjie01 --- .../main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala | 9 ++--- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 736eaa52b81c..7f0469937fef 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -851,9 +851,6 @@ private[yarn] class YarnAllocator( onHostStr, completedContainer.getState, completedContainer.getExitStatus)) -// Hadoop 2.2.X added a ContainerExitStatus we should switch to use -// there are some exit status' we shouldn't necessarily count against us, but for -// now I think its ok as none of the containers are expected to exit. val exitStatus = completedContainer.getExitStatus val (exitCausedByApp, containerExitReason) = exitStatus match { case _ if shutdown => @@ -867,7 +864,7 @@ private[yarn] class YarnAllocator( // just as easily finish on any other executor. See SPARK-8167. (false, s"Container ${containerId}${onHostStr} was preempted.") // Should probably still count memory exceeded exit codes towards task failures - case VMEM_EXCEEDED_EXIT_CODE => + case ContainerExitStatus.KILLED_EXCEEDED_VMEM => val vmemExceededPattern = raw"$MEM_REGEX of $MEM_REGEX virtual memory used".r val diag = vmemExceededPattern.findFirstIn(completedContainer.getDiagnostics) .map(_.concat(".")).getOrElse("") @@ -876,7 +873,7 @@ private[yarn] class YarnAllocator( s"${YarnConfiguration.NM_VMEM_PMEM_RATIO} or disabling " + s"${YarnConfiguration.NM_VMEM_CHECK_ENABLED} because of YARN-4714." (true, message) - case PMEM_EXCEEDED_EXIT_CODE => + case ContainerExitStatus.KILLED_EXCEEDED_PMEM => val pmemExceededPattern = raw"$MEM_REGEX of $MEM_REGEX physical memory used".r val diag = pmemExceededPattern.findFirstIn(completedContainer.getDiagnostics) .map(_.concat(".")).getOrElse("") @@ -1025,8 +1022,6 @@ private[yarn] class YarnAllocator( private object YarnAllocator { val MEM_REGEX = "[0-9.]+ [KMG]B" - val VMEM_EXCEEDED_EXIT_CODE = -103 - val PMEM_EXCEEDED_EXIT_CODE = -104 val DECOMMISSIONING_NODES_CACHE_SIZE = 200 val NOT_APP_AND_SYSTEM_FAULT_EXIT_STATUS = Set( - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-46898][CONNECT] Simplify the protobuf function transformation in Planner
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 56633e697571 [SPARK-46898][CONNECT] Simplify the protobuf function transformation in Planner 56633e697571 is described below commit 56633e69757174da8a7dd8f4ea5298fd0a00e656 Author: Ruifeng Zheng AuthorDate: Mon Jan 29 13:55:59 2024 +0800 [SPARK-46898][CONNECT] Simplify the protobuf function transformation in Planner ### What changes were proposed in this pull request? Simplify the protobuf function transformation in Planner ### Why are the changes needed? make `transformUnregisteredFunction` simple and reuse existing helper function ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #44925 from zhengruifeng/connect_proto_simple. Authored-by: Ruifeng Zheng Signed-off-by: yangjie01 --- .../sql/connect/planner/SparkConnectPlanner.scala | 80 +++--- 1 file changed, 25 insertions(+), 55 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 3e59b2644755..977bff690bac 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -1710,53 +1710,6 @@ class SparkConnectPlanner( */ private def transformUnregisteredFunction( fun: proto.Expression.UnresolvedFunction): Option[Expression] = { -def extractArgsOfProtobufFunction( -functionName: String, -argumentsCount: Int, -children: collection.Seq[Expression]) -: (String, Option[Array[Byte]], Map[String, String]) = { - val messageClassName = children(1) match { -case Literal(s, StringType) if s != null => s.toString -case other => - throw InvalidPlanInput( -s"MessageClassName in $functionName should be a literal string, but got $other") - } - val (binaryFileDescSetOpt, options) = if (argumentsCount == 2) { -(None, Map.empty[String, String]) - } else if (argumentsCount == 3) { -children(2) match { - case Literal(b, BinaryType) if b != null => -(Some(b.asInstanceOf[Array[Byte]]), Map.empty[String, String]) - case UnresolvedFunction(Seq("map"), arguments, _, _, _, _) => -(None, ExprUtils.convertToMapData(CreateMap(arguments))) - case other => -throw InvalidPlanInput( - s"The valid type for the 3rd arg in $functionName " + -s"is binary or map, but got $other") -} - } else if (argumentsCount == 4) { -val fileDescSetOpt = children(2) match { - case Literal(b, BinaryType) if b != null => -Some(b.asInstanceOf[Array[Byte]]) - case other => -throw InvalidPlanInput( - s"DescFilePath in $functionName should be a literal binary, but got $other") -} -val map = children(3) match { - case UnresolvedFunction(Seq("map"), arguments, _, _, _, _) => -ExprUtils.convertToMapData(CreateMap(arguments)) - case other => -throw InvalidPlanInput( - s"Options in $functionName should be created by map, but got $other") -} -(fileDescSetOpt, map) - } else { -throw InvalidPlanInput( - s"$functionName requires 2 ~ 4 arguments, but got $argumentsCount ones!") - } - (messageClassName, binaryFileDescSetOpt, options) -} - fun.getFunctionName match { case "product" if fun.getArgumentsCount == 1 => Some( @@ -1979,17 +1932,13 @@ class SparkConnectPlanner( // Protobuf-specific functions case "from_protobuf" if Seq(2, 3, 4).contains(fun.getArgumentsCount) => val children = fun.getArgumentsList.asScala.map(transformExpression) -val (messageClassName, binaryFileDescSetOpt, options) = - extractArgsOfProtobufFunction("from_protobuf", fun.getArgumentsCount, children) -Some( - ProtobufDataToCatalyst(children.head, messageClassName, binaryFileDescSetOpt, options)) +val (msgName, desc, options) = extractProtobufArgs(children.toSeq) +Some(ProtobufDataToCatalyst(children(0), msgName, desc, options)) case "to_protobuf" if Seq
(spark) branch master updated: [SPARK-46432][BUILD] Upgrade Netty to 4.1.106.Final
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 44b163d281b9 [SPARK-46432][BUILD] Upgrade Netty to 4.1.106.Final 44b163d281b9 is described below commit 44b163d281b9773cab9995e690ec3f4751c8be69 Author: panbingkun AuthorDate: Fri Jan 26 11:12:11 2024 +0800 [SPARK-46432][BUILD] Upgrade Netty to 4.1.106.Final ### What changes were proposed in this pull request? The pr aims to upgrade `Netty` from `4.1.100.Final` to `4.1.106.Final`. ### Why are the changes needed? - To bring the latest bug fixes Automatically close Http2StreamChannel when Http2FrameStreamExceptionreaches end ofChannelPipeline ([#13651](https://github.com/netty/netty/pull/13651)) Symbol not found: _netty_jni_util_JNI_OnLoad ([#13695](https://github.com/netty/netty/issues/13728)) - 4.1.106.Final release note: https://netty.io/news/2024/01/19/4-1-106-Final.html - 4.1.105.Final release note: https://netty.io/news/2024/01/16/4-1-105-Final.html - 4.1.104.Final release note: https://netty.io/news/2023/12/15/4-1-104-Final.html - 4.1.103.Final release note: https://netty.io/news/2023/12/13/4-1-103-Final.html - 4.1.101.Final release note: https://netty.io/news/2023/11/09/4-1-101-Final.html ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44384 from panbingkun/SPARK-46432. Lead-authored-by: panbingkun Co-authored-by: panbingkun Signed-off-by: yangjie01 --- common/network-yarn/pom.xml | 44 ++- dev/deps/spark-deps-hadoop-3-hive-2.3 | 37 +++-- pom.xml | 2 +- 3 files changed, 43 insertions(+), 40 deletions(-) diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index c809bdfbbc1d..3f2ae21eeb3b 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -173,27 +173,29 @@ unpack package - - - - - - - - - - - - - + + + + + + + + + + + + + + run diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 4ee0f5a41191..71f9ac8665b0 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -191,16 +191,16 @@ metrics-jmx/4.2.21//metrics-jmx-4.2.21.jar metrics-json/4.2.21//metrics-json-4.2.21.jar metrics-jvm/4.2.21//metrics-jvm-4.2.21.jar minlog/1.3.0//minlog-1.3.0.jar -netty-all/4.1.100.Final//netty-all-4.1.100.Final.jar -netty-buffer/4.1.100.Final//netty-buffer-4.1.100.Final.jar -netty-codec-http/4.1.100.Final//netty-codec-http-4.1.100.Final.jar -netty-codec-http2/4.1.100.Final//netty-codec-http2-4.1.100.Final.jar -netty-codec-socks/4.1.100.Final//netty-codec-socks-4.1.100.Final.jar -netty-codec/4.1.100.Final//netty-codec-4.1.100.Final.jar -netty-common/4.1.100.Final//netty-common-4.1.100.Final.jar -netty-handler-proxy/4.1.100.Final//netty-handler-proxy-4.1.100.Final.jar -netty-handler/4.1.100.Final//netty-handler-4.1.100.Final.jar -netty-resolver/4.1.100.Final//netty-resolver-4.1.100.Final.jar +netty-all/4.1.106.Final//netty-all-4.1.106.Final.jar +netty-buffer/4.1.106.Final//netty-buffer-4.1.106.Final.jar +netty-codec-http/4.1.106.Final//netty-codec-http-4.1.106.Final.jar +netty-codec-http2/4.1.106.Final//netty-codec-http2-4.1.106.Final.jar +netty-codec-socks/4.1.106.Final//netty-codec-socks-4.1.106.Final.jar +netty-codec/4.1.106.Final//netty-codec-4.1.106.Final.jar +netty-common/4.1.106.Final//netty-common-4.1.106.Final.jar +netty-handler-proxy/4.1.106.Final//netty-handler-proxy-4.1.106.Final.jar +netty-handler/4.1.106.Final//netty-handler-4.1.106.Final.jar +netty-resolver/4.1.106.Final//netty-resolver-4.1.106.Final.jar netty-tcnative-boringssl-static/2.0.61.Final//netty-tcnative-boringssl-static-2.0.61.Final.jar netty-tcnative-boringssl-static/2.0.61.Final/linux-aarch_64/netty-tcnative-boringssl-static-2.0.61.Final-linux-aarch_64.jar netty-tcnative-boringssl-static/2.0.61.Final/linux-x86_64/netty-tcnative-boringssl-static-2.0.61.Final-linux-x86_64.jar @@ -208,14 +208,15 @@ netty-tcnative-boringssl-static/2.0.61.Final/osx
(spark) branch master updated: [SPARK-46787][CONNECT] `bloomFilter` function should throw `AnalysisException` for invalid input
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new d3a8b303c5c0 [SPARK-46787][CONNECT] `bloomFilter` function should throw `AnalysisException` for invalid input d3a8b303c5c0 is described below commit d3a8b303c5c056ec0863d20b33de6f1a5865dfae Author: Ruifeng Zheng AuthorDate: Thu Jan 25 11:11:18 2024 +0800 [SPARK-46787][CONNECT] `bloomFilter` function should throw `AnalysisException` for invalid input ### What changes were proposed in this pull request? `bloomFilter` function should throw `AnalysisException` for invalid input ### Why are the changes needed? 1. `BloomFilterAggregate` itself validates the input, and throws meaningful errors. we should not handle those invalid input and throw `InvalidPlanInput` in Planner. 2. to be consistent with vanilla Scala API and other functions ### Does this PR introduce _any_ user-facing change? yes, `InvalidPlanInput` -> `AnalysisException` ### How was this patch tested? updated CI ### Was this patch authored or co-authored using generative AI tooling? no Closes #44821 from zhengruifeng/connect_bloom_filter_agg_error. Authored-by: Ruifeng Zheng Signed-off-by: yangjie01 --- .../apache/spark/sql/DataFrameStatFunctions.scala | 28 -- .../spark/sql/ClientDataFrameStatSuite.scala | 20 .../sql/connect/planner/SparkConnectPlanner.scala | 25 +-- 3 files changed, 16 insertions(+), 57 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala index 4daa9fa88e66..4eef26da706f 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala @@ -22,7 +22,6 @@ import java.io.ByteArrayInputStream import scala.jdk.CollectionConverters._ -import org.apache.spark.SparkException import org.apache.spark.connect.proto.{Relation, StatSampleBy} import org.apache.spark.sql.DataFrameStatFunctions.approxQuantileResultEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BinaryEncoder, PrimitiveDoubleEncoder} @@ -599,7 +598,7 @@ final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, roo * @since 3.5.0 */ def bloomFilter(colName: String, expectedNumItems: Long, fpp: Double): BloomFilter = { -buildBloomFilter(Column(colName), expectedNumItems, -1L, fpp) +bloomFilter(Column(colName), expectedNumItems, fpp) } /** @@ -614,7 +613,8 @@ final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, roo * @since 3.5.0 */ def bloomFilter(col: Column, expectedNumItems: Long, fpp: Double): BloomFilter = { -buildBloomFilter(col, expectedNumItems, -1L, fpp) +val numBits = BloomFilter.optimalNumOfBits(expectedNumItems, fpp) +bloomFilter(col, expectedNumItems, numBits) } /** @@ -629,7 +629,7 @@ final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, roo * @since 3.5.0 */ def bloomFilter(colName: String, expectedNumItems: Long, numBits: Long): BloomFilter = { -buildBloomFilter(Column(colName), expectedNumItems, numBits, Double.NaN) +bloomFilter(Column(colName), expectedNumItems, numBits) } /** @@ -644,25 +644,7 @@ final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, roo * @since 3.5.0 */ def bloomFilter(col: Column, expectedNumItems: Long, numBits: Long): BloomFilter = { -buildBloomFilter(col, expectedNumItems, numBits, Double.NaN) - } - - private def buildBloomFilter( - col: Column, - expectedNumItems: Long, - numBits: Long, - fpp: Double): BloomFilter = { -def numBitsValue: Long = if (!fpp.isNaN) { - BloomFilter.optimalNumOfBits(expectedNumItems, fpp) -} else { - numBits -} - -if (fpp <= 0d || fpp >= 1d) { - throw new SparkException("False positive probability must be within range (0.0, 1.0)") -} -val agg = Column.fn("bloom_filter_agg", col, lit(expectedNumItems), lit(numBitsValue)) - +val agg = Column.fn("bloom_filter_agg", col, lit(expectedNumItems), lit(numBits)) val ds = sparkSession.newDataset(BinaryEncoder) { builder => builder.getProjectBuilder .setInput(root) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDataFrameStatSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDataFrameStatSuite.scala index d0a89f67
(spark) branch master updated: [SPARK-46826][INFRA] Reset `grpcio` installation version of `Python linter dependencies for branch-3.4/branch-3.5`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 74b6301f152d [SPARK-46826][INFRA] Reset `grpcio` installation version of `Python linter dependencies for branch-3.4/branch-3.5` 74b6301f152d is described below commit 74b6301f152d438246756d665a3aa69e401e6273 Author: yangjie01 AuthorDate: Wed Jan 24 19:06:31 2024 +0800 [SPARK-46826][INFRA] Reset `grpcio` installation version of `Python linter dependencies for branch-3.4/branch-3.5` ### What changes were proposed in this pull request? https://github.com/apache/spark/pull/43942 upgraded the `grpcio` version and simultaneously upgraded the `grpcio` version installed in `Install Python linter dependencies for branch-3.4` and `Install Python linter dependencies for branch-3.5` in `build_and_test.yml`. These two steps are used to install Python linter dependencies for `branch-3.4/branch-3.5` in daily tests. They should use the same configuration as `branch-3.4/branch-3.5` for safety. So this pr reset the version of grpc [...] - branch-3.4 https://github.com/apache/spark/blob/e56bd97c04c184104046e51e6759e616c86683fa/.github/workflows/build_and_test.yml#L588-L595 - branch-3.5 https://github.com/apache/spark/blob/0956db6901bf03d2d948b23f00bcd6e74a0c251b/.github/workflows/build_and_test.yml#L637-L644 ### Why are the changes needed? The versions of the dependencies installed in `Install Python linter dependencies for branch-3.4` and `Install Python linter dependencies for branch-3.5` should be consistent with `branch-3.4/branch-3.5`. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Monitor GA after merged ### Was this patch authored or co-authored using generative AI tooling? No Closes #44866 from LuciferYang/SPARK-46826. Authored-by: yangjie01 Signed-off-by: yangjie01 --- .github/workflows/build_and_test.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 4038f63fb0dc..1d98727a4231 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -687,14 +687,14 @@ jobs: # SPARK-44554: Copy from https://github.com/apache/spark/blob/a05c27e85829fe742c1828507a1fd180cdc84b54/.github/workflows/build_and_test.yml#L571-L578 # Should delete this section after SPARK 3.4 EOL. python3.9 -m pip install 'flake8==3.9.0' pydata_sphinx_theme 'mypy==0.920' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' numpydoc 'jinja2<3.0.0' 'black==22.6.0' -python3.9 -m pip install 'pandas-stubs==1.2.0.53' ipython 'grpcio==1.59.3' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' +python3.9 -m pip install 'pandas-stubs==1.2.0.53' ipython 'grpcio==1.48.1' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' - name: Install Python linter dependencies for branch-3.5 if: inputs.branch == 'branch-3.5' run: | # SPARK-45212: Copy from https://github.com/apache/spark/blob/555c8def51e5951c7bf5165a332795e9e330ec9d/.github/workflows/build_and_test.yml#L631-L638 # Should delete this section after SPARK 3.5 EOL. python3.9 -m pip install 'flake8==3.9.0' pydata_sphinx_theme 'mypy==0.982' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' numpydoc 'jinja2<3.0.0' 'black==22.6.0' -python3.9 -m pip install 'pandas-stubs==1.2.0.53' ipython 'grpcio==1.59.3' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' +python3.9 -m pip install 'pandas-stubs==1.2.0.53' ipython 'grpcio==1.56.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' - name: Install Python dependencies for python linter and documentation generation if: inputs.branch != 'branch-3.4' && inputs.branch != 'branch-3.5' run: | - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-45593][BUILD][FOLLOWUP] Correct relocation connect guava dependency
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new ea0436752fe5 [SPARK-45593][BUILD][FOLLOWUP] Correct relocation connect guava dependency ea0436752fe5 is described below commit ea0436752fe5b2a1ca58fad3877f48905b3c2d8a Author: yikaifei AuthorDate: Wed Jan 24 19:03:00 2024 +0800 [SPARK-45593][BUILD][FOLLOWUP] Correct relocation connect guava dependency ### What changes were proposed in this pull request? This PR amins to correct relocation connect guava dependency and remove duplicate connect-common from SBT build jars. **Item 1:** In https://github.com/apache/spark/pull/43436, We fixed the connect module dependency on guava, but the dependency on guava was relocation incorrectly. - connect server and connect client jvm don't relocation guava dependency, this runs the risk of causing conflict problems; - connect common relocation does not take effect because it defines conflicting relocation rules with the parent pom(Now, we remove guava dependency from connect-common as it never use this library); **Item2:** Remove duplicate connect-common from SBT build jars as it is shaded in the spark connect. Also, in fact, before this PR, in the output jars built using SBT, connect-common and common-server were the same thing, because they both hit the `jar.getName.contains("spark-connect")` condition. ### Why are the changes needed? Bugfix ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? GA ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44801 from Yikf/SPARK-45593-SBT. Authored-by: yikaifei Signed-off-by: yangjie01 --- connector/connect/client/jvm/pom.xml | 22 +- connector/connect/common/pom.xml | 25 - connector/connect/server/pom.xml | 26 ++ project/SparkBuild.scala | 6 +- 4 files changed, 52 insertions(+), 27 deletions(-) diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index 9bedebf523a7..81ffb140226e 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -59,6 +59,18 @@ protobuf-java compile + + com.google.guava + guava + ${connect.guava.version} + compile + + + com.google.guava + failureaccess + ${guava.failureaccess.version} + compile + com.lihaoyi ammonite_${scala.version} @@ -105,6 +117,7 @@ true + com.google.guava:* com.google.android:* com.google.api.grpc:* com.google.code.findbugs:* @@ -124,6 +137,13 @@ + + com.google.common + ${spark.shade.packageName}.connect.guava + +com.google.common.** + + io.grpc ${spark.shade.packageName}.io.grpc @@ -135,7 +155,7 @@ com.google ${spark.shade.packageName}.com.google - + com.google.common.** diff --git a/connector/connect/common/pom.xml b/connector/connect/common/pom.xml index 336d83e04c15..b0f015246f4c 100644 --- a/connector/connect/common/pom.xml +++ b/connector/connect/common/pom.xml @@ -47,23 +47,6 @@ com.google.protobuf protobuf-java - - -com.google.guava -guava -${connect.guava.version} -compile - - -com.google.guava -failureaccess -${guava.failureaccess.version} -compile - io.grpc grpc-netty @@ -158,17 +141,9 @@ org.spark-project.spark:unused -com.google.guava:guava -com.google.guava:failureaccess org.apache.tomcat:annotations-api - - -com.google.common - ${spark.shade.packageName}.connect.guava - - diff --git a/connector/connect/server/pom.xml b/connector/connect/server/pom.xml index 82127f736ccb..bdea8a627000 100644 --- a/connector/connect/server/pom
(spark) branch master updated: [SPARK-44495][INFRA][K8S] Use the latest minikube in K8s IT
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new d114e262ba29 [SPARK-44495][INFRA][K8S] Use the latest minikube in K8s IT d114e262ba29 is described below commit d114e262ba295995bb6a85035c1717cd353a526a Author: Dongjoon Hyun AuthorDate: Sun Jan 21 08:53:16 2024 +0800 [SPARK-44495][INFRA][K8S] Use the latest minikube in K8s IT ### What changes were proposed in this pull request? This PR aims to recover GitHub Action K8s IT to use the latest Minikube and to make it sure that Apache Spark K8s module are tested with all Minikubes without any issues. **BEFORE** - Minikube: v1.30.1 - K8s: v1.26.3 **AFTER** - Minikube: v1.32.0 - K8s: v1.28.3 ### Why are the changes needed? - Previously, it was pinned due to the failure. - After this PR, we will track the latest Minikube and K8s version always. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44813 from dongjoon-hyun/SPARK-44495. Authored-by: Dongjoon Hyun Signed-off-by: yangjie01 --- .github/workflows/build_and_test.yml | 8 +++- .../deploy/k8s/integrationtest/KubernetesTestComponents.scala | 2 ++ .../apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala| 3 ++- .../spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala | 4 ++-- 4 files changed, 9 insertions(+), 8 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 99bb2b12e083..69636629ca9d 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -1063,9 +1063,7 @@ jobs: - name: start minikube run: | # See more in "Installation" https://minikube.sigs.k8s.io/docs/start/ - # curl -LO https://storage.googleapis.com/minikube/releases/latest/minikube-linux-amd64 - # TODO(SPARK-44495): Resume to use the latest minikube for k8s-integration-tests. - curl -LO https://storage.googleapis.com/minikube/releases/v1.30.1/minikube-linux-amd64 + curl -LO https://storage.googleapis.com/minikube/releases/latest/minikube-linux-amd64 sudo install minikube-linux-amd64 /usr/local/bin/minikube rm minikube-linux-amd64 # Github Action limit cpu:2, memory: 6947MB, limit to 2U6G for better resource statistic @@ -1074,7 +1072,7 @@ jobs: run: | kubectl get pods -A kubectl describe node - - name: Run Spark on K8S integration test (With driver cpu 0.5, executor cpu 0.2 limited) + - name: Run Spark on K8S integration test run: | # Prepare PV test PVC_TMP_DIR=$(mktemp -d) @@ -1084,7 +1082,7 @@ jobs: kubectl create clusterrolebinding serviceaccounts-cluster-admin --clusterrole=cluster-admin --group=system:serviceaccounts || true kubectl apply -f https://raw.githubusercontent.com/volcano-sh/volcano/v1.8.2/installer/volcano-development.yaml || true eval $(minikube docker-env) - build/sbt -Phadoop-3 -Psparkr -Pkubernetes -Pvolcano -Pkubernetes-integration-tests -Dspark.kubernetes.test.driverRequestCores=0.5 -Dspark.kubernetes.test.executorRequestCores=0.2 -Dspark.kubernetes.test.volcanoMaxConcurrencyJobNum=1 -Dtest.exclude.tags=local "kubernetes-integration-tests/test" + build/sbt -Phadoop-3 -Psparkr -Pkubernetes -Pvolcano -Pkubernetes-integration-tests -Dspark.kubernetes.test.volcanoMaxConcurrencyJobNum=1 -Dtest.exclude.tags=local "kubernetes-integration-tests/test" - name: Upload Spark on K8S integration tests log files if: ${{ !success() }} uses: actions/upload-artifact@v4 diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala index 3762c31538dc..9581a78619dd 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala @@ -75,6 +75,8 @@ private[spark] class KubernetesTestComponents(val kubernetesClient: KubernetesCl .set(UI_ENABLED.key, "true") .set("spark.kubernetes.submis
(spark) branch master updated: [SPARK-46767][PYTHON][DOCS] Refine docstring of `abs/acos/acosh`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new dceb8bdc72ef [SPARK-46767][PYTHON][DOCS] Refine docstring of `abs/acos/acosh` dceb8bdc72ef is described below commit dceb8bdc72ef24ffa1eb5c1820e6350207f042f5 Author: yangjie01 AuthorDate: Sat Jan 20 17:39:01 2024 +0800 [SPARK-46767][PYTHON][DOCS] Refine docstring of `abs/acos/acosh` ### What changes were proposed in this pull request? This pr refine docstring of `abs/acos/acosh` and add some new examples. ### Why are the changes needed? To improve PySpark documentation ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #44794 from LuciferYang/math-functions-1. Authored-by: yangjie01 Signed-off-by: yangjie01 --- python/pyspark/sql/functions/builtin.py | 169 ++-- 1 file changed, 137 insertions(+), 32 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 1f6d86de28dc..62400accba10 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -734,7 +734,7 @@ def try_sum(col: "ColumnOrName") -> Column: @_try_remote_functions def abs(col: "ColumnOrName") -> Column: """ -Computes the absolute value. +Mathematical Function: Computes the absolute value of the given column or expression. .. versionadded:: 1.3.0 @@ -744,22 +744,66 @@ def abs(col: "ColumnOrName") -> Column: Parameters -- col : :class:`~pyspark.sql.Column` or str -target column to compute on. +The target column or expression to compute the absolute value on. Returns --- :class:`~pyspark.sql.Column` -column for computed results. +A new column object representing the absolute value of the input. Examples ->>> df = spark.range(1) ->>> df.select(abs(lit(-1))).show() -+---+ -|abs(-1)| -+---+ -| 1| -+---+ +Example 1: Compute the absolute value of a negative number + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([(1, -1), (2, -2), (3, -3)], ["id", "value"]) +>>> df.select(sf.abs(df.value)).show() ++--+ +|abs(value)| ++--+ +| 1| +| 2| +| 3| ++--+ + +Example 2: Compute the absolute value of an expression + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([(1, 1), (2, -2), (3, 3)], ["id", "value"]) +>>> df.select(sf.abs(df.id - df.value)).show() ++-+ +|abs((id - value))| ++-+ +|0| +|4| +|0| ++-+ + +Example 3: Compute the absolute value of a column with null values + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([(1, None), (2, -2), (3, None)], ["id", "value"]) +>>> df.select(sf.abs(df.value)).show() ++--+ +|abs(value)| ++--+ +| NULL| +| 2| +| NULL| ++--+ + +Example 4: Compute the absolute value of a column with double values + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([(1, -1.5), (2, -2.5), (3, -3.5)], ["id", "value"]) +>>> df.select(sf.abs(df.value)).show() ++--+ +|abs(value)| ++--+ +| 1.5| +| 2.5| +| 3.5| ++--+ """ return _invoke_function_over_columns("abs", col) @@ -1478,7 +1522,8 @@ def product(col: "ColumnOrName") -> Column: @_try_remote_functions def acos(col: "ColumnOrName") -> Column: """ -Computes inverse cosine of the input column. +Mathematical Function: Computes the inverse cosine (also known as arccosine) +of the given column or expression. .. versionadded:: 1.4.0 @@ -1488,23 +1533,54 @@ def acos(col: "ColumnOrName") -> Column: Parameters -- col : :class:`~pyspark.sql.Column` or str -target column to compute on. +The target column or expression to compute the inverse cosine on. Returns --- :class:`~pyspark.sql
(spark) branch master updated: [SPARK-45593][BUILD] Building a runnable distribution from master code running spark-sql raise error
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 44d2c86e71fc [SPARK-45593][BUILD] Building a runnable distribution from master code running spark-sql raise error 44d2c86e71fc is described below commit 44d2c86e71fca7044e6d5d9e9222eecff17c360c Author: yikaifei AuthorDate: Thu Jan 18 11:32:01 2024 +0800 [SPARK-45593][BUILD] Building a runnable distribution from master code running spark-sql raise error ### What changes were proposed in this pull request? Fix a build issue, when building a runnable distribution from master code running spark-sql raise error: ``` Caused by: java.lang.ClassNotFoundException: org.sparkproject.guava.util.concurrent.internal.InternalFutureFailureAccess at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:641) at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:188) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:520) ... 58 more ``` the problem is due to a gauva dependency in spark-connect-common POM that **conflicts** with the shade plugin of the parent pom. - the spark-connect-common contains `connect.guava.version` version of guava, and it is relocation as `${spark.shade.packageName}.guava` not the `${spark.shade.packageName}.connect.guava`; - The spark-network-common also contains guava related classes, it has also been relocation is `${spark.shade.packageName}.guava`, but guava version `${guava.version}`; - As a result, in the presence of different versions of the classpath org.sparkproject.guava.xx; In addition, after investigation, it seems that module spark-connect-common is not related to guava, so we can remove guava dependency from spark-connect-common. ### Why are the changes needed? Building a runnable distribution from master code is not runnable. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? I ran the build command output a runnable distribution package manually for the tests; Build command: ``` ./dev/make-distribution.sh --name ui --pip --tgz -Phive -Phive-thriftserver -Pyarn -Pconnect ``` Test result: https://github.com/apache/spark/assets/51110188/aefbc433-ea5c-4287-8ebd-367806043ac8;> I also checked the `org.sparkproject.guava.cache.LocalCache` from jars dir; Before: ``` ➜ jars grep -lr 'org.sparkproject.guava.cache.LocalCache' ./ .//spark-connect_2.13-4.0.0-SNAPSHOT.jar .//spark-network-common_2.13-4.0.0-SNAPSHOT.jar .//spark-connect-common_2.13-4.0.0-SNAPSHOT.jar ``` Now: ``` ➜ jars grep -lr 'org.sparkproject.guava.cache.LocalCache' ./ .//spark-network-common_2.13-4.0.0-SNAPSHOT.jar ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #43436 from Yikf/SPARK-45593. Authored-by: yikaifei Signed-off-by: yangjie01 --- assembly/pom.xml | 6 ++ connector/connect/client/jvm/pom.xml | 8 +--- connector/connect/common/pom.xml | 34 ++ connector/connect/server/pom.xml | 25 - 4 files changed, 41 insertions(+), 32 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 77ff87c17f52..cd8c3fca9d23 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -149,6 +149,12 @@ org.apache.spark spark-connect_${scala.binary.version} ${project.version} + + + org.apache.spark + spark-connect-common_${scala.binary.version} + + org.apache.spark diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index 8057a33df178..9bedebf523a7 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -51,15 +51,9 @@ ${project.version} - - com.google.guava - guava - ${connect.guava.version} - compile - com.google.protobuf protobuf-java diff --git a/connector/connect/common/pom.xml b/connector/connect/common/pom.xml index a374646f8f29..336d83e04c15 100644 --- a/connector/connect/common/pom.xml +++ b/connector/connect/common/pom.xml @@ -47,6 +47,11 @@ com.google.protobuf protobuf-java + com.google.guava guava @@ -145,6 +150,35 @@ + +org.apache.maven.plugins +maven-sh
(spark) branch master updated: [SPARK-46713][PYTHON][DOCS] Refine docstring of `map_keys/map_values/map_entries`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 02573741605d [SPARK-46713][PYTHON][DOCS] Refine docstring of `map_keys/map_values/map_entries` 02573741605d is described below commit 02573741605dc240de0af6c8d23f983cfd303cc6 Author: yangjie01 AuthorDate: Mon Jan 15 15:15:01 2024 +0800 [SPARK-46713][PYTHON][DOCS] Refine docstring of `map_keys/map_values/map_entries` ### What changes were proposed in this pull request? This pr refine docstring of `map_keys/map_values/map_entries` and add some new examples. ### Why are the changes needed? To improve PySpark documentation ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #44724 from LuciferYang/SPARK-46713. Lead-authored-by: yangjie01 Co-authored-by: YangJie Signed-off-by: yangjie01 --- python/pyspark/sql/functions/builtin.py | 192 +--- 1 file changed, 154 insertions(+), 38 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index a05ce7b04368..f1422d17b071 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -15800,7 +15800,7 @@ def map_contains_key(col: "ColumnOrName", value: Any) -> Column: @_try_remote_functions def map_keys(col: "ColumnOrName") -> Column: """ -Collection function: Returns an unordered array containing the keys of the map. +Map function: Returns an unordered array containing the keys of the map. .. versionadded:: 2.3.0 @@ -15810,23 +15810,61 @@ def map_keys(col: "ColumnOrName") -> Column: Parameters -- col : :class:`~pyspark.sql.Column` or str -name of column or expression +Name of column or expression Returns --- :class:`~pyspark.sql.Column` -keys of the map as an array. +Keys of the map as an array. Examples ->>> from pyspark.sql.functions import map_keys +Example 1: Extracting keys from a simple map + +>>> from pyspark.sql import functions as sf >>> df = spark.sql("SELECT map(1, 'a', 2, 'b') as data") ->>> df.select(map_keys("data").alias("keys")).show() -+--+ -| keys| -+--+ -|[1, 2]| -+--+ +>>> df.select(sf.sort_array(sf.map_keys("data"))).show() +++ +|sort_array(map_keys(data), true)| +++ +| [1, 2]| +++ + +Example 2: Extracting keys from a map with complex keys + +>>> from pyspark.sql import functions as sf +>>> df = spark.sql("SELECT map(array(1, 2), 'a', array(3, 4), 'b') as data") +>>> df.select(sf.sort_array(sf.map_keys("data"))).show() +++ +|sort_array(map_keys(data), true)| +++ +|[[1, 2], [3, 4]]| +++ + +Example 3: Extracting keys from a map with duplicate keys + +>>> from pyspark.sql import functions as sf +>>> originalmapKeyDedupPolicy = spark.conf.get("spark.sql.mapKeyDedupPolicy") +>>> spark.conf.set("spark.sql.mapKeyDedupPolicy", "LAST_WIN") +>>> df = spark.sql("SELECT map(1, 'a', 1, 'b') as data") +>>> df.select(sf.map_keys("data")).show() ++--+ +|map_keys(data)| ++--+ +| [1]| ++--+ +>>> spark.conf.set("spark.sql.mapKeyDedupPolicy", originalmapKeyDedupPolicy) + +Example 4: Extracting keys from an empty map + +>>> from pyspark.sql import functions as sf +>>> df = spark.sql("SELECT map() as data") +>>> df.select(sf.map_keys("data")).show() ++--+ +|map_keys(data)| ++--+ +|[]| ++--+ """ return _invoke_function_over_columns("map_keys", col) @@ -15834,7 +15872,7 @@ def map_keys(col: "ColumnOrName") -> Column: @_try_remote_functions def map_values(col: "ColumnOrName") -> Column: """ -Collection function: Returns an unordered array containing the values of the map. +Map functio
(spark) branch master updated: [SPARK-46586][SQL][TESTS][FOLLOWUP] Replace `appended` with `:+`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9e68a4ca5a2b [SPARK-46586][SQL][TESTS][FOLLOWUP] Replace `appended` with `:+` 9e68a4ca5a2b is described below commit 9e68a4ca5a2b8b496a4c150e7a0f139a286704f7 Author: panbingkun AuthorDate: Fri Jan 12 16:27:49 2024 +0800 [SPARK-46586][SQL][TESTS][FOLLOWUP] Replace `appended` with `:+` ### What changes were proposed in this pull request? The pr is following up https://github.com/apache/spark/pull/44591. ### Why are the changes needed? Fix issues in comments. https://github.com/apache/spark/assets/15246973/deec66a1-1bd1-44bc-9c16-f1fe8bbb14e8;> ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Pass GA. - Manually test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44702 from panbingkun/SPARK-46586_fix. Authored-by: panbingkun Signed-off-by: yangjie01 --- sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 2bd649ea85e5..7952e0f174b7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -840,7 +840,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { Row(ArrayBuffer(100))) val myUdf2 = udf((a: immutable.ArraySeq[Int]) => - immutable.ArraySeq.unsafeWrapArray[Int](a.appended(5).appended(6).toArray)) + immutable.ArraySeq.unsafeWrapArray[Int]((a :+ 5 :+ 6).toArray)) checkAnswer(Seq(Array(1, 2, 3)) .toDF("col") .select(myUdf2(Column("col"))), - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-46635][PYTHON][DOCS] Refine docstring of `from_csv/schema_of_csv/to_csv`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new bda9957bc1d8 [SPARK-46635][PYTHON][DOCS] Refine docstring of `from_csv/schema_of_csv/to_csv` bda9957bc1d8 is described below commit bda9957bc1d896306f88448362b0555d65aef53d Author: yangjie01 AuthorDate: Wed Jan 10 16:47:34 2024 +0800 [SPARK-46635][PYTHON][DOCS] Refine docstring of `from_csv/schema_of_csv/to_csv` ### What changes were proposed in this pull request? This pr refine docstring of `from_csv/schema_of_csv/to_csv` and add some new examples. ### Why are the changes needed? To improve PySpark documentation ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #44639 from LuciferYang/csv-functions. Authored-by: yangjie01 Signed-off-by: yangjie01 --- python/pyspark/sql/functions/builtin.py | 195 +++- 1 file changed, 167 insertions(+), 28 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 711d9b447a60..5b3c1d83d5ac 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -14925,7 +14925,7 @@ def to_xml(col: "ColumnOrName", options: Optional[Dict[str, str]] = None) -> Col @_try_remote_functions def schema_of_csv(csv: Union[Column, str], options: Optional[Dict[str, str]] = None) -> Column: """ -Parses a CSV string and infers its schema in DDL format. +CSV Function: Parses a CSV string and infers its schema in DDL format. .. versionadded:: 3.0.0 @@ -14935,9 +14935,9 @@ def schema_of_csv(csv: Union[Column, str], options: Optional[Dict[str, str]] = N Parameters -- csv : :class:`~pyspark.sql.Column` or str -a CSV string or a foldable string column containing a CSV string. +A CSV string or a foldable string column containing a CSV string. options : dict, optional -options to control parsing. accepts the same options as the CSV datasource. +Options to control parsing. Accepts the same options as the CSV datasource. See `Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_ for the version you use. @@ -14946,15 +14946,53 @@ def schema_of_csv(csv: Union[Column, str], options: Optional[Dict[str, str]] = N Returns --- :class:`~pyspark.sql.Column` -a string representation of a :class:`StructType` parsed from given CSV. +A string representation of a :class:`StructType` parsed from the given CSV. Examples +Example 1: Inferring the schema of a CSV string with different data types + +>>> from pyspark.sql import functions as sf +>>> df = spark.range(1) +>>> df.select(sf.schema_of_csv(sf.lit('1|a|true'), {'sep':'|'})).show(truncate=False) ++---+ +|schema_of_csv(1|a|true)| ++---+ +|STRUCT<_c0: INT, _c1: STRING, _c2: BOOLEAN>| ++---+ + +Example 2: Inferring the schema of a CSV string with missing values + +>>> from pyspark.sql import functions as sf +>>> df = spark.range(1) +>>> df.select(sf.schema_of_csv(sf.lit('1||true'), {'sep':'|'})).show(truncate=False) ++---+ +|schema_of_csv(1||true) | ++---+ +|STRUCT<_c0: INT, _c1: STRING, _c2: BOOLEAN>| ++---+ + +Example 3: Inferring the schema of a CSV string with a different delimiter + +>>> from pyspark.sql import functions as sf >>> df = spark.range(1) ->>> df.select(schema_of_csv(lit('1|a'), {'sep':'|'}).alias("csv")).collect() -[Row(csv='STRUCT<_c0: INT, _c1: STRING>')] ->>> df.select(schema_of_csv('1|a', {'sep':'|'}).alias("csv")).collect() -[Row(csv='STRUCT<_c0: INT, _c1: STRING>')] +>>> df.select(sf.schema_of_csv(sf.lit('1;a;true'), {'sep':';'})).show(truncate=False) ++---+ +|schema_of_csv(1;a;true)| ++---+ +|STRUCT<_c0: INT, _c1: STRING, _c2: BOOLEAN>| ++---+ + +Example 4: Inferring the schema of a CSV string wit
(spark) branch master updated: [SPARK-46621][PYTHON] Address null from Exception.getMessage in Py4J captured exception
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 5a74e5a049aa [SPARK-46621][PYTHON] Address null from Exception.getMessage in Py4J captured exception 5a74e5a049aa is described below commit 5a74e5a049aa52d19e802bea84b4b898d14f3a9b Author: Hyukjin Kwon AuthorDate: Mon Jan 8 19:54:36 2024 +0800 [SPARK-46621][PYTHON] Address null from Exception.getMessage in Py4J captured exception ### What changes were proposed in this pull request? This PR proposes to address null from `Exception.getMessage` in Py4J captured exception. It returns an empty string. ### Why are the changes needed? So whitelisted exceptions with an empty arguments are also covered. ### Does this PR introduce _any_ user-facing change? Virtually no. It only happens when whitelisted exceptions are created without any argument so `null` is located in `message`. ### How was this patch tested? Manually. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44623 from HyukjinKwon/SPARK-46621. Authored-by: Hyukjin Kwon Signed-off-by: yangjie01 --- python/pyspark/errors/exceptions/captured.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/pyspark/errors/exceptions/captured.py b/python/pyspark/errors/exceptions/captured.py index 687bdec14154..e78a7c7bce8a 100644 --- a/python/pyspark/errors/exceptions/captured.py +++ b/python/pyspark/errors/exceptions/captured.py @@ -59,6 +59,8 @@ class CapturedException(PySparkException): ) self._desc = desc if desc is not None else cast(Py4JJavaError, origin).getMessage() +if self._desc is None: +self._desc = "" assert SparkContext._jvm is not None self._stackTrace = ( stackTrace - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-46611][CORE] Remove ThreadLocal by replace SimpleDateFormat with DateTimeFormatter
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 443a95792073 [SPARK-46611][CORE] Remove ThreadLocal by replace SimpleDateFormat with DateTimeFormatter 443a95792073 is described below commit 443a957920737143834888726f067bc54acb97f2 Author: beliefer AuthorDate: Mon Jan 8 13:42:59 2024 +0800 [SPARK-46611][CORE] Remove ThreadLocal by replace SimpleDateFormat with DateTimeFormatter ### What changes were proposed in this pull request? This PR propose to remove `ThreadLocal` by replace `SimpleDateFormat` with `DateTimeFormatter`. ### Why are the changes needed? `SimpleDateFormat` is not thread safe, so we wrap it with `ThreadLocal`. `DateTimeFormatter` is thread safe, we can use it instead. According to the javadoc of `SimpleDateFormat`, it recommended to use `DateTimeFormatter` too. ![1](https://github.com/apache/spark/assets/8486025/97b16bbb-e5b7-4b3f-9bc8-0b0b8c907542) In addition, `DateTimeFormatter` have better performance than `SimpleDateFormat` too. ### Does this PR introduce _any_ user-facing change? 'No'. ### How was this patch tested? GA tests. ### Was this patch authored or co-authored using generative AI tooling? 'No'. Closes #44613 from beliefer/sdf-to-dtf. Authored-by: beliefer Signed-off-by: yangjie01 --- .../main/scala/org/apache/spark/ui/UIUtils.scala | 47 ++ .../apache/spark/sql/streaming/ui/UIUtils.scala| 18 - 2 files changed, 30 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 4cbc45e03350..74bb78a22765 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -21,7 +21,8 @@ import java.{util => ju} import java.lang.{Long => JLong} import java.net.URLDecoder import java.nio.charset.StandardCharsets.UTF_8 -import java.text.SimpleDateFormat +import java.time.{Instant, ZoneId} +import java.time.format.DateTimeFormatter import java.util.{Date, Locale, TimeZone} import javax.servlet.http.HttpServletRequest import javax.ws.rs.core.{MediaType, MultivaluedMap, Response} @@ -42,15 +43,14 @@ private[spark] object UIUtils extends Logging { val TABLE_CLASS_STRIPED = TABLE_CLASS_NOT_STRIPED + " table-striped" val TABLE_CLASS_STRIPED_SORTABLE = TABLE_CLASS_STRIPED + " sortable" - // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. - private val dateFormat = new ThreadLocal[SimpleDateFormat]() { -override def initialValue(): SimpleDateFormat = - new SimpleDateFormat("/MM/dd HH:mm:ss", Locale.US) - } + private val dateTimeFormatter = DateTimeFormatter +.ofPattern("/MM/dd HH:mm:ss", Locale.US) +.withZone(ZoneId.systemDefault()) - def formatDate(date: Date): String = dateFormat.get.format(date) + def formatDate(date: Date): String = dateTimeFormatter.format(date.toInstant) - def formatDate(timestamp: Long): String = dateFormat.get.format(new Date(timestamp)) + def formatDate(timestamp: Long): String = +dateTimeFormatter.format(Instant.ofEpochMilli(timestamp)) def formatDuration(milliseconds: Long): String = { if (milliseconds < 100) { @@ -124,16 +124,13 @@ private[spark] object UIUtils extends Logging { } } - // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. - private val batchTimeFormat = new ThreadLocal[SimpleDateFormat]() { -override def initialValue(): SimpleDateFormat = - new SimpleDateFormat("/MM/dd HH:mm:ss", Locale.US) - } + private val batchTimeFormat = DateTimeFormatter +.ofPattern("/MM/dd HH:mm:ss", Locale.US) +.withZone(ZoneId.systemDefault()) - private val batchTimeFormatWithMilliseconds = new ThreadLocal[SimpleDateFormat]() { -override def initialValue(): SimpleDateFormat = - new SimpleDateFormat("/MM/dd HH:mm:ss.SSS", Locale.US) - } + private val batchTimeFormatWithMilliseconds = DateTimeFormatter +.ofPattern("/MM/dd HH:mm:ss.SSS", Locale.US) +.withZone(ZoneId.systemDefault()) /** * If `batchInterval` is less than 1 second, format `batchTime` with milliseconds. Otherwise, @@ -150,19 +147,19 @@ private[spark] object UIUtils extends Logging { batchInterval: Long, showMMSS: Boolean = true, timezone: TimeZone = null): String = { -val oldTimezones = - (batchTimeFormat.get.getTimeZone, batchTimeFormatWithMilliseconds.get.getTimeZone) +val oldTimezones = (batchTimeFormat.getZone, batchTimeFormatWithMilliseco
(spark) branch master updated: [SPARK-46595][PYTHON][DOCS] Refine docstring of `map_from_arrays/map_from_entries/map_concat`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new b96e13d09361 [SPARK-46595][PYTHON][DOCS] Refine docstring of `map_from_arrays/map_from_entries/map_concat` b96e13d09361 is described below commit b96e13d09361b7aaf084333ff07ed5ad610addfe Author: yangjie01 AuthorDate: Fri Jan 5 13:55:07 2024 +0800 [SPARK-46595][PYTHON][DOCS] Refine docstring of `map_from_arrays/map_from_entries/map_concat` ### What changes were proposed in this pull request? This pr refine docstring of `map_from_arrays/map_from_entries/map_concat` and add some new examples. ### Why are the changes needed? To improve PySpark documentation ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #44595 from LuciferYang/SPARK-46595. Authored-by: yangjie01 Signed-off-by: yangjie01 --- python/pyspark/sql/functions/builtin.py | 199 ++-- 1 file changed, 165 insertions(+), 34 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 50d3b6cc01ce..bd151050e77f 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -11989,8 +11989,9 @@ def create_map( @_try_remote_functions def map_from_arrays(col1: "ColumnOrName", col2: "ColumnOrName") -> Column: -"""Creates a new map from two arrays. - +""" +Map function: Creates a new map from two arrays. This function takes two arrays of +keys and values respectively, and returns a new map column. .. versionadded:: 2.4.0 .. versionchanged:: 3.4.0 @@ -11999,30 +12000,59 @@ def map_from_arrays(col1: "ColumnOrName", col2: "ColumnOrName") -> Column: Parameters -- col1 : :class:`~pyspark.sql.Column` or str -name of column containing a set of keys. All elements should not be null +Name of column containing a set of keys. All elements should not be null. col2 : :class:`~pyspark.sql.Column` or str -name of column containing a set of values +Name of column containing a set of values. Returns --- :class:`~pyspark.sql.Column` -a column of map type. +A column of map type. + +Notes +- +The input arrays for keys and values must have the same length and all elements +in keys should not be null. If these conditions are not met, an exception will be thrown. Examples +Example 1: Basic usage of map_from_arrays + +>>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([([2, 5], ['a', 'b'])], ['k', 'v']) ->>> df = df.select(map_from_arrays(df.k, df.v).alias("col")) ->>> df.show() -++ -| col| -++ -|{2 -> a, 5 -> b}| -++ ->>> df.printSchema() -root - |-- col: map (nullable = true) - ||-- key: long - ||-- value: string (valueContainsNull = true) +>>> df.select(sf.map_from_arrays(df.k, df.v)).show() ++-+ +|map_from_arrays(k, v)| ++-+ +| {2 -> a, 5 -> b}| ++-+ + +Example 2: map_from_arrays with null values + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([([1, 2], ['a', None])], ['k', 'v']) +>>> df.select(sf.map_from_arrays(df.k, df.v)).show() ++-+ +|map_from_arrays(k, v)| ++-+ +| {1 -> a, 2 -> NULL}| ++-+ + +Example 3: map_from_arrays with empty arrays + +>>> from pyspark.sql import functions as sf +>>> from pyspark.sql.types import ArrayType, StringType, IntegerType, StructType, StructField +>>> schema = StructType([ +... StructField('k', ArrayType(IntegerType())), +... StructField('v', ArrayType(StringType())) +... ]) +>>> df = spark.createDataFrame([([], [])], schema=schema) +>>> df.select(sf.map_from_arrays(df.k, df.v)).show() ++-+ +|map_from_arrays(k, v)| ++-+ +| {}| ++-+ """ return _invoke_function_over_columns("map_from_arrays", col1, col2) @@ -15581,8 +15611,9 @@ def map_entries(col: "ColumnOrName") -> Column: @_try_remote_f
(spark) branch master updated: [SPARK-46585][CORE] Directly constructed `metricPeaks` as an `immutable.ArraySeq` instead of use `mutable.ArraySeq.toSeq` in `Executor`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c8137960a0ba [SPARK-46585][CORE] Directly constructed `metricPeaks` as an `immutable.ArraySeq` instead of use `mutable.ArraySeq.toSeq` in `Executor` c8137960a0ba is described below commit c8137960a0ba725d1633795a057c68f2bbef414b Author: yangjie01 AuthorDate: Thu Jan 4 16:06:47 2024 +0800 [SPARK-46585][CORE] Directly constructed `metricPeaks` as an `immutable.ArraySeq` instead of use `mutable.ArraySeq.toSeq` in `Executor` ### What changes were proposed in this pull request? The following code exists in branch-3.5: https://github.com/apache/spark/blob/fb90ade2c7390077d2755fc43b73e63f5cf44f21/core/src/main/scala/org/apache/spark/executor/Executor.scala#L744-L749 From the code comments, when using Scala 2.12, `metricsPoller.getTaskMetricPeaks(taskId)` is wrapped as `WrappedArray` to avoid collection copying, and the subsequent `metricPeaks.toSeq` is a redundant collection conversion for Scala 2.12. However, for Scala 2.13, if `metricsPoller.getTaskMetricPeaks(taskId)` is still wrapped as `WrappedArray/mutable.ArraySeq`, it is impossible to avoid collection conversion, because the subsequent `metricPeaks.toSeq` will trigger a collection copy when using Scala 2.13. So this pr changes the process to directly wrap `metricPeaks` as `immutable.ArraySeq` to ensure the same effect as when using Scala 2.12, which is also more in line with the original comment description. ### Why are the changes needed? Avoid unnecessary collection copying when using Scala 2.13. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #44586 from LuciferYang/executor-metricPeaks. Lead-authored-by: yangjie01 Co-authored-by: YangJie Signed-off-by: yangjie01 --- .../scala/org/apache/spark/executor/Executor.scala | 21 +++-- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index f2a65aab1ba4..12471915cd97 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -29,7 +29,7 @@ import java.util.concurrent.locks.ReentrantLock import javax.annotation.concurrent.GuardedBy import javax.ws.rs.core.UriBuilder -import scala.collection.{immutable, mutable} +import scala.collection.immutable import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.duration._ import scala.jdk.CollectionConverters._ @@ -54,6 +54,7 @@ import org.apache.spark.shuffle.{FetchFailedException, ShuffleBlockPusher} import org.apache.spark.status.api.v1.ThreadStackTrace import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} import org.apache.spark.util._ +import org.apache.spark.util.ArrayImplicits._ private[spark] class IsolatedSessionState( val sessionUUID: String, @@ -749,10 +750,10 @@ private[spark] class Executor( logInfo(s"Executor killed $taskName, reason: ${t.reason}") val (accums, accUpdates) = collectAccumulatorsAndResetStatusOnFailure(taskStartTimeNs) - // Here and below, put task metric peaks in a ArraySeq to expose them as a Seq - // without requiring a copy. - val metricPeaks = mutable.ArraySeq.make(metricsPoller.getTaskMetricPeaks(taskId)) - val reason = TaskKilled(t.reason, accUpdates, accums, metricPeaks.toSeq) + // Here and below, put task metric peaks in an immutable.ArraySeq to expose them as an + // immutable.Seq without requiring a copy. + val metricPeaks = metricsPoller.getTaskMetricPeaks(taskId).toImmutableArraySeq + val reason = TaskKilled(t.reason, accUpdates, accums, metricPeaks) plugins.foreach(_.onTaskFailed(reason)) execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(reason)) @@ -762,8 +763,8 @@ private[spark] class Executor( logInfo(s"Executor interrupted and killed $taskName, reason: $killReason") val (accums, accUpdates) = collectAccumulatorsAndResetStatusOnFailure(taskStartTimeNs) - val metricPeaks = mutable.ArraySeq.make(metricsPoller.getTaskMetricPeaks(taskId)) - val reason = TaskKilled(killReason, accUpdates, accums, metricPeaks.toSeq) + val metricPeaks = metricsPoller.getTaskMetricPeaks(taskId).toImmutableArraySeq + val reason = TaskKilled(killReason, accUpdates, accums, metricPeaks) plugins.fo
(spark) branch master updated: [SPARK-46559][MLLIB] Wrap the `export` in the package name with backticks
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e690e96860a6 [SPARK-46559][MLLIB] Wrap the `export` in the package name with backticks e690e96860a6 is described below commit e690e96860a6ac25fe2a35e66d14adee9f96d075 Author: yangjie01 AuthorDate: Tue Jan 2 20:23:35 2024 +0800 [SPARK-46559][MLLIB] Wrap the `export` in the package name with backticks ### What changes were proposed in this pull request? This pr wrap the `export` in the package name with backticks due to `export` will become keywords in Scala 3. ### Why are the changes needed? `export` will become keywords in Scala 3, Scala 2.13 compiler will check for relevant cases in the code, but it does not check for cases in the package name. However, if we write a Scala file as follows and compile it with Scala 3, ```scala package org.apache.spark.mllib.pmml.export private class ExportABC() {} ``` it will throw an error: ```scala bin/scalac test.scala -explain -- [E040] Syntax Error: test.scala:1:36 1 |package org.apache.spark.mllib.pmml.export |^^ |an identifier expected, but 'export' found |- | Explanation (enabled by `-explain`) |- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - | | If you want to use 'export' as identifier, you may put it in backticks: `export`. - 1 error found ``` We can workaround by wrapping `export` with backticks. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #44555 from LuciferYang/export-backtick. Authored-by: yangjie01 Signed-off-by: yangjie01 --- .../spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala | 2 +- .../spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala | 2 +- .../org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala | 2 +- .../main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala | 2 +- .../org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala | 2 +- .../mllib/pmml/export/BinaryClassificationPMMLModelExportSuite.scala| 2 +- .../spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala | 2 +- .../org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala | 2 +- .../apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala| 2 +- 9 files changed, 9 insertions(+), 9 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala index 27935c6f5291..7aa9051b5c58 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.pmml.export +package org.apache.spark.mllib.pmml.`export` import scala.{Array => SArray} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala index 723224de168e..bc6a4a40c441 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.pmml.export +package org.apache.spark.mllib.pmml.`export` import scala.{Array => SArray} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala index d86410c1ae37..674c8cea9d42 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.pmml.export +package org.apache.spark.mllib.pmml.`export` import scala.{Array => SArray} diff --git a/
(spark) branch master updated: [SPARK-46554][BUILD] Upgrade slf4j to 2.0.10
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e6082119fedc [SPARK-46554][BUILD] Upgrade slf4j to 2.0.10 e6082119fedc is described below commit e6082119fedce95928397a6ddb56712fe8c8abbd Author: panbingkun AuthorDate: Tue Jan 2 10:14:00 2024 +0800 [SPARK-46554][BUILD] Upgrade slf4j to 2.0.10 ### What changes were proposed in this pull request? The pr aims to upgrade `slf4j` from 2.0.9 to 2.0.10. ### Why are the changes needed? The release notes as follows: - https://www.slf4j.org/news.html#2.0.10 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44544 from panbingkun/slf4j_api_2010. Authored-by: panbingkun Signed-off-by: yangjie01 --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 6 +++--- pom.xml | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index bebc63bae72b..0c90a96f329b 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -121,7 +121,7 @@ javassist/3.29.2-GA//javassist-3.29.2-GA.jar javax.jdo/3.2.0-m3//javax.jdo-3.2.0-m3.jar javolution/5.5.1//javolution-5.5.1.jar jaxb-runtime/2.3.2//jaxb-runtime-2.3.2.jar -jcl-over-slf4j/2.0.9//jcl-over-slf4j-2.0.9.jar +jcl-over-slf4j/2.0.10//jcl-over-slf4j-2.0.10.jar jdo-api/3.0.1//jdo-api-3.0.1.jar jdom2/2.0.6//jdom2-2.0.6.jar jersey-client/2.41//jersey-client-2.41.jar @@ -146,7 +146,7 @@ json4s-jackson_2.13/3.7.0-M11//json4s-jackson_2.13-3.7.0-M11.jar json4s-scalap_2.13/3.7.0-M11//json4s-scalap_2.13-3.7.0-M11.jar jsr305/3.0.0//jsr305-3.0.0.jar jta/1.1//jta-1.1.jar -jul-to-slf4j/2.0.9//jul-to-slf4j-2.0.9.jar +jul-to-slf4j/2.0.10//jul-to-slf4j-2.0.10.jar kryo-shaded/4.0.2//kryo-shaded-4.0.2.jar kubernetes-client-api/6.9.1//kubernetes-client-api-6.9.1.jar kubernetes-client/6.9.1//kubernetes-client-6.9.1.jar @@ -244,7 +244,7 @@ scala-parallel-collections_2.13/1.0.4//scala-parallel-collections_2.13-1.0.4.jar scala-parser-combinators_2.13/2.3.0//scala-parser-combinators_2.13-2.3.0.jar scala-reflect/2.13.12//scala-reflect-2.13.12.jar scala-xml_2.13/2.2.0//scala-xml_2.13-2.2.0.jar -slf4j-api/2.0.9//slf4j-api-2.0.9.jar +slf4j-api/2.0.10//slf4j-api-2.0.10.jar snakeyaml-engine/2.7//snakeyaml-engine-2.7.jar snakeyaml/2.2//snakeyaml-2.2.jar snappy-java/1.1.10.5//snappy-java-1.1.10.5.jar diff --git a/pom.xml b/pom.xml index 1938708fee95..95a70fbf91cb 100644 --- a/pom.xml +++ b/pom.xml @@ -119,7 +119,7 @@ 3.1.0 spark 9.6 -2.0.9 +2.0.10 2.22.0 3.3.6 - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-46551][PYTHON][DOCS] Refine docstring of `flatten/sequence/shuffle`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9bf5e82a5577 [SPARK-46551][PYTHON][DOCS] Refine docstring of `flatten/sequence/shuffle` 9bf5e82a5577 is described below commit 9bf5e82a5577313e48b2ef983d1bc4285682960a Author: yangjie01 AuthorDate: Tue Jan 2 10:12:46 2024 +0800 [SPARK-46551][PYTHON][DOCS] Refine docstring of `flatten/sequence/shuffle` ### What changes were proposed in this pull request? This pr refine docstring of `flatten/sequence/shuffle` and add some new examples. ### Why are the changes needed? To improve PySpark documentation ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #44548 from LuciferYang/SPARK-46551. Authored-by: yangjie01 Signed-off-by: yangjie01 --- python/pyspark/sql/functions/builtin.py | 171 +--- 1 file changed, 134 insertions(+), 37 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 458bb8c8feaf..7f5e90739507 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -15224,32 +15224,73 @@ def array_sort( @_try_remote_functions def shuffle(col: "ColumnOrName") -> Column: """ -Collection function: Generates a random permutation of the given array. +Array function: Generates a random permutation of the given array. .. versionadded:: 2.4.0 .. versionchanged:: 3.4.0 Supports Spark Connect. -Notes -- -The function is non-deterministic. - Parameters -- col : :class:`~pyspark.sql.Column` or str -name of column or expression +The name of the column or expression to be shuffled. Returns --- :class:`~pyspark.sql.Column` -an array of elements in random order. +A new column that contains an array of elements in random order. + +Notes +- +The `shuffle` function is non-deterministic, meaning the order of the output array +can be different for each execution. Examples ->>> df = spark.createDataFrame([([1, 20, 3, 5],), ([1, 20, None, 3],)], ['data']) ->>> df.select(shuffle(df.data).alias('s')).collect() # doctest: +SKIP -[Row(s=[3, 1, 5, 20]), Row(s=[20, None, 3, 1])] +Example 1: Shuffling a simple array + +>>> import pyspark.sql.functions as sf +>>> df = spark.createDataFrame([([1, 20, 3, 5],)], ['data']) +>>> df.select(sf.shuffle(df.data)).show() # doctest: +SKIP ++-+ +|shuffle(data)| ++-+ +|[1, 3, 20, 5]| ++-+ + +Example 2: Shuffling an array with null values + +>>> import pyspark.sql.functions as sf +>>> df = spark.createDataFrame([([1, 20, None, 3],)], ['data']) +>>> df.select(sf.shuffle(df.data)).show() # doctest: +SKIP +++ +| shuffle(data)| +++ +|[20, 3, NULL, 1]| +++ + +Example 3: Shuffling an array with duplicate values + +>>> import pyspark.sql.functions as sf +>>> df = spark.createDataFrame([([1, 2, 2, 3, 3, 3],)], ['data']) +>>> df.select(sf.shuffle(df.data)).show() # doctest: +SKIP ++--+ +| shuffle(data)| ++--+ +|[3, 2, 1, 3, 2, 3]| ++--+ + +Example 4: Shuffling an array with different types of elements + +>>> import pyspark.sql.functions as sf +>>> df = spark.createDataFrame([(['a', 'b', 'c', 1, 2, 3],)], ['data']) +>>> df.select(sf.shuffle(df.data)).show() # doctest: +SKIP ++--+ +| shuffle(data)| ++--+ +|[1, c, 2, a, b, 3]| ++--+ """ return _invoke_function_over_columns("shuffle", col) @@ -15289,7 +15330,7 @@ def reverse(col: "ColumnOrName") -> Column: @_try_remote_functions def flatten(col: "ColumnOrName") -> Column: """ -Collection function: creates a single array from an array of arrays. +Array function: creates a single array from an array of arrays. If a structure of nested arrays is deeper than two levels, only one level of nesting is removed. @@ -15301,29 +15342,57 @@ def flatten(col: "ColumnOrName") -> Column: Parameters -- col : :class:`~pyspark.sql.Column` or str -name of column or e
(spark) branch master updated: [SPARK-46548][PYTHON][DOCS] Refine docstring of `get/array_zip/sort_array`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9a6b27ecb15e [SPARK-46548][PYTHON][DOCS] Refine docstring of `get/array_zip/sort_array` 9a6b27ecb15e is described below commit 9a6b27ecb15e656a34fe3b8744f23855548c830d Author: yangjie01 AuthorDate: Sun Dec 31 15:47:22 2023 +0800 [SPARK-46548][PYTHON][DOCS] Refine docstring of `get/array_zip/sort_array` ### What changes were proposed in this pull request? This pr refine docstring of `get/array_zip/sort_array` and add some new examples. ### Why are the changes needed? To improve PySpark documentation ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #44545 from LuciferYang/array-functions-sort. Authored-by: yangjie01 Signed-off-by: yangjie01 --- python/pyspark/sql/functions/builtin.py | 198 1 file changed, 150 insertions(+), 48 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 0ff1ee2a7394..458bb8c8feaf 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -12810,7 +12810,7 @@ def try_element_at(col: "ColumnOrName", extraction: "ColumnOrName") -> Column: @_try_remote_functions def get(col: "ColumnOrName", index: Union["ColumnOrName", int]) -> Column: """ -Collection function: Returns element of array at given (0-based) index. +Array function: Returns the element of an array at the given (0-based) index. If the index points outside of the array boundaries, then this function returns NULL. @@ -12819,18 +12819,18 @@ def get(col: "ColumnOrName", index: Union["ColumnOrName", int]) -> Column: Parameters -- col : :class:`~pyspark.sql.Column` or str -name of column containing array +Name of the column containing the array. index : :class:`~pyspark.sql.Column` or str or int -index to check for in array +Index to check for in the array. Returns --- :class:`~pyspark.sql.Column` -value at given position. +Value at the given position. Notes - -The position is not 1 based, but 0 based index. +The position is not 1-based, but 0-based index. Supports Spark Connect. See Also @@ -12839,41 +12839,61 @@ def get(col: "ColumnOrName", index: Union["ColumnOrName", int]) -> Column: Examples ->>> df = spark.createDataFrame([(["a", "b", "c"], 1)], ['data', 'index']) ->>> df.select(get(df.data, 1)).show() +Example 1: Getting an element at a fixed position + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([(["a", "b", "c"],)], ['data']) +>>> df.select(sf.get(df.data, 1)).show() ++ |get(data, 1)| ++ | b| ++ ->>> df.select(get(df.data, -1)).show() -+-+ -|get(data, -1)| -+-+ -| NULL| -+-+ +Example 2: Getting an element at a position outside the array boundaries ->>> df.select(get(df.data, 3)).show() +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([(["a", "b", "c"],)], ['data']) +>>> df.select(sf.get(df.data, 3)).show() ++ |get(data, 3)| ++ |NULL| ++ ->>> df.select(get(df.data, "index")).show() +Example 3: Getting an element at a position specified by another column + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([(["a", "b", "c"], 2)], ['data', 'index']) +>>> df.select(sf.get(df.data, df.index)).show() ++ |get(data, index)| ++ -| b| +| c| ++ ->>> df.select(get(df.data, col("index") - 1)).show() + +Example 4: Getting an element at a position calculated from another column + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([(["a", "b", "c"], 2)], ['data', 'index']) +>>> df.select(sf.
(spark) branch master updated: [SPARK-46533][PYTHON][DOCS] Refine docstring of `array_min/array_max/array_size/array_repeat`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c7c43cf5b1e1 [SPARK-46533][PYTHON][DOCS] Refine docstring of `array_min/array_max/array_size/array_repeat` c7c43cf5b1e1 is described below commit c7c43cf5b1e162bece7c8975a05d62b5b0fc7e76 Author: yangjie01 AuthorDate: Fri Dec 29 15:02:26 2023 +0800 [SPARK-46533][PYTHON][DOCS] Refine docstring of `array_min/array_max/array_size/array_repeat` ### What changes were proposed in this pull request? This pr refine docstring of `array_min/array_max/array_size/array_repeat` and add some new examples. ### Why are the changes needed? To improve PySpark documentation ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #44522 from LuciferYang/SPARK-46533. Authored-by: yangjie01 Signed-off-by: yangjie01 --- python/pyspark/sql/functions/builtin.py | 256 +--- 1 file changed, 235 insertions(+), 21 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 3b579f20333e..b86fb4692012 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -14769,7 +14769,7 @@ def size(col: "ColumnOrName") -> Column: @_try_remote_functions def array_min(col: "ColumnOrName") -> Column: """ -Collection function: returns the minimum value of the array. +Array function: returns the minimum value of the array. .. versionadded:: 2.4.0 @@ -14779,18 +14779,74 @@ def array_min(col: "ColumnOrName") -> Column: Parameters -- col : :class:`~pyspark.sql.Column` or str -name of column or expression +The name of the column or an expression that represents the array. Returns --- :class:`~pyspark.sql.Column` -minimum value of array. +A new column that contains the minimum value of each array. Examples +Example 1: Basic usage with integer array + +>>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([([2, 1, 3],), ([None, 10, -1],)], ['data']) ->>> df.select(array_min(df.data).alias('min')).collect() -[Row(min=1), Row(min=-1)] +>>> df.select(sf.array_min(df.data)).show() ++---+ +|array_min(data)| ++---+ +| 1| +| -1| ++---+ + +Example 2: Usage with string array + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([(['apple', 'banana', 'cherry'],)], ['data']) +>>> df.select(sf.array_min(df.data)).show() ++---+ +|array_min(data)| ++---+ +| apple| ++---+ + +Example 3: Usage with mixed type array + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([(['apple', 1, 'cherry'],)], ['data']) +>>> df.select(sf.array_min(df.data)).show() ++---+ +|array_min(data)| ++---+ +| 1| ++---+ + +Example 4: Usage with array of arrays + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([([[2, 1], [3, 4]],)], ['data']) +>>> df.select(sf.array_min(df.data)).show() ++---+ +|array_min(data)| ++---+ +| [2, 1]| ++---+ + +Example 5: Usage with empty array + +>>> from pyspark.sql import functions as sf +>>> from pyspark.sql.types import ArrayType, IntegerType, StructType, StructField +>>> schema = StructType([ +... StructField("data", ArrayType(IntegerType()), True) +... ]) +>>> df = spark.createDataFrame([([],)], schema=schema) +>>> df.select(sf.array_min(df.data)).show() ++---+ +|array_min(data)| ++---+ +| NULL| ++---+ """ return _invoke_function_over_columns("array_min", col) @@ -14798,7 +14854,7 @@ def array_min(col: "ColumnOrName") -> Column: @_try_remote_functions def array_max(col: "ColumnOrName") -> Column: """ -Collection function: returns the maximum value of the array. +Array function: returns the maximum value of the array. .. versionadded:: 2.4.0 @@ -14808,18 +14864,74 @@ def arr
(spark) branch master updated (8ea09de807f9 -> fcf5d571e9ac)
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 8ea09de807f9 [SPARK-46498][CORE] Remove `shuffleServiceEnabled` from `o.a.spark.util.Utils#getConfiguredLocalDirs` add fcf5d571e9ac [SPARK-46506][PYTHON][DOCS] Refine docstring of `array_intersect/array_union/array_except` No new revisions were added by this update. Summary of changes: python/pyspark/sql/functions/builtin.py | 222 +--- 1 file changed, 201 insertions(+), 21 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-46479][CORE] Change `Utils.isJavaVersionAtLeast21` to use the utility method from `commons-lang3`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 0c29593436d5 [SPARK-46479][CORE] Change `Utils.isJavaVersionAtLeast21` to use the utility method from `commons-lang3` 0c29593436d5 is described below commit 0c29593436d542f816d95e3575296d0588895c7c Author: yangjie01 AuthorDate: Sat Dec 23 18:08:22 2023 +0800 [SPARK-46479][CORE] Change `Utils.isJavaVersionAtLeast21` to use the utility method from `commons-lang3` ### What changes were proposed in this pull request? Apache commons-lang3 added support for checking `JAVA_21` after version 3.13.0, so this pr change `Utils.isJavaVersionAtLeast21` to use the utility method from commons-lang3. ### Why are the changes needed? Use commons-lang3 utility method. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #4 from LuciferYang/SPARK-46479. Authored-by: yangjie01 Signed-off-by: yangjie01 --- core/src/main/scala/org/apache/spark/util/Utils.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index a074bd53d26d..09f6c5344546 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -50,7 +50,7 @@ import com.google.common.io.{ByteStreams, Files => GFiles} import com.google.common.net.InetAddresses import org.apache.commons.codec.binary.Hex import org.apache.commons.io.IOUtils -import org.apache.commons.lang3.SystemUtils +import org.apache.commons.lang3.{JavaVersion, SystemUtils} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.apache.hadoop.io.compress.{CompressionCodecFactory, SplittableCompressionCodec} @@ -1774,8 +1774,7 @@ private[spark] object Utils /** * Whether the underlying Java version is at least 21. */ - val isJavaVersionAtLeast21 = -System.getProperty("java.version").split("[+.\\-]+", 3)(0).toInt >= 21 + val isJavaVersionAtLeast21 = SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_21) /** * Whether the underlying operating system is Mac OS X and processor is Apple Silicon. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-46483][INFRA] Exclude `apache-rat.jar` from `ScalaUnidoc/unidoc/unidocAllClasspaths`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 0ad2bf6b8ec2 [SPARK-46483][INFRA] Exclude `apache-rat.jar` from `ScalaUnidoc/unidoc/unidocAllClasspaths` 0ad2bf6b8ec2 is described below commit 0ad2bf6b8ec2f965c87f98d6ef01ef294d7b28e9 Author: yangjie01 AuthorDate: Sat Dec 23 13:47:07 2023 +0800 [SPARK-46483][INFRA] Exclude `apache-rat.jar` from `ScalaUnidoc/unidoc/unidocAllClasspaths` ### What changes were proposed in this pull request? This pr aims to exclude `lib/apache-rat-*.jar` from `ScalaUnidoc / unidoc / unidocAllClasspaths` to avoid accidental contamination of the classpath when executing `sbt unidoc`. ### Why are the changes needed? Exclude unexpected dependencies from `ScalaUnidoc / unidoc / unidocAllClasspaths`: `lib/apache-rat-*.jar` is downloaded by `check-license` for license checking. It is not a direct dependency of Apache Spark, but it is accidentally added to `ScalaUnidoc / unidoc / unidocAllClasspaths` when the `sbt unidoc` command is executed. And the following contents have been shaded in `lib/apache-rat-*.jar`, but none of them have been relocated: ``` org.apache.commons:commons-collections4:jar:4.4 org.apache.commons:commons-lang3:jar:3.12.0 commons-io:commons-io:jar:2.11.0 org.apache.commons:commons-compress:jar:1.25.0 commons-cli:commons-cli:jar:1.5.0 ``` On the other hand, the aforementioned fact could lead to a build failure when executing `sbt unidoc` if Apache Spark uses a higher version of the related dependency API in the code. For instance, https://github.com/apache/spark/pull/4 encountered the following error after using `JavaVersion.JAVA_21`, an enumeration value that is only supported in versions higher than commons-lang3 3.13.0: - https://github.com/LuciferYang/spark/actions/runs/7287625573/job/19859603486 ``` [error] /__w/spark/spark/core/src/main/scala/org/apache/spark/util/Utils.scala:1777:77: value JAVA_21 is not a member of object org.apache.commons.lang3.JavaVersion [error] did you mean JAVA_11? or perhaps JAVA_10, JAVA_12, or JAVA_13? or...? [error] val isJavaVersionAtLeast21 = SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_21) [error] ^ ``` Additionally, since `lib/apache-rat-*.jar` is not a direct dependency of Apache Spark, it cannot be excluded through the configuration of `ExcludedDependencies`. Nor can the other dependencies shaded by `lib/apache-rat-*.jar` be version-pin through the configuration of `DependencyOverrides`. So this pr adopts the approach of excluding it from `ScalaUnidoc / unidoc / unidocAllClasspaths` to solve the problem. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #44453 from LuciferYang/SPARK-46483. Lead-authored-by: yangjie01 Co-authored-by: YangJie Signed-off-by: yangjie01 --- project/SparkBuild.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index bcfac15ffdde..75d3a2cf18df 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1373,6 +1373,7 @@ object Unidoc { classpaths .map(_.filterNot(_.data.getCanonicalPath.matches(""".*kafka-clients-0\.10.*"""))) .map(_.filterNot(_.data.getCanonicalPath.matches(""".*kafka_2\..*-0\.10.*"""))) + .map(_.filterNot(_.data.getCanonicalPath.contains("apache-rat"))) } val unidocSourceBase = settingKey[String]("Base URL of source links in Scaladoc.") - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-46472][PYTHON][DOCS] Refine docstring of `array_prepend/array_append/array_insert`
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 6b931530d75 [SPARK-46472][PYTHON][DOCS] Refine docstring of `array_prepend/array_append/array_insert` 6b931530d75 is described below commit 6b931530d75cb4f00236f9c6283de8ef450963ad Author: yangjie01 AuthorDate: Fri Dec 22 11:01:05 2023 +0800 [SPARK-46472][PYTHON][DOCS] Refine docstring of `array_prepend/array_append/array_insert` ### What changes were proposed in this pull request? This pr refine docstring of `array_prepend/array_append/array_insert` and add some new examples. ### Why are the changes needed? To improve PySpark documentation ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #44436 from LuciferYang/SPARK-46472. Authored-by: yangjie01 Signed-off-by: yangjie01 --- python/pyspark/sql/functions/builtin.py | 217 1 file changed, 191 insertions(+), 26 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 54a91792404..571572df30a 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -12875,9 +12875,8 @@ def get(col: "ColumnOrName", index: Union["ColumnOrName", int]) -> Column: @_try_remote_functions def array_prepend(col: "ColumnOrName", value: Any) -> Column: """ -Collection function: Returns an array containing element as -well as all elements from array. The new element is positioned -at the beginning of the array. +Array function: Returns an array containing the given element as +the first element and the rest of the elements from the original array. .. versionadded:: 3.5.0 @@ -12891,13 +12890,72 @@ def array_prepend(col: "ColumnOrName", value: Any) -> Column: Returns --- :class:`~pyspark.sql.Column` -an array excluding given value. +an array with the given value prepended. Examples ->>> df = spark.createDataFrame([([2, 3, 4],), ([],)], ['data']) ->>> df.select(array_prepend(df.data, 1)).collect() -[Row(array_prepend(data, 1)=[1, 2, 3, 4]), Row(array_prepend(data, 1)=[1])] +Example 1: Prepending a column value to an array column + +>>> from pyspark.sql import Row, functions as sf +>>> df = spark.createDataFrame([Row(c1=["b", "a", "c"], c2="c")]) +>>> df.select(sf.array_prepend(df.c1, df.c2)).show() ++-+ +|array_prepend(c1, c2)| ++-+ +| [c, b, a, c]| ++-+ + +Example 2: Prepending a numeric value to an array column + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([([1, 2, 3],)], ['data']) +>>> df.select(sf.array_prepend(df.data, 4)).show() ++--+ +|array_prepend(data, 4)| ++--+ +| [4, 1, 2, 3]| ++--+ + +Example 3: Prepending a null value to an array column + +>>> from pyspark.sql import functions as sf +>>> df = spark.createDataFrame([([1, 2, 3],)], ['data']) +>>> df.select(sf.array_prepend(df.data, None)).show() ++-+ +|array_prepend(data, NULL)| ++-+ +| [NULL, 1, 2, 3]| ++-+ + +Example 4: Prepending a value to a NULL array column + +>>> from pyspark.sql import functions as sf +>>> from pyspark.sql.types import ArrayType, IntegerType, StructType, StructField +>>> schema = StructType([ +... StructField("data", ArrayType(IntegerType()), True) +... ]) +>>> df = spark.createDataFrame([(None,)], schema=schema) +>>> df.select(sf.array_prepend(df.data, 4)).show() ++--+ +|array_prepend(data, 4)| ++--+ +| NULL| ++--+ + +Example 5: Prepending a value to an empty array + +>>> from pyspark.sql import functions as sf +>>> from pyspark.sql.types import ArrayType, IntegerType, StructType, StructField +>>> schema = StructType([ +... StructField("data", ArrayType(IntegerType()), True) +... ]) +>>> df = spark.createDataFrame([([],)], schema=schema) +>>> df.select(sf.
(spark) branch master updated: [SPARK-46476][BUILD][CORE][CONNECT] Move `IvyTestUtils` back to `src/test` directory
This is an automated email from the ASF dual-hosted git repository. yangjie01 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 2723bb11e3c7 [SPARK-46476][BUILD][CORE][CONNECT] Move `IvyTestUtils` back to `src/test` directory 2723bb11e3c7 is described below commit 2723bb11e3c7b9e2252e26cca904e2e53aaa68fb Author: yangjie01 AuthorDate: Thu Dec 21 20:50:34 2023 +0800 [SPARK-46476][BUILD][CORE][CONNECT] Move `IvyTestUtils` back to `src/test` directory ### What changes were proposed in this pull request? This pr move `IvyTestUtils` back to `src/test` directory because it has been in the `src/test` directory before the refactoring work of https://github.com/apache/spark/pull/43354. Meanwhile, in order to make the `core` and `connect-client-jvm` module use `IvyTestUtils` in the tests, this pr has added the corresponding Maven dependencies in the respective `pom.xml` files. ### Why are the changes needed? Move `IvyTestUtils` back to `src/test` directory ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #0 from LuciferYang/mv-IvyTestUtils-to-test-dir. Authored-by: yangjie01 Signed-off-by: yangjie01 --- .../{main => test}/scala/org/apache/spark/util/IvyTestUtils.scala | 0 connector/connect/client/jvm/pom.xml | 7 +++ core/pom.xml | 7 +++ 3 files changed, 14 insertions(+) diff --git a/common/utils/src/main/scala/org/apache/spark/util/IvyTestUtils.scala b/common/utils/src/test/scala/org/apache/spark/util/IvyTestUtils.scala similarity index 100% rename from common/utils/src/main/scala/org/apache/spark/util/IvyTestUtils.scala rename to common/utils/src/test/scala/org/apache/spark/util/IvyTestUtils.scala diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index eb98e4203a96..8057a33df178 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -82,6 +82,13 @@ scalacheck_${scala.binary.version} test + + org.apache.spark + spark-common-utils_${scala.binary.version} + ${project.version} + tests + test + com.typesafe diff --git a/core/pom.xml b/core/pom.xml index f2d1f4eb144a..c093213bd6b9 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -451,6 +451,13 @@ tests test + + org.apache.spark + spark-common-utils_${scala.binary.version} + ${project.version} + tests + test +